From 9478f4117de3a2d0ea40614ed4cb801918610724 Mon Sep 17 00:00:00 2001 From: manuzhang Date: Tue, 15 Mar 2016 16:15:16 +0800 Subject: [PATCH 001/346] [BEAM-79] add Gearpump runner --- runners/gearpump/README.md | 22 + runners/gearpump/pom.xml | 296 ++++++++++ .../gearpump/GearpumpPipelineOptions.java | 67 +++ .../gearpump/GearpumpPipelineResult.java | 42 ++ .../gearpump/GearpumpPipelineRunner.java | 193 +++++++ .../GearpumpPipelineRunnerRegistrar.java | 63 +++ .../gearpump/GearpumpPipelineTranslator.java | 139 +++++ .../runners/gearpump/TestGearpumpRunner.java | 64 +++ .../gearpump/examples/StreamingWordCount.java | 105 ++++ .../examples/UnboundedTextSource.java | 138 +++++ .../translators/CreateValuesTranslator.java | 49 ++ .../FlattenPCollectionTranslator.java | 47 ++ .../translators/GroupByKeyTranslator.java | 103 ++++ .../ParDoBoundMultiTranslator.java | 154 ++++++ .../translators/ParDoBoundTranslator.java | 54 ++ .../translators/ReadBoundedTranslator.java | 44 ++ .../translators/ReadUnboundedTranslator.java | 46 ++ .../translators/TransformTranslator.java | 31 ++ .../translators/TranslationContext.java | 95 ++++ .../translators/functions/DoFnFunction.java | 88 +++ .../translators/io/BoundedSourceWrapper.java | 44 ++ .../translators/io/GearpumpSource.java | 100 ++++ .../io/UnboundedSourceWrapper.java | 45 ++ .../gearpump/translators/io/ValuesSource.java | 164 ++++++ .../translators/utils/GearpumpDoFnRunner.java | 513 ++++++++++++++++++ .../utils/NoOpSideInputReader.java | 48 ++ .../translators/utils/NoOpStepContext.java | 71 +++ runners/pom.xml | 11 + .../apache/beam/sdk/testing/TestPipeline.java | 2 + 29 files changed, 2838 insertions(+) create mode 100644 runners/gearpump/README.md create mode 100644 runners/gearpump/pom.xml create mode 100644 runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpPipelineOptions.java create mode 100644 runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpPipelineResult.java create mode 100644 runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpPipelineRunner.java create mode 100644 runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpPipelineRunnerRegistrar.java create mode 100644 runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpPipelineTranslator.java create mode 100644 runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/TestGearpumpRunner.java create mode 100644 runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/examples/StreamingWordCount.java create mode 100644 runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/examples/UnboundedTextSource.java create mode 100644 runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/CreateValuesTranslator.java create mode 100644 runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/FlattenPCollectionTranslator.java create mode 100644 runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/GroupByKeyTranslator.java create mode 100644 runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/ParDoBoundMultiTranslator.java create mode 100644 runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/ParDoBoundTranslator.java create mode 100644 runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/ReadBoundedTranslator.java create mode 100644 runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/ReadUnboundedTranslator.java create mode 100644 runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/TransformTranslator.java create mode 100644 runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/TranslationContext.java create mode 100644 runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/functions/DoFnFunction.java create mode 100644 runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/io/BoundedSourceWrapper.java create mode 100644 runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/io/GearpumpSource.java create mode 100644 runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/io/UnboundedSourceWrapper.java create mode 100644 runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/io/ValuesSource.java create mode 100644 runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/utils/GearpumpDoFnRunner.java create mode 100644 runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/utils/NoOpSideInputReader.java create mode 100644 runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/utils/NoOpStepContext.java diff --git a/runners/gearpump/README.md b/runners/gearpump/README.md new file mode 100644 index 0000000000000..ad043faea4c7e --- /dev/null +++ b/runners/gearpump/README.md @@ -0,0 +1,22 @@ + + +## Gearpump Beam Runner + +The Gearpump Beam runner allows users to execute pipelines written using the Apache Beam programming API with Apache Gearpump (incubating) as an execution engine. \ No newline at end of file diff --git a/runners/gearpump/pom.xml b/runners/gearpump/pom.xml new file mode 100644 index 0000000000000..c725daecba805 --- /dev/null +++ b/runners/gearpump/pom.xml @@ -0,0 +1,296 @@ + + + + + 4.0.0 + + + org.apache.beam + beam-runners-parent + 0.2.0-incubating-SNAPSHOT + ../pom.xml + + + beam-runners-gearpump + + Apache Beam :: Runners :: Gearpump + jar + + + UTF-8 + UTF-8 + 0.8.1-SNAPSHOT + + + + + apache.snapshots + Apache Development Snapshot Repository + https://repository.apache.org/content/repositories/snapshots/ + + false + + + true + + + + gearpump-shaded-repo + Vincent at Bintray + http://dl.bintray.com/fvunicorn/maven + + + + + + org.apache.gearpump + gearpump-streaming_2.11 + ${gearpump.version} + provided + + + org.apache.gearpump + gearpump-core_2.11 + ${gearpump.version} + provided + + + com.google.code.findbugs + jsr305 + + + + + org.apache.gearpump + gearpump-daemon_2.11 + ${gearpump.version} + provided + + + org.apache.gearpump + gearpump-experimental-cgroup_2.11 + + + + + com.typesafe + config + provided + 1.3.0 + + + org.scala-lang + scala-library + 2.11.8 + provided + + + org.apache.beam + beam-sdks-java-core + + + org.slf4j + slf4j-jdk14 + + + com.google.collections + google-collections + + + + + org.apache.beam + beam-runners-core-java + + + com.google.code.findbugs + annotations + 3.0.1 + + + org.slf4j + slf4j-api + + + joda-time + joda-time + + + com.fasterxml.jackson.core + jackson-core + + + com.fasterxml.jackson.core + jackson-databind + + + com.fasterxml.jackson.core + jackson-annotations + + + com.google.http-client + google-http-client + + + com.google.guava + guava + + + junit + junit + test + + + org.hamcrest + hamcrest-all + test + + + org.apache.beam + beam-sdks-java-core + test-jar + test + + + org.slf4j + slf4j-jdk14 + + + + + org.mockito + mockito-all + test + + + com.google.auto.service + auto-service + 1.0-rc2 + + + + + + + + org.apache.maven.plugins + maven-jar-plugin + + + + true + true + + + + + + + + org.apache.maven.plugins + maven-compiler-plugin + + + + + maven-failsafe-plugin + + + + integration-test + verify + + + + + 1 + -Dlog4j.configuration=log4j-test.properties -XX:-UseGCOverheadLimit + + + + + + org.apache.maven.plugins + maven-surefire-plugin + 2.19.1 + + + runnable-on-service-tests + + org.apache.beam.sdk.testing.RunnableOnService + none + true + + org.apache.beam:beam-sdks-java-core + org.apache.beam:beam-runners-java-core + + + + + org.apache.beam.sdk.io.BigQueryIOTest, + org.apache.beam.sdk.io.CountingInputTest, + org.apache.beam.sdk.io.CountingSourceTest, + org.apache.beam.sdk.testing.PAssertTest, + org.apache.beam.sdk.transforms.ApproximateUniqueTest, + org.apache.beam.sdk.transforms.CombineTest, + org.apache.beam.sdk.transforms.CombineFnsTest, + org.apache.beam.sdk.transforms.CountTest, + org.apache.beam.sdk.transforms.FlattenTest, + org.apache.beam.sdk.transforms.ParDoTest, + org.apache.beam.sdk.transforms.SampleTest, + org.apache.beam.sdk.transforms.ViewTest, + org.apache.beam.sdk.transforms.join.CoGroupByKeyTest + + + + org.apache.beam.sdk.transforms.windowing.WindowingTest, + org.apache.beam.sdk.util.ReshuffleTest + + + + + [ + "--runner=TestGearpumpRunner", + "--streaming=true" + ] + + + + + + + + + + org.apache.maven.plugins + maven-assembly-plugin + + + jar-with-dependencies + + + + + + org.apache.maven.plugins + maven-checkstyle-plugin + + + + \ No newline at end of file diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpPipelineOptions.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpPipelineOptions.java new file mode 100644 index 0000000000000..5b6ee960745d8 --- /dev/null +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpPipelineOptions.java @@ -0,0 +1,67 @@ +/* + * 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.gearpump; + +import org.apache.beam.sdk.options.Default; +import org.apache.beam.sdk.options.Description; +import org.apache.beam.sdk.options.PipelineOptions; + +import com.fasterxml.jackson.annotation.JsonIgnore; + +import org.apache.gearpump.cluster.client.ClientContext; +import org.apache.gearpump.cluster.embedded.EmbeddedCluster; + +import java.util.Map; + +/** + * Options that configure the Gearpump pipeline. + */ +public interface GearpumpPipelineOptions extends PipelineOptions { + + @Description("set unique application name for Gearpump runner") + void setApplicationName(String name); + + String getApplicationName(); + + @Description("set parallelism for Gearpump processor") + void setParallelism(int parallelism); + + @Default.Integer(1) + int getParallelism(); + + @Description("register Kryo serializers") + void setSerializers(Map serializers); + + @JsonIgnore + Map getSerializers(); + + @Description("set EmbeddedCluster for tests") + void setEmbeddedCluster(EmbeddedCluster cluster); + + @JsonIgnore + EmbeddedCluster getEmbeddedCluster(); + + void setClientContext(ClientContext clientContext); + + @JsonIgnore + @Description("get client context to query application status") + ClientContext getClientContext(); + +} + diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpPipelineResult.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpPipelineResult.java new file mode 100644 index 0000000000000..bc2714727259b --- /dev/null +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpPipelineResult.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.runners.gearpump; + +import org.apache.beam.sdk.Pipeline; +import org.apache.beam.sdk.PipelineResult; +import org.apache.beam.sdk.runners.AggregatorRetrievalException; +import org.apache.beam.sdk.runners.AggregatorValues; +import org.apache.beam.sdk.transforms.Aggregator; + +/** + * Result of executing a {@link Pipeline} with Gearpump. + */ +public class GearpumpPipelineResult implements PipelineResult { + @Override + public State getState() { + return null; + } + + @Override + public AggregatorValues getAggregatorValues(Aggregator aggregator) + throws AggregatorRetrievalException { + throw new AggregatorRetrievalException( + "PipelineResult getAggregatorValues not supported in Gearpump pipeline", + new UnsupportedOperationException()); + } +} diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpPipelineRunner.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpPipelineRunner.java new file mode 100644 index 0000000000000..660d7039737d3 --- /dev/null +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpPipelineRunner.java @@ -0,0 +1,193 @@ +/* + * 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.gearpump; + +import org.apache.beam.runners.gearpump.translators.TranslationContext; +import org.apache.beam.sdk.Pipeline; +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.transforms.Create; +import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.transforms.Flatten; +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.Window; +import org.apache.beam.sdk.transforms.windowing.WindowFn; +import org.apache.beam.sdk.util.AssignWindows; +import org.apache.beam.sdk.util.WindowingStrategy; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.PCollectionList; +import org.apache.beam.sdk.values.PInput; +import org.apache.beam.sdk.values.POutput; + +import com.typesafe.config.Config; +import com.typesafe.config.ConfigValueFactory; + +import org.apache.gearpump.cluster.ClusterConfig; +import org.apache.gearpump.cluster.UserConfig; +import org.apache.gearpump.cluster.client.ClientContext; +import org.apache.gearpump.cluster.embedded.EmbeddedCluster; +import org.apache.gearpump.streaming.dsl.javaapi.JavaStreamApp; + +import java.util.HashMap; +import java.util.Map; + +/** + * A {@link PipelineRunner} that executes the operations in the + * pipeline by first translating them to Gearpump Stream DSL + * and then executing them on a Gearpump cluster. + *

> + * This is based on DataflowPipelineRunner. + */ +@SuppressWarnings({"rawtypes", "unchecked"}) +public class GearpumpPipelineRunner extends PipelineRunner { + + private final GearpumpPipelineOptions options; + + private static final String GEARPUMP_SERIALIZERS = "gearpump.serializers"; + private static final String DEFAULT_APPNAME = "beam_gearpump_app"; + + public GearpumpPipelineRunner(GearpumpPipelineOptions options) { + this.options = options; + } + + public static GearpumpPipelineRunner fromOptions(PipelineOptions options) { + GearpumpPipelineOptions pipelineOptions = + PipelineOptionsValidator.validate(GearpumpPipelineOptions.class, options); + return new GearpumpPipelineRunner(pipelineOptions); + } + + + public OutputT apply( + PTransform transform, InputT input) { + if (Window.Bound.class.equals(transform.getClass())) { + return (OutputT) super.apply( + new AssignWindowsAndSetStrategy((Window.Bound) transform), input); + } else if (Flatten.FlattenPCollectionList.class.equals(transform.getClass()) + && ((PCollectionList) input).size() == 0) { + return (OutputT) Pipeline.applyTransform(input, Create.of()); + } else if (Create.Values.class.equals(transform.getClass())) { + return (OutputT) PCollection + .createPrimitiveOutputInternal( + input.getPipeline(), + WindowingStrategy.globalDefault(), + PCollection.IsBounded.BOUNDED); + } else { + return super.apply(transform, input); + } + } + + @Override + public GearpumpPipelineResult run(Pipeline pipeline) { + String appName = options.getApplicationName(); + if (null == appName) { + appName = DEFAULT_APPNAME; + } + Config config = registerSerializers(ClusterConfig.defaultConfig(), + options.getSerializers()); + ClientContext clientContext = getClientContext(options, config); + options.setClientContext(clientContext); + JavaStreamApp streamApp = new JavaStreamApp( + appName, clientContext, UserConfig.empty()); + TranslationContext translationContext = new TranslationContext(streamApp, options); + GearpumpPipelineTranslator translator = new GearpumpPipelineTranslator(translationContext); + translator.translate(pipeline); + streamApp.run(); + + return null; + } + + private ClientContext getClientContext(GearpumpPipelineOptions options, Config config) { + EmbeddedCluster cluster = options.getEmbeddedCluster(); + if (cluster != null) { + return cluster.newClientContext(); + } else { + return ClientContext.apply(config); + } + } + + /** + * register class with default kryo serializers. + */ + private Config registerSerializers(Config config, Map userSerializers) { + Map serializers = new HashMap<>(); + serializers.put("org.apache.beam.sdk.util.WindowedValue$TimestampedValueInSingleWindow", ""); + serializers.put("org.apache.beam.sdk.transforms.windowing.PaneInfo", ""); + serializers.put("org.apache.beam.sdk.transforms.windowing.PaneInfo$Timing", ""); + serializers.put("org.joda.time.Instant", ""); + serializers.put("org.apache.beam.sdk.values.KV", ""); + serializers.put("org.apache.beam.sdk.transforms.windowing.IntervalWindow", ""); + serializers.put("org.apache.beam.sdk.values.TimestampedValue", ""); + if (userSerializers != null && !userSerializers.isEmpty()) { + serializers.putAll(userSerializers); + } + return config.withValue(GEARPUMP_SERIALIZERS, ConfigValueFactory.fromMap(serializers)); + } + + + /** + * copied from DirectPipelineRunner. + * used to replace Window.Bound till window function is added to Gearpump Stream DSL + */ + private static class AssignWindowsAndSetStrategy + extends PTransform, PCollection> { + + private final Window.Bound wrapped; + + public AssignWindowsAndSetStrategy(Window.Bound wrapped) { + this.wrapped = wrapped; + } + + @Override + public PCollection apply(PCollection input) { + WindowingStrategy outputStrategy = + wrapped.getOutputStrategyInternal(input.getWindowingStrategy()); + + WindowFn windowFn = + (WindowFn) outputStrategy.getWindowFn(); + + if (!windowFn.isNonMerging()) { + throw new UnsupportedOperationException( + "merging window is not supported in Gearpump pipeline"); + } + + // 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 { + @Override + public void processElement(ProcessContext c) { + c.output(c.element()); + } + } +} diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpPipelineRunnerRegistrar.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpPipelineRunnerRegistrar.java new file mode 100644 index 0000000000000..2b9e89e6a4c42 --- /dev/null +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpPipelineRunnerRegistrar.java @@ -0,0 +1,63 @@ +/* + * 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.gearpump; + +import org.apache.beam.sdk.options.PipelineOptions; +import org.apache.beam.sdk.options.PipelineOptionsRegistrar; +import org.apache.beam.sdk.runners.PipelineRunner; +import org.apache.beam.sdk.runners.PipelineRunnerRegistrar; + +import com.google.auto.service.AutoService; +import com.google.common.collect.ImmutableList; + +/** + * Contains the {@link PipelineRunnerRegistrar} and {@link PipelineOptionsRegistrar} for the + * {@link GearpumpPipelineRunner}. + * + * {@link AutoService} will register Gearpump's implementations of the {@link PipelineRunner} + * and {@link PipelineOptions} as available pipeline runner services. + */ +public class GearpumpPipelineRunnerRegistrar { + private GearpumpPipelineRunnerRegistrar() { } + + /** + * Registers the {@link GearpumpPipelineRunner}. + */ + @AutoService(PipelineRunnerRegistrar.class) + public static class Runner implements PipelineRunnerRegistrar { + + @Override + public Iterable>> getPipelineRunners() { + return ImmutableList.>>of( + TestGearpumpRunner.class); + } + } + + /** + * Registers the {@link GearpumpPipelineOptions}. + */ + @AutoService(PipelineOptionsRegistrar.class) + public static class Options implements PipelineOptionsRegistrar { + + @Override + public Iterable> getPipelineOptions() { + return ImmutableList.>of(GearpumpPipelineOptions.class); + } + } +} diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpPipelineTranslator.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpPipelineTranslator.java new file mode 100644 index 0000000000000..59f0df7e8d3e5 --- /dev/null +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpPipelineTranslator.java @@ -0,0 +1,139 @@ +/* + * 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.gearpump; + +import org.apache.beam.runners.gearpump.translators.CreateValuesTranslator; +import org.apache.beam.runners.gearpump.translators.FlattenPCollectionTranslator; +import org.apache.beam.runners.gearpump.translators.GroupByKeyTranslator; +import org.apache.beam.runners.gearpump.translators.ParDoBoundMultiTranslator; +import org.apache.beam.runners.gearpump.translators.ParDoBoundTranslator; +import org.apache.beam.runners.gearpump.translators.ReadBoundedTranslator; +import org.apache.beam.runners.gearpump.translators.ReadUnboundedTranslator; +import org.apache.beam.runners.gearpump.translators.TransformTranslator; +import org.apache.beam.runners.gearpump.translators.TranslationContext; +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.transforms.Create; +import org.apache.beam.sdk.transforms.Flatten; +import org.apache.beam.sdk.transforms.GroupByKey; +import org.apache.beam.sdk.transforms.PTransform; +import org.apache.beam.sdk.transforms.ParDo; +import org.apache.beam.sdk.values.PValue; + +import org.apache.gearpump.util.Graph; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.HashMap; +import java.util.Map; + +/** + * {@link GearpumpPipelineTranslator} knows how to translate {@link Pipeline} objects + * into Gearpump {@link Graph}. + */ +@SuppressWarnings({"rawtypes", "unchecked"}) +public class GearpumpPipelineTranslator implements Pipeline.PipelineVisitor { + + private static final Logger LOG = LoggerFactory.getLogger( + GearpumpPipelineTranslator.class); + + /** + * A map from {@link PTransform} subclass to the corresponding + * {@link TransformTranslator} to use to translate that transform. + */ + private static final Map, TransformTranslator> + transformTranslators = new HashMap<>(); + + private final TranslationContext translationContext; + + static { + // register TransformTranslators + registerTransformTranslator(ParDo.Bound.class, new ParDoBoundTranslator()); + registerTransformTranslator(Read.Unbounded.class, new ReadUnboundedTranslator()); + registerTransformTranslator(Read.Bounded.class, new ReadBoundedTranslator()); + registerTransformTranslator(GroupByKey.class, new GroupByKeyTranslator()); + registerTransformTranslator(Flatten.FlattenPCollectionList.class, + new FlattenPCollectionTranslator()); + registerTransformTranslator(ParDo.BoundMulti.class, new ParDoBoundMultiTranslator()); + registerTransformTranslator(Create.Values.class, new CreateValuesTranslator()); + } + + public GearpumpPipelineTranslator(TranslationContext translationContext) { + this.translationContext = translationContext; + } + + public void translate(Pipeline pipeline) { + pipeline.traverseTopologically(this); + } + + @Override + public CompositeBehavior enterCompositeTransform(TransformTreeNode node) { + LOG.debug("entering composite transform {}", node.getTransform()); + return CompositeBehavior.ENTER_TRANSFORM; + } + + @Override + public void leaveCompositeTransform(TransformTreeNode node) { + LOG.debug("leaving composite transform {}", node.getTransform()); + } + + @Override + public void visitPrimitiveTransform(TransformTreeNode node) { + LOG.debug("visiting transform {}", node.getTransform()); + PTransform transform = node.getTransform(); + TransformTranslator translator = getTransformTranslator(transform.getClass()); + if (null == translator) { + throw new IllegalStateException( + "no translator registered for " + transform); + } + translationContext.setCurrentTransform(node); + translator.translate(transform, translationContext); + } + + @Override + public void visitValue(PValue value, TransformTreeNode producer) { + LOG.debug("visiting value {}", value); + } + + /** + * Records that instances of the specified PTransform class + * should be translated by default by the corresponding + * {@link TransformTranslator}. + */ + private static void registerTransformTranslator( + Class transformClass, + TransformTranslator transformTranslator) { + if (transformTranslators.put(transformClass, transformTranslator) != null) { + throw new IllegalArgumentException( + "defining multiple translators for " + transformClass); + } + } + + /** + * Returns the {@link TransformTranslator} to use for instances of the + * specified PTransform class, or null if none registered. + */ + private + TransformTranslator getTransformTranslator(Class transformClass) { + return transformTranslators.get(transformClass); + } + + +} diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/TestGearpumpRunner.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/TestGearpumpRunner.java new file mode 100644 index 0000000000000..cedd31ff4ab21 --- /dev/null +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/TestGearpumpRunner.java @@ -0,0 +1,64 @@ +/* + * 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.gearpump; + +import org.apache.beam.sdk.Pipeline; +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.transforms.PTransform; +import org.apache.beam.sdk.values.PInput; +import org.apache.beam.sdk.values.POutput; + +import org.apache.gearpump.cluster.embedded.EmbeddedCluster; + +/** + * Gearpump {@link PipelineRunner} for tests, which uses {@link EmbeddedCluster}. + */ +public class TestGearpumpRunner extends PipelineRunner { + + private final GearpumpPipelineRunner delegate; + private final EmbeddedCluster cluster; + + private TestGearpumpRunner(GearpumpPipelineOptions options) { + cluster = EmbeddedCluster.apply(); + cluster.start(); + options.setEmbeddedCluster(cluster); + delegate = GearpumpPipelineRunner.fromOptions(options); + } + + public static TestGearpumpRunner fromOptions(PipelineOptions options) { + GearpumpPipelineOptions pipelineOptions = + PipelineOptionsValidator.validate(GearpumpPipelineOptions.class, options); + return new TestGearpumpRunner(pipelineOptions); + } + + @Override + public GearpumpPipelineResult run(Pipeline pipeline) { + GearpumpPipelineResult result = delegate.run(pipeline); + cluster.stop(); + return result; + } + + @Override + public + OutputT apply(PTransform transform, InputT input) { + return delegate.apply(transform, input); + } +} diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/examples/StreamingWordCount.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/examples/StreamingWordCount.java new file mode 100644 index 0000000000000..c51289d5fefe1 --- /dev/null +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/examples/StreamingWordCount.java @@ -0,0 +1,105 @@ +/* + * 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.gearpump.examples; + +import org.apache.beam.runners.gearpump.GearpumpPipelineOptions; +import org.apache.beam.runners.gearpump.GearpumpPipelineRunner; +import org.apache.beam.sdk.Pipeline; +import org.apache.beam.sdk.io.Read; +import org.apache.beam.sdk.options.PipelineOptionsFactory; +import org.apache.beam.sdk.transforms.Aggregator; +import org.apache.beam.sdk.transforms.Count; +import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.transforms.ParDo; +import org.apache.beam.sdk.transforms.Sum; +import org.apache.beam.sdk.transforms.windowing.FixedWindows; +import org.apache.beam.sdk.transforms.windowing.Window; +import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.PCollection; + +import org.apache.gearpump.cluster.client.ClientContext; +import org.joda.time.Duration; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + + +/** + * streaming word count example on Gearpump runner. + */ +public class StreamingWordCount { + + static class ExtractWordsFn extends DoFn { + private final Aggregator emptyLines = + createAggregator("emptyLines", new Sum.SumLongFn()); + + @Override + 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); + } + } + } + } + + static class FormatAsStringFn extends DoFn, String> { + private static final Logger LOG = LoggerFactory.getLogger(FormatAsStringFn.class); + + @Override + public void processElement(ProcessContext c) { + String row = c.element().getKey() + + " - " + c.element().getValue() + + " @ " + c.timestamp().toString(); + LOG.debug("output {}", row); + c.output(row); + } + } + + + public static void main(String[] args) { + GearpumpPipelineOptions options = PipelineOptionsFactory.fromArgs(args).withValidation() + .as(GearpumpPipelineOptions.class); + options.setApplicationName("StreamingWordCount"); + options.setRunner(GearpumpPipelineRunner.class); + options.setParallelism(1); + Pipeline p = Pipeline.create(options); + + PCollection> wordCounts = + p.apply(Read.from(new UnboundedTextSource())) + .apply(ParDo.of(new ExtractWordsFn())) + .apply(Window.into(FixedWindows.of(Duration.standardSeconds(10)))) + .apply(Count.perElement()); + + wordCounts.apply(ParDo.of(new FormatAsStringFn())); + + p.run(); + + ClientContext clientContext = options.getClientContext(); + clientContext.close(); + + } +} diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/examples/UnboundedTextSource.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/examples/UnboundedTextSource.java new file mode 100644 index 0000000000000..caf066c9710a5 --- /dev/null +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/examples/UnboundedTextSource.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.gearpump.examples; + +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.coders.StringUtf8Coder; +import org.apache.beam.sdk.io.UnboundedSource; +import org.apache.beam.sdk.options.PipelineOptions; + +import org.joda.time.Instant; + +import java.io.IOException; +import java.io.Serializable; +import java.util.Collections; +import java.util.List; +import java.util.NoSuchElementException; + +import javax.annotation.Nullable; + +/** + * unbounded source that reads from text. + */ +public class UnboundedTextSource extends UnboundedSource { + + @Override + public List> generateInitialSplits( + int desiredNumSplits, PipelineOptions options) throws Exception { + return Collections.>singletonList(this); + } + + @Override + public UnboundedReader createReader(PipelineOptions options, + @Nullable CheckpointMark checkpointMark) { + return new UnboundedTextReader(this); + } + + @Nullable + @Override + public Coder getCheckpointMarkCoder() { + return null; + } + + @Override + public void validate() { + } + + @Override + public Coder getDefaultOutputCoder() { + return StringUtf8Coder.of(); + } + + /** + * reads from text. + */ + public static class UnboundedTextReader extends UnboundedReader implements Serializable { + + private static final long serialVersionUID = 7526472295622776147L; + + private final UnboundedTextSource source; + + private final String[] texts = new String[]{"foo foo foo bar bar", "foo foo bar bar bar"}; + private long index = 0; + + private String currentRecord; + + private Instant currentTimestamp; + + public UnboundedTextReader(UnboundedTextSource source) { + this.source = source; + } + + @Override + public boolean start() throws IOException { + currentRecord = texts[0]; + currentTimestamp = new Instant(0); + return true; + } + + @Override + public boolean advance() throws IOException { + index++; + currentRecord = texts[(int) index % (texts.length)]; + currentTimestamp = new Instant(index * 1000); + + return true; + } + + @Override + public byte[] getCurrentRecordId() throws NoSuchElementException { + return new byte[0]; + } + + @Override + public String getCurrent() throws NoSuchElementException { + return this.currentRecord; + } + + @Override + public Instant getCurrentTimestamp() throws NoSuchElementException { + return currentTimestamp; + } + + @Override + public void close() throws IOException { + } + + @Override + public Instant getWatermark() { + return currentTimestamp; + } + + @Override + public CheckpointMark getCheckpointMark() { + return null; + } + + @Override + public UnboundedSource getCurrentSource() { + return this.source; + } + } +} diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/CreateValuesTranslator.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/CreateValuesTranslator.java new file mode 100644 index 0000000000000..452127aa2c9b3 --- /dev/null +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/CreateValuesTranslator.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.runners.gearpump.translators; + +import org.apache.beam.runners.gearpump.translators.io.UnboundedSourceWrapper; +import org.apache.beam.runners.gearpump.translators.io.ValuesSource; +import org.apache.beam.sdk.coders.CannotProvideCoderException; +import org.apache.beam.sdk.io.UnboundedSource; +import org.apache.beam.sdk.transforms.Create; +import org.apache.beam.sdk.util.WindowedValue; + +import org.apache.gearpump.streaming.dsl.javaapi.JavaStream; + +/** + * Wraps elements from Create.Values into an {@link UnboundedSource}. + * mainly used for test + */ +public class CreateValuesTranslator implements TransformTranslator> { + + @Override + public void translate(Create.Values transform, TranslationContext context) { + try { + UnboundedSourceWrapper unboundedSourceWrapper = new UnboundedSourceWrapper<>( + new ValuesSource<>(transform.getElements(), + transform.getDefaultOutputCoder(context.getInput(transform))), + context.getPipelineOptions()); + JavaStream> sourceStream = context.getSourceStream(unboundedSourceWrapper); + context.setOutputStream(context.getOutput(transform), sourceStream); + } catch (CannotProvideCoderException e) { + throw new RuntimeException(e); + } + } +} diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/FlattenPCollectionTranslator.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/FlattenPCollectionTranslator.java new file mode 100644 index 0000000000000..b06d5a840817e --- /dev/null +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/FlattenPCollectionTranslator.java @@ -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. + */ + +package org.apache.beam.runners.gearpump.translators; + +import org.apache.beam.sdk.transforms.Flatten; +import org.apache.beam.sdk.values.PCollection; + +import org.apache.gearpump.streaming.dsl.javaapi.JavaStream; + +/** + * Flatten.FlattenPCollectionList is translated to Gearpump merge function. + * Note only two-way merge is working now + */ +public class FlattenPCollectionTranslator implements + TransformTranslator> { + + @Override + public void translate(Flatten.FlattenPCollectionList transform, TranslationContext context) { + JavaStream merged = null; + System.out.println("PCollectionList size " + context.getInput(transform).size()); + for (PCollection collection : context.getInput(transform).getAll()) { + JavaStream inputStream = context.getInputStream(collection); + if (null == merged) { + merged = inputStream; + } else { + merged = merged.merge(inputStream, transform.getName()); + } + } + context.setOutputStream(context.getOutput(transform), merged); + } +} diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/GroupByKeyTranslator.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/GroupByKeyTranslator.java new file mode 100644 index 0000000000000..f36b908f842d9 --- /dev/null +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/GroupByKeyTranslator.java @@ -0,0 +1,103 @@ +/* + * 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.gearpump.translators; + +import org.apache.beam.sdk.transforms.GroupByKey; +import org.apache.beam.sdk.transforms.windowing.BoundedWindow; +import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.KV; + +import com.google.common.collect.Iterables; + +import org.apache.gearpump.streaming.dsl.javaapi.JavaStream; +import org.apache.gearpump.streaming.javaapi.dsl.functions.FlatMapFunction; +import org.apache.gearpump.streaming.javaapi.dsl.functions.GroupByFunction; +import org.apache.gearpump.streaming.javaapi.dsl.functions.MapFunction; +import org.apache.gearpump.streaming.javaapi.dsl.functions.ReduceFunction; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.Iterator; +import java.util.List; + +/** + * {@link GroupByKey} is translated to Gearpump groupBy function. + */ +public class GroupByKeyTranslator implements TransformTranslator> { + @Override + public void translate(GroupByKey transform, TranslationContext context) { + JavaStream>> inputStream = + context.getInputStream(context.getInput(transform)); + int parallelism = context.getPipelineOptions().getParallelism(); + JavaStream>>> outputStream = inputStream + .flatMap(new KeyedByKeyAndWindow(), "keyed_by_Key_and_Window") + .groupBy(new GroupByKeyAndWindow(), parallelism, "group_by_Key_and_Window") + .map(new ExtractKeyValue(), "extract_Key_and_Value") + .reduce(new MergeValue(), "merge_value"); + + context.setOutputStream(context.getOutput(transform), outputStream); + } + + private static class KeyedByKeyAndWindow implements + FlatMapFunction>, WindowedValue, V>>> { + + @Override + public Iterator, V>>> apply(WindowedValue> wv) { + List, V>>> ret = new ArrayList<>(wv.getWindows().size + ()); + for (BoundedWindow window : wv.getWindows()) { + KV keyWin = KV.of(wv.getValue().getKey(), window); + ret.add(WindowedValue.of(KV.of(keyWin, wv.getValue().getValue()), + wv.getTimestamp(), window, wv.getPane())); + } + return ret.iterator(); + } + } + + private static class GroupByKeyAndWindow implements + GroupByFunction, V>>, KV> { + + @Override + public KV apply(WindowedValue, V>> wv) { + return wv.getValue().getKey(); + } + } + + private static class ExtractKeyValue implements + MapFunction, V>>, + WindowedValue>>> { + @Override + public WindowedValue>> apply(WindowedValue, V>> wv) { + return WindowedValue.of(KV.of(wv.getValue().getKey().getKey(), + (Iterable) Collections.singletonList(wv.getValue().getValue())), + wv.getTimestamp(), wv.getWindows(), wv.getPane()); + } + } + + private static class MergeValue implements + ReduceFunction>>> { + @Override + public WindowedValue>> apply(WindowedValue>> wv1, + WindowedValue>> wv2) { + return WindowedValue.of(KV.of(wv1.getValue().getKey(), + Iterables.concat(wv1.getValue().getValue(), wv2.getValue().getValue())), + wv1.getTimestamp(), wv1.getWindows(), wv1.getPane()); + } + } +} diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/ParDoBoundMultiTranslator.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/ParDoBoundMultiTranslator.java new file mode 100644 index 0000000000000..af5bcbcf4c885 --- /dev/null +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/ParDoBoundMultiTranslator.java @@ -0,0 +1,154 @@ +/* + * 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.gearpump.translators; + +import org.apache.beam.runners.gearpump.GearpumpPipelineOptions; +import org.apache.beam.runners.gearpump.translators.utils.GearpumpDoFnRunner; +import org.apache.beam.runners.gearpump.translators.utils.NoOpSideInputReader; +import org.apache.beam.runners.gearpump.translators.utils.NoOpStepContext; +import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.transforms.ParDo; +import org.apache.beam.sdk.util.DoFnRunner; +import org.apache.beam.sdk.util.DoFnRunners; +import org.apache.beam.sdk.util.SideInputReader; +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.TupleTag; +import org.apache.beam.sdk.values.TupleTagList; + +import com.google.common.collect.Lists; + +import org.apache.gearpump.streaming.dsl.javaapi.JavaStream; +import org.apache.gearpump.streaming.javaapi.dsl.functions.FilterFunction; +import org.apache.gearpump.streaming.javaapi.dsl.functions.FlatMapFunction; +import org.apache.gearpump.streaming.javaapi.dsl.functions.MapFunction; + +import java.util.Iterator; +import java.util.List; +import java.util.Map; + +/** + * {@link ParDo.BoundMulti} is translated to Gearpump flatMap function + * with {@link DoFn} wrapped in {@link DoFnMultiFunction}. The outputs are + * further filtered with Gearpump filter function by output tag + */ +@SuppressWarnings({"rawtypes", "unchecked"}) +public class ParDoBoundMultiTranslator implements + TransformTranslator> { + + @Override + public void translate(ParDo.BoundMulti transform, TranslationContext context) { + PCollection inputT = (PCollection) context.getInput(transform); + JavaStream> inputStream = context.getInputStream(inputT); + Map, PCollection> outputs = context.getOutput(transform).getAll(); + + JavaStream, OutputT>>> outputStream = inputStream.flatMap( + new DoFnMultiFunction<>( + context.getPipelineOptions(), + transform.getFn(), + transform.getMainOutputTag(), + transform.getSideOutputTags(), + inputT.getWindowingStrategy(), + new NoOpSideInputReader() + ), transform.getName()); + for (Map.Entry, PCollection> output : outputs.entrySet()) { + JavaStream> taggedStream = outputStream + .filter(new FilterByOutputTag<>((TupleTag) output.getKey()) + , "filter_by_output_tag") + .map(new ExtractOutput(), "extract output"); + + context.setOutputStream(output.getValue(), taggedStream); + } + } + + /** + * Gearpump {@link FlatMapFunction} wrapper over Beam {@link DoFnMultiFunction}. + */ + private static class DoFnMultiFunction implements + FlatMapFunction, WindowedValue, OutputT>>>, + DoFnRunners.OutputManager { + + private final DoFnRunner doFnRunner; + private final List, OutputT>>> outputs = Lists + .newArrayList(); + + public DoFnMultiFunction( + GearpumpPipelineOptions pipelineOptions, + DoFn doFn, + TupleTag mainOutputTag, + TupleTagList sideOutputTags, + WindowingStrategy windowingStrategy, + SideInputReader sideInputReader) { + this.doFnRunner = new GearpumpDoFnRunner<>( + pipelineOptions, + doFn, + sideInputReader, + this, + mainOutputTag, + sideOutputTags.getAll(), + new NoOpStepContext(), + windowingStrategy + ); + } + + @Override + public Iterator, OutputT>>> apply(WindowedValue wv) { + doFnRunner.startBundle(); + doFnRunner.processElement(wv); + doFnRunner.finishBundle(); + + return outputs.iterator(); + } + + @Override + public void output(TupleTag tag, WindowedValue output) { + KV, OutputT> kv = KV.of((TupleTag) tag, + (OutputT) output.getValue()); + outputs.add(WindowedValue.of(kv, output.getTimestamp(), + output.getWindows(), output.getPane())); + } + } + + private static class FilterByOutputTag implements + FilterFunction, OutputT>>> { + + private final TupleTag tupleTag; + + public FilterByOutputTag(TupleTag tupleTag) { + this.tupleTag = tupleTag; + } + + @Override + public boolean apply(WindowedValue, OutputT>> wv) { + return wv.getValue().getKey().equals(tupleTag); + } + } + + private static class ExtractOutput implements + MapFunction, OutputT>>, WindowedValue> { + + @Override + public WindowedValue apply(WindowedValue, OutputT>> wv) { + return WindowedValue.of(wv.getValue().getValue(), wv.getTimestamp(), + wv.getWindows(), wv.getPane()); + } + } +} diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/ParDoBoundTranslator.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/ParDoBoundTranslator.java new file mode 100644 index 0000000000000..689bc08e0f47e --- /dev/null +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/ParDoBoundTranslator.java @@ -0,0 +1,54 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.beam.runners.gearpump.translators; + +import org.apache.beam.runners.gearpump.translators.functions.DoFnFunction; +import org.apache.beam.runners.gearpump.translators.utils.NoOpSideInputReader; +import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.transforms.ParDo; +import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.util.WindowingStrategy; +import org.apache.beam.sdk.values.PCollection; + +import org.apache.gearpump.streaming.dsl.javaapi.JavaStream; + + +/** + * {@link ParDo.Bound} is translated to Gearpump flatMap function + * with {@link DoFn} wrapped in {@link DoFnFunction}. + */ +public class ParDoBoundTranslator implements + TransformTranslator> { + + @Override + public void translate(ParDo.Bound transform, TranslationContext context) { + DoFn doFn = transform.getFn(); + PCollection output = context.getOutput(transform); + WindowingStrategy windowingStrategy = output.getWindowingStrategy(); + + DoFnFunction doFnFunction = new DoFnFunction<>(context.getPipelineOptions(), + doFn, windowingStrategy, new NoOpSideInputReader()); + JavaStream> inputStream = + context.getInputStream(context.getInput(transform)); + JavaStream> outputStream = + inputStream.flatMap(doFnFunction, transform.getName()); + + context.setOutputStream(context.getOutput(transform), outputStream); + } +} diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/ReadBoundedTranslator.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/ReadBoundedTranslator.java new file mode 100644 index 0000000000000..478d58f87b1ee --- /dev/null +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/ReadBoundedTranslator.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.runners.gearpump.translators; + +import org.apache.beam.runners.gearpump.translators.io.BoundedSourceWrapper; +import org.apache.beam.sdk.io.BoundedSource; +import org.apache.beam.sdk.io.Read; +import org.apache.beam.sdk.util.WindowedValue; +import org.apache.gearpump.streaming.dsl.javaapi.JavaStream; +import org.apache.gearpump.streaming.source.DataSource; + +/** + * {@link Read.Bounded} is translated to Gearpump source function + * and {@link BoundedSource} is wrapped into Gearpump {@link DataSource}. + */ +public class ReadBoundedTranslator implements TransformTranslator> { + + @Override + public void translate(Read.Bounded transform, TranslationContext context) { + BoundedSource boundedSource = transform.getSource(); + BoundedSourceWrapper sourceWrapper = new BoundedSourceWrapper<>(boundedSource, + context.getPipelineOptions()); + JavaStream> sourceStream = context.getSourceStream(sourceWrapper); + + context.setOutputStream(context.getOutput(transform), sourceStream); + } + +} diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/ReadUnboundedTranslator.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/ReadUnboundedTranslator.java new file mode 100644 index 0000000000000..7e12a9c629fdb --- /dev/null +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/ReadUnboundedTranslator.java @@ -0,0 +1,46 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.beam.runners.gearpump.translators; + +import org.apache.beam.runners.gearpump.translators.io.UnboundedSourceWrapper; +import org.apache.beam.sdk.io.Read; +import org.apache.beam.sdk.io.UnboundedSource; +import org.apache.beam.sdk.util.WindowedValue; + +import org.apache.gearpump.streaming.dsl.javaapi.JavaStream; +import org.apache.gearpump.streaming.source.DataSource; + +/** + * {@link Read.Unbounded} is translated to Gearpump source function + * and {@link UnboundedSource} is wrapped into Gearpump {@link DataSource}. + */ + +public class ReadUnboundedTranslator implements TransformTranslator> { + + @Override + public void translate(Read.Unbounded transform, TranslationContext context) { + UnboundedSource unboundedSource = transform.getSource(); + UnboundedSourceWrapper unboundedSourceWrapper = new UnboundedSourceWrapper<>( + unboundedSource, context.getPipelineOptions()); + JavaStream> sourceStream = context.getSourceStream(unboundedSourceWrapper); + + context.setOutputStream(context.getOutput(transform), sourceStream); + } + +} diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/TransformTranslator.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/TransformTranslator.java new file mode 100644 index 0000000000000..1ed6d5daa28e1 --- /dev/null +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/TransformTranslator.java @@ -0,0 +1,31 @@ +/* + * 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.gearpump.translators; + + +import org.apache.beam.sdk.transforms.PTransform; + +import java.io.Serializable; + +/** + * translates {@link PTransform} to Gearpump functions. + */ +public interface TransformTranslator extends Serializable { + void translate(T transform, TranslationContext context); +} diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/TranslationContext.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/TranslationContext.java new file mode 100644 index 0000000000000..b9b2c7aeeb913 --- /dev/null +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/TranslationContext.java @@ -0,0 +1,95 @@ +/* + * 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.gearpump.translators; + +import static com.google.common.base.Preconditions.checkArgument; + +import org.apache.beam.runners.gearpump.GearpumpPipelineOptions; +import org.apache.beam.sdk.runners.TransformTreeNode; +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; + +import org.apache.gearpump.cluster.UserConfig; +import org.apache.gearpump.streaming.dsl.javaapi.JavaStream; +import org.apache.gearpump.streaming.dsl.javaapi.JavaStreamApp; +import org.apache.gearpump.streaming.source.DataSource; + +import java.util.HashMap; +import java.util.Map; + +/** + * Maintains context data for {@link TransformTranslator}s. + */ +@SuppressWarnings({"rawtypes", "unchecked"}) +public class TranslationContext { + + private final JavaStreamApp streamApp; + private final GearpumpPipelineOptions pipelineOptions; + private AppliedPTransform currentTransform; + private final Map> streams = new HashMap<>(); + + public TranslationContext(JavaStreamApp streamApp, GearpumpPipelineOptions pipelineOptions) { + this.streamApp = streamApp; + this.pipelineOptions = pipelineOptions; + + } + + public void setCurrentTransform(TransformTreeNode treeNode) { + this.currentTransform = AppliedPTransform.of(treeNode.getFullName(), + treeNode.getInput(), treeNode.getOutput(), (PTransform) treeNode.getTransform()); + } + + public GearpumpPipelineOptions getPipelineOptions() { + return pipelineOptions; + } + + public JavaStream getInputStream(PValue input) { + return (JavaStream) streams.get(input); + } + + public void setOutputStream(PValue output, JavaStream outputStream) { + if (!streams.containsKey(output)) { + streams.put(output, outputStream); + } + } + + public InputT getInput(PTransform transform) { + return (InputT) getCurrentTransform(transform).getInput(); + } + + public OutputT getOutput(PTransform transform) { + return (OutputT) getCurrentTransform(transform).getOutput(); + } + + private AppliedPTransform getCurrentTransform(PTransform transform) { + checkArgument( + currentTransform != null && currentTransform.getTransform() == transform, + "can only be called with current transform"); + return currentTransform; + } + + public JavaStream getSourceStream(DataSource dataSource) { + return streamApp.source(dataSource, pipelineOptions.getParallelism(), + UserConfig.empty(), "source"); + } + +} diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/functions/DoFnFunction.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/functions/DoFnFunction.java new file mode 100644 index 0000000000000..088fc14c10f8e --- /dev/null +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/functions/DoFnFunction.java @@ -0,0 +1,88 @@ +/* + * 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.gearpump.translators.functions; + +import org.apache.beam.runners.gearpump.GearpumpPipelineOptions; +import org.apache.beam.runners.gearpump.translators.utils.GearpumpDoFnRunner; +import org.apache.beam.runners.gearpump.translators.utils.NoOpStepContext; +import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.util.DoFnRunner; +import org.apache.beam.sdk.util.DoFnRunners; +import org.apache.beam.sdk.util.SideInputReader; +import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.util.WindowingStrategy; +import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.TupleTagList; + +import com.google.api.client.util.Lists; + +import org.apache.gearpump.streaming.javaapi.dsl.functions.FlatMapFunction; + +import java.util.Iterator; +import java.util.List; + +/** + * Gearpump {@link FlatMapFunction} wrapper over Beam {@link DoFn}. + */ +public class DoFnFunction implements + FlatMapFunction, WindowedValue>, DoFnRunners.OutputManager { + + private final TupleTag mainTag = new TupleTag() { + }; + private final DoFnRunner doFnRunner; + private List> outputs = Lists.newArrayList(); + + public DoFnFunction( + GearpumpPipelineOptions pipelineOptions, + DoFn doFn, + WindowingStrategy windowingStrategy, + SideInputReader sideInputReader) { + this.doFnRunner = new GearpumpDoFnRunner<>( + pipelineOptions, + doFn, + sideInputReader, + this, + mainTag, + TupleTagList.empty().getAll(), + new NoOpStepContext(), + windowingStrategy + ); + } + + @Override + public Iterator> apply(WindowedValue value) { + outputs = Lists.newArrayList(); + + doFnRunner.startBundle(); + doFnRunner.processElement(value); + doFnRunner.finishBundle(); + + return outputs.iterator(); + } + + @SuppressWarnings({"rawtypes", "unchecked"}) + @Override + public void output(TupleTag tag, WindowedValue output) { + if (mainTag.equals(tag)) { + outputs.add((WindowedValue) output); + } else { + throw new RuntimeException("output is not of main tag"); + } + } +} diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/io/BoundedSourceWrapper.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/io/BoundedSourceWrapper.java new file mode 100644 index 0000000000000..f25d113e5c9d1 --- /dev/null +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/io/BoundedSourceWrapper.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.runners.gearpump.translators.io; + +import org.apache.beam.sdk.io.BoundedSource; +import org.apache.beam.sdk.io.Source; +import org.apache.beam.sdk.options.PipelineOptions; + +import java.io.IOException; + +/** + * wrapper over BoundedSource for Gearpump DataSource API. + */ +public class BoundedSourceWrapper extends GearpumpSource { + + private final BoundedSource source; + + public BoundedSourceWrapper(BoundedSource source, PipelineOptions options) { + super(options); + this.source = source; + } + + + @Override + protected Source.Reader createReader(PipelineOptions options) throws IOException { + return source.createReader(options); + } +} diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/io/GearpumpSource.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/io/GearpumpSource.java new file mode 100644 index 0000000000000..892ccc3b39e91 --- /dev/null +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/io/GearpumpSource.java @@ -0,0 +1,100 @@ +/* + * 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.gearpump.translators.io; + +import org.apache.beam.sdk.io.Source; +import org.apache.beam.sdk.options.PipelineOptions; +import org.apache.beam.sdk.transforms.windowing.GlobalWindow; +import org.apache.beam.sdk.transforms.windowing.PaneInfo; +import org.apache.beam.sdk.util.WindowedValue; + +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.ObjectMapper; + +import org.apache.gearpump.Message; +import org.apache.gearpump.streaming.source.DataSource; +import org.apache.gearpump.streaming.task.TaskContext; + +import org.joda.time.Instant; + +import java.io.IOException; + +/** + * common methods for {@link BoundedSourceWrapper} and {@link UnboundedSourceWrapper}. + */ +public abstract class GearpumpSource implements DataSource { + + protected final byte[] serializedOptions; + + protected Source.Reader reader; + protected boolean available = false; + + public GearpumpSource(PipelineOptions options) { + try { + this.serializedOptions = new ObjectMapper().writeValueAsBytes(options); + } catch (JsonProcessingException e) { + throw new RuntimeException(e); + } + } + + protected abstract Source.Reader createReader(PipelineOptions options) throws IOException; + + @Override + public void open(TaskContext context, long startTime) { + try { + PipelineOptions options = new ObjectMapper() + .readValue(serializedOptions, PipelineOptions.class); + this.reader = createReader(options); + } catch (Exception e) { + throw new RuntimeException(e); + } finally { + close(); + } + } + @Override + public Message read() { + Message message = null; + try { + if (available) { + T data = reader.getCurrent(); + Instant timestamp = reader.getCurrentTimestamp(); + available = reader.advance(); + message = Message.apply( + WindowedValue.of(data, timestamp, GlobalWindow.INSTANCE, PaneInfo.NO_FIRING), + timestamp.getMillis()); + } + } catch (Exception e) { + throw new RuntimeException(e); + } finally { + close(); + } + return message; + } + + @Override + public void close() { + try { + if (reader != null) { + reader.close(); + } + } catch (IOException e) { + throw new RuntimeException(e); + } + } +} diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/io/UnboundedSourceWrapper.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/io/UnboundedSourceWrapper.java new file mode 100644 index 0000000000000..b39f29f4219a5 --- /dev/null +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/io/UnboundedSourceWrapper.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.runners.gearpump.translators.io; + +import org.apache.beam.sdk.io.Source; +import org.apache.beam.sdk.io.UnboundedSource; +import org.apache.beam.sdk.options.PipelineOptions; + +import java.io.IOException; + +/** + * wrapper over UnboundedSource for Gearpump DataSource API. + */ +public class UnboundedSourceWrapper + extends GearpumpSource { + + private final UnboundedSource source; + + public UnboundedSourceWrapper(UnboundedSource source, + PipelineOptions options) { + super(options); + this.source = source; + } + + @Override + protected Source.Reader createReader(PipelineOptions options) throws IOException { + return source.createReader(options, null); + } +} diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/io/ValuesSource.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/io/ValuesSource.java new file mode 100644 index 0000000000000..24055f7e4d0ad --- /dev/null +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/io/ValuesSource.java @@ -0,0 +1,164 @@ +/* + * 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.gearpump.translators.io; + +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.io.UnboundedSource; +import org.apache.beam.sdk.options.PipelineOptions; + +import org.joda.time.Instant; + +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.io.Serializable; +import java.util.Collections; +import java.util.Iterator; +import java.util.LinkedList; +import java.util.List; +import java.util.NoSuchElementException; + +import javax.annotation.Nullable; + +/** + * unbounded source that reads from a Java {@link Iterable}. + */ +public class ValuesSource extends UnboundedSource { + + private final Iterable values; + private final Coder coder; + + public ValuesSource(Iterable values, Coder coder) { + this.values = encode(values, coder); + this.coder = coder; + } + + private Iterable encode(Iterable values, Coder coder) { + List bytes = new LinkedList<>(); + for (T t: values) { + try { + ByteArrayOutputStream stream = new ByteArrayOutputStream(); + coder.encode(t, stream, Coder.Context.OUTER); + bytes.add(stream.toByteArray()); + } catch (IOException e) { + throw new RuntimeException(e); + } + } + return bytes; + } + + @Override + public java.util.List> generateInitialSplits( + int desiredNumSplits, PipelineOptions options) throws Exception { + return Collections.singletonList(this); + } + + @Override + public UnboundedReader createReader(PipelineOptions options, + @Nullable CheckpointMark checkpointMark) { + return new ValuesReader<>(values, coder, this); + } + + @Nullable + @Override + public Coder getCheckpointMarkCoder() { + return null; + } + + @Override + public void validate() { + } + + @Override + public Coder getDefaultOutputCoder() { + return coder; + } + + private static class ValuesReader extends UnboundedReader implements Serializable { + + private final Iterable values; + private final Coder coder; + private final UnboundedSource source; + private transient Iterator iterator; + private T current; + + public ValuesReader(Iterable values, Coder coder, + UnboundedSource source) { + this.values = values; + this.coder = coder; + this.source = source; + } + + private T decode(byte[] bytes) throws IOException { + ByteArrayInputStream inputStream = new ByteArrayInputStream(bytes); + try { + return coder.decode(inputStream, Coder.Context.OUTER); + } finally { + inputStream.close(); + } + } + + @Override + public boolean start() throws IOException { + if (null == iterator) { + iterator = values.iterator(); + } + return advance(); + } + + @Override + public boolean advance() throws IOException { + if (iterator.hasNext()) { + current = decode(iterator.next()); + return true; + } else { + return false; + } + } + + @Override + public T getCurrent() throws NoSuchElementException { + return current; + } + + @Override + public Instant getCurrentTimestamp() throws NoSuchElementException { + return Instant.now(); + } + + @Override + public void close() throws IOException { + } + + @Override + public Instant getWatermark() { + return Instant.now(); + } + + @Override + public CheckpointMark getCheckpointMark() { + return null; + } + + @Override + public UnboundedSource getCurrentSource() { + return source; + } + } +} diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/utils/GearpumpDoFnRunner.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/utils/GearpumpDoFnRunner.java new file mode 100644 index 0000000000000..608ad7c85135b --- /dev/null +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/utils/GearpumpDoFnRunner.java @@ -0,0 +1,513 @@ +/* + * 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.gearpump.translators.utils; + +import org.apache.beam.runners.gearpump.GearpumpPipelineOptions; +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.Combine; +import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.transforms.windowing.BoundedWindow; +import org.apache.beam.sdk.transforms.windowing.GlobalWindow; +import org.apache.beam.sdk.transforms.windowing.GlobalWindows; +import org.apache.beam.sdk.transforms.windowing.PaneInfo; +import org.apache.beam.sdk.transforms.windowing.WindowFn; +import org.apache.beam.sdk.util.DoFnRunner; +import org.apache.beam.sdk.util.DoFnRunners; +import org.apache.beam.sdk.util.ExecutionContext; +import org.apache.beam.sdk.util.SideInputReader; +import org.apache.beam.sdk.util.SimpleDoFnRunner; +import org.apache.beam.sdk.util.SystemDoFnInternal; +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.StateInternals; +import org.apache.beam.sdk.values.PCollectionView; +import org.apache.beam.sdk.values.TupleTag; + +import com.google.common.base.Preconditions; +import com.google.common.collect.Iterables; +import com.google.common.collect.Sets; + +import org.joda.time.Instant; + +import java.io.IOException; +import java.io.Serializable; +import java.util.Collection; +import java.util.Iterator; +import java.util.List; +import java.util.Set; + +/** + * a serializable {@link SimpleDoFnRunner}. + */ +public class GearpumpDoFnRunner implements DoFnRunner, + Serializable { + + private final DoFn fn; + private final transient PipelineOptions options; + private final SideInputReader sideInputReader; + private final DoFnRunners.OutputManager outputManager; + private final TupleTag mainOutputTag; + private final List> sideOutputTags; + private final ExecutionContext.StepContext stepContext; + private final WindowFn windowFn; + private DoFnContext context; + + public GearpumpDoFnRunner( + GearpumpPipelineOptions pipelineOptions, + DoFn doFn, + SideInputReader sideInputReader, + DoFnRunners.OutputManager outputManager, + TupleTag mainOutputTag, + List> sideOutputTags, + ExecutionContext.StepContext stepContext, + WindowingStrategy windowingStrategy) { + this.fn = doFn; + this.options = pipelineOptions; + this.sideInputReader = sideInputReader; + this.outputManager = outputManager; + this.mainOutputTag = mainOutputTag; + this.sideOutputTags = sideOutputTags; + this.stepContext = stepContext; + this.windowFn = windowingStrategy == null ? null : windowingStrategy.getWindowFn(); + } + + @Override + public void startBundle() { + // This can contain user code. Wrap it in case it throws an exception. + try { + if (null == context) { + this.context = new DoFnContext<>( + options, + fn, + sideInputReader, + outputManager, + mainOutputTag, + sideOutputTags, + stepContext, + windowFn + ); + } + fn.startBundle(context); + } catch (Throwable t) { + // Exception in user code. + throw wrapUserCodeException(t); + } + } + + @Override + public void processElement(WindowedValue elem) { + if (elem.getWindows().size() <= 1 + || (!DoFn.RequiresWindowAccess.class.isAssignableFrom(fn.getClass()) + && context.sideInputReader.isEmpty())) { + invokeProcessElement(elem); + } else { + // We could modify the windowed value (and the processContext) to + // avoid repeated allocations, but this is more straightforward. + for (BoundedWindow window : elem.getWindows()) { + invokeProcessElement(WindowedValue.of( + elem.getValue(), elem.getTimestamp(), window, elem.getPane())); + } + } + } + + @Override + public void finishBundle() { + // This can contain user code. Wrap it in case it throws an exception. + try { + fn.finishBundle(context); + } catch (Throwable t) { + // Exception in user code. + throw wrapUserCodeException(t); + } + } + + private void invokeProcessElement(WindowedValue elem) { + final DoFn.ProcessContext processContext = + new DoFnProcessContext<>(fn, context, elem); + // This can contain user code. Wrap it in case it throws an exception. + try { + fn.processElement(processContext); + } catch (Exception ex) { + throw wrapUserCodeException(ex); + } + } + + private RuntimeException wrapUserCodeException(Throwable t) { + throw UserCodeException.wrapIf(!isSystemDoFn(), t); + } + + private boolean isSystemDoFn() { + return fn.getClass().isAnnotationPresent(SystemDoFnInternal.class); + } + + /** + * A concrete implementation of {@code DoFn.Context} used for running a {@link DoFn}. + * + * @param the type of the DoFn's (main) input elements + * @param the type of the DoFn's (main) output elements + */ + private static class DoFnContext + extends DoFn.Context { + private static final int MAX_SIDE_OUTPUTS = 1000; + + final transient PipelineOptions options; + final DoFn fn; + final SideInputReader sideInputReader; + final DoFnRunners.OutputManager outputManager; + final TupleTag mainOutputTag; + final ExecutionContext.StepContext stepContext; + final WindowFn windowFn; + + /** + * The set of known output tags, some of which may be undeclared, so we can throw an + * exception when it exceeds {@link #MAX_SIDE_OUTPUTS}. + */ + private final Set> outputTags; + + public DoFnContext(PipelineOptions options, + DoFn fn, + SideInputReader sideInputReader, + DoFnRunners.OutputManager outputManager, + TupleTag mainOutputTag, + List> sideOutputTags, + ExecutionContext.StepContext stepContext, + WindowFn windowFn) { + fn.super(); + this.options = options; + this.fn = fn; + this.sideInputReader = sideInputReader; + this.outputManager = outputManager; + this.mainOutputTag = mainOutputTag; + this.outputTags = Sets.newHashSet(); + + outputTags.add(mainOutputTag); + for (TupleTag sideOutputTag : sideOutputTags) { + outputTags.add(sideOutputTag); + } + + this.stepContext = stepContext; + this.windowFn = windowFn; + super.setupDelegateAggregators(); + } + + ////////////////////////////////////////////////////////////////////////////// + + @Override + public PipelineOptions getPipelineOptions() { + return options; + } + + WindowedValue makeWindowedValue( + T output, Instant timestamp, Collection windows, PaneInfo pane) { + final Instant inputTimestamp = timestamp; + + if (timestamp == null) { + timestamp = BoundedWindow.TIMESTAMP_MIN_VALUE; + } + + if (windows == null) { + try { + // The windowFn can never succeed at accessing the element, so its type does not + // matter here + @SuppressWarnings("unchecked") + WindowFn objectWindowFn = (WindowFn) windowFn; + windows = objectWindowFn.assignWindows(objectWindowFn.new AssignContext() { + @Override + public Object element() { + throw new UnsupportedOperationException( + "WindowFn attempted to access input element when none was available"); + } + + @Override + public Instant timestamp() { + if (inputTimestamp == null) { + throw new UnsupportedOperationException( + "WindowFn attempted to access input timestamp when none was available"); + } + return inputTimestamp; + } + + @Override + public BoundedWindow window() { + throw new UnsupportedOperationException( + "WindowFn attempted to access input windows when none were available"); + } + }); + } catch (Exception e) { + throw UserCodeException.wrap(e); + } + } + + return WindowedValue.of(output, timestamp, windows, pane); + } + + public T sideInput(PCollectionView view, BoundedWindow mainInputWindow) { + if (!sideInputReader.contains(view)) { + throw new IllegalArgumentException("calling sideInput() with unknown view"); + } + BoundedWindow sideInputWindow = + view.getWindowingStrategyInternal().getWindowFn().getSideInputWindow(mainInputWindow); + return sideInputReader.get(view, sideInputWindow); + } + + void outputWindowedValue( + OutputT output, + Instant timestamp, + Collection windows, + PaneInfo pane) { + outputWindowedValue(makeWindowedValue(output, timestamp, windows, pane)); + } + + void outputWindowedValue(WindowedValue windowedElem) { + outputManager.output(mainOutputTag, windowedElem); + if (stepContext != null) { + stepContext.noteOutput(windowedElem); + } + } + + protected void sideOutputWindowedValue(TupleTag tag, + T output, + Instant timestamp, + Collection windows, + PaneInfo pane) { + sideOutputWindowedValue(tag, makeWindowedValue(output, timestamp, windows, pane)); + } + + protected void sideOutputWindowedValue(TupleTag tag, WindowedValue windowedElem) { + if (!outputTags.contains(tag)) { + // This tag wasn't declared nor was it seen before during this execution. + // Thus, this must be a new, undeclared and unconsumed output. + // To prevent likely user errors, enforce the limit on the number of side + // outputs. + if (outputTags.size() >= MAX_SIDE_OUTPUTS) { + throw new IllegalArgumentException( + "the number of side outputs has exceeded a limit of " + MAX_SIDE_OUTPUTS); + } + outputTags.add(tag); + } + + outputManager.output(tag, windowedElem); + if (stepContext != null) { + stepContext.noteSideOutput(tag, windowedElem); + } + } + + // Following implementations of output, outputWithTimestamp, and sideOutput + // are only accessible in DoFn.startBundle and DoFn.finishBundle, and will be shadowed by + // ProcessContext's versions in DoFn.processElement. + @Override + public void output(OutputT output) { + outputWindowedValue(output, null, null, PaneInfo.NO_FIRING); + } + + @Override + public void outputWithTimestamp(OutputT output, Instant timestamp) { + outputWindowedValue(output, timestamp, null, PaneInfo.NO_FIRING); + } + + @Override + public void sideOutput(TupleTag tag, T output) { + Preconditions.checkNotNull(tag, "TupleTag passed to sideOutput cannot be null"); + sideOutputWindowedValue(tag, output, null, null, PaneInfo.NO_FIRING); + } + + @Override + public void sideOutputWithTimestamp(TupleTag tag, T output, Instant timestamp) { + Preconditions.checkNotNull(tag, "TupleTag passed to sideOutputWithTimestamp cannot be null"); + sideOutputWindowedValue(tag, output, timestamp, null, PaneInfo.NO_FIRING); + } + + @Override + protected Aggregator createAggregatorInternal( + String name, Combine.CombineFn combiner) { + Preconditions.checkNotNull(combiner, + "Combiner passed to createAggregator cannot be null"); + throw new UnsupportedOperationException("aggregator not supported in Gearpump runner"); + } + } + + + /** + * A concrete implementation of {@code DoFn.ProcessContext} used for + * running a {@link DoFn} over a single element. + * + * @param the type of the DoFn's (main) input elements + * @param the type of the DoFn's (main) output elements + */ + private static class DoFnProcessContext + extends DoFn.ProcessContext { + + + final DoFn fn; + final DoFnContext context; + final WindowedValue windowedValue; + + public DoFnProcessContext(DoFn fn, + DoFnContext context, + WindowedValue windowedValue) { + fn.super(); + this.fn = fn; + this.context = context; + this.windowedValue = windowedValue; + } + + @Override + public PipelineOptions getPipelineOptions() { + return context.getPipelineOptions(); + } + + @Override + public InputT element() { + return windowedValue.getValue(); + } + + @Override + public T sideInput(PCollectionView view) { + Preconditions.checkNotNull(view, "View passed to sideInput cannot be null"); + Iterator windowIter = windows().iterator(); + BoundedWindow window; + if (!windowIter.hasNext()) { + if (context.windowFn instanceof GlobalWindows) { + // TODO: Remove this once GroupByKeyOnly no longer outputs elements + // without windows + window = GlobalWindow.INSTANCE; + } else { + throw new IllegalStateException( + "sideInput called when main input element is not in any windows"); + } + } else { + window = windowIter.next(); + if (windowIter.hasNext()) { + throw new IllegalStateException( + "sideInput called when main input element is in multiple windows"); + } + } + return context.sideInput(view, window); + } + + @Override + public BoundedWindow window() { + if (!(fn instanceof DoFn.RequiresWindowAccess)) { + throw new UnsupportedOperationException( + "window() is only available in the context of a DoFn marked as RequiresWindow."); + } + return Iterables.getOnlyElement(windows()); + } + + @Override + public PaneInfo pane() { + return windowedValue.getPane(); + } + + @Override + public void output(OutputT output) { + context.outputWindowedValue(windowedValue.withValue(output)); + } + + @Override + public void outputWithTimestamp(OutputT output, Instant timestamp) { + context.outputWindowedValue(output, timestamp, + windowedValue.getWindows(), windowedValue.getPane()); + } + + @Override + public void sideOutput(TupleTag tag, T output) { + Preconditions.checkNotNull(tag, "Tag passed to sideOutput cannot be null"); + context.sideOutputWindowedValue(tag, windowedValue.withValue(output)); + } + + @Override + public void sideOutputWithTimestamp(TupleTag tag, T output, Instant timestamp) { + Preconditions.checkNotNull(tag, "Tag passed to sideOutputWithTimestamp cannot be null"); + context.sideOutputWindowedValue( + tag, output, timestamp, windowedValue.getWindows(), windowedValue.getPane()); + } + + @Override + public Instant timestamp() { + return windowedValue.getTimestamp(); + } + + public Collection windows() { + return windowedValue.getWindows(); + } + + @Override + public WindowingInternals windowingInternals() { + return new WindowingInternals() { + @Override + public void outputWindowedValue(OutputT output, Instant timestamp, + Collection windows, PaneInfo pane) { + context.outputWindowedValue(output, timestamp, windows, pane); + } + + @Override + public Collection windows() { + return windowedValue.getWindows(); + } + + @Override + public PaneInfo pane() { + return windowedValue.getPane(); + } + + @Override + 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(); + } + + @Override + public T sideInput(PCollectionView view, BoundedWindow mainInputWindow) { + return context.sideInput(view, mainInputWindow); + } + }; + } + + @Override + protected Aggregator + createAggregatorInternal( + String name, Combine.CombineFn combiner) { + return context.createAggregatorInternal(name, combiner); + } + } +} diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/utils/NoOpSideInputReader.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/utils/NoOpSideInputReader.java new file mode 100644 index 0000000000000..600ebfb225d09 --- /dev/null +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/utils/NoOpSideInputReader.java @@ -0,0 +1,48 @@ +/* + * 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.gearpump.translators.utils; + +import org.apache.beam.sdk.transforms.windowing.BoundedWindow; +import org.apache.beam.sdk.util.SideInputReader; +import org.apache.beam.sdk.values.PCollectionView; + +import java.io.Serializable; + +import javax.annotation.Nullable; + +/** + * no-op side input reader. + */ +public class NoOpSideInputReader implements SideInputReader, Serializable { + @Nullable + @Override + public T get(PCollectionView view, BoundedWindow window) { + return null; + } + + @Override + public boolean contains(PCollectionView view) { + return false; + } + + @Override + public boolean isEmpty() { + return false; + } +} diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/utils/NoOpStepContext.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/utils/NoOpStepContext.java new file mode 100644 index 0000000000000..ce0935a38081d --- /dev/null +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/utils/NoOpStepContext.java @@ -0,0 +1,71 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.beam.runners.gearpump.translators.utils; + +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; +import org.apache.beam.sdk.values.TupleTag; + +import java.io.IOException; +import java.io.Serializable; + +/** + * serializable {@link ExecutionContext.StepContext} that basically does nothing. + */ +public class NoOpStepContext implements ExecutionContext.StepContext, Serializable { + + @Override + public String getStepName() { + throw new UnsupportedOperationException(); + } + + @Override + public String getTransformName() { + throw new UnsupportedOperationException(); + } + + @Override + public void noteOutput(WindowedValue output) { + } + + @Override + public void noteSideOutput(TupleTag tag, WindowedValue output) { + } + + @Override + public void writePCollectionViewData(TupleTag tag, + Iterable> data, + Coder>> dataCoder, W window, Coder windowCoder) throws + IOException { + } + + @Override + public StateInternals stateInternals() { + throw new UnsupportedOperationException(); + } + + @Override + public TimerInternals timerInternals() { + throw new UnsupportedOperationException(); + } +} diff --git a/runners/pom.xml b/runners/pom.xml index 612d4936fe34a..3f985c5b4ac3e 100644 --- a/runners/pom.xml +++ b/runners/pom.xml @@ -39,6 +39,17 @@ spark + + + java8 + + [1.8,) + + + gearpump + + + 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 0de3024c2cdc2..eda78112c6b39 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 @@ -100,6 +100,7 @@ public static TestPipeline create() { } public static TestPipeline fromOptions(PipelineOptions options) { + System.out.println(options); return new TestPipeline(PipelineRunner.fromOptions(options), options); } @@ -138,6 +139,7 @@ public static PipelineOptions testingPipelineOptions() { @Nullable String beamTestPipelineOptions = System.getProperty(PROPERTY_BEAM_TEST_PIPELINE_OPTIONS); + System.out.println("options " + beamTestPipelineOptions); PipelineOptions options = Strings.isNullOrEmpty(beamTestPipelineOptions) ? PipelineOptionsFactory.create() From 1672b5483e029292816397248dc6fe63bf51f4af Mon Sep 17 00:00:00 2001 From: manuzhang Date: Sat, 23 Jul 2016 14:10:15 +0800 Subject: [PATCH 002/346] move integration tests to profile --- runners/gearpump/pom.xml | 107 ++++++++++++++++++++++----------------- 1 file changed, 61 insertions(+), 46 deletions(-) diff --git a/runners/gearpump/pom.xml b/runners/gearpump/pom.xml index c725daecba805..85994ad3e94c5 100644 --- a/runners/gearpump/pom.xml +++ b/runners/gearpump/pom.xml @@ -57,6 +57,67 @@ + + + local-runnable-on-service-tests + false + + + + org.apache.maven.plugins + maven-surefire-plugin + 2.19.1 + + + runnable-on-service-tests + + org.apache.beam.sdk.testing.RunnableOnService + none + true + + org.apache.beam:beam-sdks-java-core + org.apache.beam:beam-runners-java-core + + + + + org.apache.beam.sdk.io.BigQueryIOTest, + org.apache.beam.sdk.io.CountingInputTest, + org.apache.beam.sdk.io.CountingSourceTest, + org.apache.beam.sdk.testing.PAssertTest, + org.apache.beam.sdk.transforms.ApproximateUniqueTest, + org.apache.beam.sdk.transforms.CombineTest, + org.apache.beam.sdk.transforms.CombineFnsTest, + org.apache.beam.sdk.transforms.CountTest, + org.apache.beam.sdk.transforms.FlattenTest, + org.apache.beam.sdk.transforms.ParDoTest, + org.apache.beam.sdk.transforms.SampleTest, + org.apache.beam.sdk.transforms.ViewTest, + org.apache.beam.sdk.transforms.join.CoGroupByKeyTest + + + + org.apache.beam.sdk.transforms.windowing.WindowingTest, + org.apache.beam.sdk.util.ReshuffleTest + + + + + [ + "--runner=TestGearpumpRunner", + "--streaming=true" + ] + + + + + + + + + + + org.apache.gearpump @@ -228,52 +289,6 @@ org.apache.maven.plugins maven-surefire-plugin - 2.19.1 - - - runnable-on-service-tests - - org.apache.beam.sdk.testing.RunnableOnService - none - true - - org.apache.beam:beam-sdks-java-core - org.apache.beam:beam-runners-java-core - - - - - org.apache.beam.sdk.io.BigQueryIOTest, - org.apache.beam.sdk.io.CountingInputTest, - org.apache.beam.sdk.io.CountingSourceTest, - org.apache.beam.sdk.testing.PAssertTest, - org.apache.beam.sdk.transforms.ApproximateUniqueTest, - org.apache.beam.sdk.transforms.CombineTest, - org.apache.beam.sdk.transforms.CombineFnsTest, - org.apache.beam.sdk.transforms.CountTest, - org.apache.beam.sdk.transforms.FlattenTest, - org.apache.beam.sdk.transforms.ParDoTest, - org.apache.beam.sdk.transforms.SampleTest, - org.apache.beam.sdk.transforms.ViewTest, - org.apache.beam.sdk.transforms.join.CoGroupByKeyTest - - - - org.apache.beam.sdk.transforms.windowing.WindowingTest, - org.apache.beam.sdk.util.ReshuffleTest - - - - - [ - "--runner=TestGearpumpRunner", - "--streaming=true" - ] - - - - - From 276a2e106aa1a5736666fc2eb2426b640f63cf68 Mon Sep 17 00:00:00 2001 From: manuzhang Date: Thu, 28 Jul 2016 16:30:13 +0800 Subject: [PATCH 003/346] add package-info.java --- .../gearpump/examples/package-info.java | 22 +++++++++++++++++++ .../beam/runners/gearpump/package-info.java | 22 +++++++++++++++++++ .../translators/functions/package-info.java | 22 +++++++++++++++++++ .../gearpump/translators/io/package-info.java | 22 +++++++++++++++++++ .../gearpump/translators/package-info.java | 22 +++++++++++++++++++ .../translators/utils/package-info.java | 22 +++++++++++++++++++ 6 files changed, 132 insertions(+) create mode 100644 runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/examples/package-info.java create mode 100644 runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/package-info.java create mode 100644 runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/functions/package-info.java create mode 100644 runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/io/package-info.java create mode 100644 runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/package-info.java create mode 100644 runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/utils/package-info.java diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/examples/package-info.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/examples/package-info.java new file mode 100644 index 0000000000000..a62a6c08a2582 --- /dev/null +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/examples/package-info.java @@ -0,0 +1,22 @@ +/* + * 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. + */ + +/** + * Examples showcase Beam application over Gearpump runner. + */ +package org.apache.beam.runners.gearpump.examples; diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/package-info.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/package-info.java new file mode 100644 index 0000000000000..501361602b4cb --- /dev/null +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/package-info.java @@ -0,0 +1,22 @@ +/* + * 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. + */ + +/** + * Internal implementation of the Beam runner for Apache Gearpump. + */ +package org.apache.beam.runners.gearpump; diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/functions/package-info.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/functions/package-info.java new file mode 100644 index 0000000000000..cba23637d354a --- /dev/null +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/functions/package-info.java @@ -0,0 +1,22 @@ +/* + * 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. + */ + +/** + * Gearpump specific wrappers for Beam DoFn. + */ +package org.apache.beam.runners.gearpump.translators.functions; diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/io/package-info.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/io/package-info.java new file mode 100644 index 0000000000000..dfdf51a03bbc3 --- /dev/null +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/io/package-info.java @@ -0,0 +1,22 @@ +/* + * 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. + */ + +/** + * Gearpump specific wrappers for Beam I/O. + */ +package org.apache.beam.runners.gearpump.translators.io; diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/package-info.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/package-info.java new file mode 100644 index 0000000000000..612096a5d70a5 --- /dev/null +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/package-info.java @@ -0,0 +1,22 @@ +/* + * 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. + */ + +/** + * Gearpump specific translators. + */ +package org.apache.beam.runners.gearpump.translators; diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/utils/package-info.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/utils/package-info.java new file mode 100644 index 0000000000000..ab2a6eace94f9 --- /dev/null +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/utils/package-info.java @@ -0,0 +1,22 @@ +/* + * 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. + */ + +/** + * Utilities for translators. + */ +package org.apache.beam.runners.gearpump.translators.utils; From 40be715a696bb1218b209f7ad9a979b7e5d088d3 Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Wed, 10 Aug 2016 10:26:57 -0700 Subject: [PATCH 004/346] Update Gearpump runner version to 0.3.0-incubating --- runners/gearpump/pom.xml | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/runners/gearpump/pom.xml b/runners/gearpump/pom.xml index 85994ad3e94c5..8fc46357ca1e4 100644 --- a/runners/gearpump/pom.xml +++ b/runners/gearpump/pom.xml @@ -23,7 +23,7 @@ org.apache.beam beam-runners-parent - 0.2.0-incubating-SNAPSHOT + 0.3.0-incubating-SNAPSHOT ../pom.xml @@ -308,4 +308,4 @@ - \ No newline at end of file + From bc1b354949416db3b52c4f37c66968bdb86f0813 Mon Sep 17 00:00:00 2001 From: manuzhang Date: Fri, 12 Aug 2016 07:22:00 +0800 Subject: [PATCH 005/346] Rename DoFn to OldDoFn in Gearpump runner --- .../gearpump/GearpumpPipelineResult.java | 23 +++++++++++++++++-- .../gearpump/GearpumpPipelineRunner.java | 6 ++--- .../gearpump/examples/StreamingWordCount.java | 6 ++--- .../ParDoBoundMultiTranslator.java | 3 ++- .../translators/ParDoBoundTranslator.java | 3 ++- .../translators/functions/DoFnFunction.java | 3 ++- .../translators/utils/GearpumpDoFnRunner.java | 23 ++++++++++--------- 7 files changed, 45 insertions(+), 22 deletions(-) diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpPipelineResult.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpPipelineResult.java index bc2714727259b..6184bc39a347e 100644 --- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpPipelineResult.java +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpPipelineResult.java @@ -17,11 +17,14 @@ */ package org.apache.beam.runners.gearpump; +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.runners.AggregatorRetrievalException; -import org.apache.beam.sdk.runners.AggregatorValues; import org.apache.beam.sdk.transforms.Aggregator; +import org.joda.time.Duration; + +import java.io.IOException; /** * Result of executing a {@link Pipeline} with Gearpump. @@ -32,6 +35,21 @@ public State getState() { return null; } + @Override + public State cancel() throws IOException { + return null; + } + + @Override + public State waitUntilFinish(Duration duration) throws IOException, InterruptedException { + return null; + } + + @Override + public State waitUntilFinish() throws IOException, InterruptedException { + return null; + } + @Override public AggregatorValues getAggregatorValues(Aggregator aggregator) throws AggregatorRetrievalException { @@ -39,4 +57,5 @@ public AggregatorValues getAggregatorValues(Aggregator aggregator) "PipelineResult getAggregatorValues not supported in Gearpump pipeline", new UnsupportedOperationException()); } + } diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpPipelineRunner.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpPipelineRunner.java index 660d7039737d3..4182ee4785df6 100644 --- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpPipelineRunner.java +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpPipelineRunner.java @@ -23,8 +23,8 @@ import org.apache.beam.sdk.options.PipelineOptionsValidator; 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.Flatten; +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; @@ -151,7 +151,7 @@ private static class AssignWindowsAndSetStrategy private final Window.Bound wrapped; - public AssignWindowsAndSetStrategy(Window.Bound wrapped) { + AssignWindowsAndSetStrategy(Window.Bound wrapped) { this.wrapped = wrapped; } @@ -184,7 +184,7 @@ public PCollection apply(PCollection input) { } } - private static class IdentityFn extends DoFn { + private static class IdentityFn extends OldDoFn { @Override public void processElement(ProcessContext c) { c.output(c.element()); diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/examples/StreamingWordCount.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/examples/StreamingWordCount.java index c51289d5fefe1..5f35c6b155a69 100644 --- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/examples/StreamingWordCount.java +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/examples/StreamingWordCount.java @@ -25,7 +25,7 @@ import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.transforms.Aggregator; import org.apache.beam.sdk.transforms.Count; -import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.transforms.OldDoFn; import org.apache.beam.sdk.transforms.ParDo; import org.apache.beam.sdk.transforms.Sum; import org.apache.beam.sdk.transforms.windowing.FixedWindows; @@ -44,7 +44,7 @@ */ public class StreamingWordCount { - static class ExtractWordsFn extends DoFn { + static class ExtractWordsFn extends OldDoFn { private final Aggregator emptyLines = createAggregator("emptyLines", new Sum.SumLongFn()); @@ -66,7 +66,7 @@ public void processElement(ProcessContext c) { } } - static class FormatAsStringFn extends DoFn, String> { + static class FormatAsStringFn extends OldDoFn, String> { private static final Logger LOG = LoggerFactory.getLogger(FormatAsStringFn.class); @Override diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/ParDoBoundMultiTranslator.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/ParDoBoundMultiTranslator.java index af5bcbcf4c885..d5ed0d2b06c11 100644 --- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/ParDoBoundMultiTranslator.java +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/ParDoBoundMultiTranslator.java @@ -23,6 +23,7 @@ import org.apache.beam.runners.gearpump.translators.utils.NoOpSideInputReader; import org.apache.beam.runners.gearpump.translators.utils.NoOpStepContext; 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.util.DoFnRunner; import org.apache.beam.sdk.util.DoFnRunners; @@ -92,7 +93,7 @@ private static class DoFnMultiFunction implements public DoFnMultiFunction( GearpumpPipelineOptions pipelineOptions, - DoFn doFn, + OldDoFn doFn, TupleTag mainOutputTag, TupleTagList sideOutputTags, WindowingStrategy windowingStrategy, diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/ParDoBoundTranslator.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/ParDoBoundTranslator.java index 689bc08e0f47e..b97cbb4c73da4 100644 --- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/ParDoBoundTranslator.java +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/ParDoBoundTranslator.java @@ -21,6 +21,7 @@ import org.apache.beam.runners.gearpump.translators.functions.DoFnFunction; import org.apache.beam.runners.gearpump.translators.utils.NoOpSideInputReader; 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.util.WindowedValue; import org.apache.beam.sdk.util.WindowingStrategy; @@ -38,7 +39,7 @@ public class ParDoBoundTranslator implements @Override public void translate(ParDo.Bound transform, TranslationContext context) { - DoFn doFn = transform.getFn(); + OldDoFn doFn = transform.getFn(); PCollection output = context.getOutput(transform); WindowingStrategy windowingStrategy = output.getWindowingStrategy(); diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/functions/DoFnFunction.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/functions/DoFnFunction.java index 088fc14c10f8e..b1ebd2a0d1799 100644 --- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/functions/DoFnFunction.java +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/functions/DoFnFunction.java @@ -22,6 +22,7 @@ import org.apache.beam.runners.gearpump.translators.utils.GearpumpDoFnRunner; import org.apache.beam.runners.gearpump.translators.utils.NoOpStepContext; import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.transforms.OldDoFn; import org.apache.beam.sdk.util.DoFnRunner; import org.apache.beam.sdk.util.DoFnRunners; import org.apache.beam.sdk.util.SideInputReader; @@ -50,7 +51,7 @@ public class DoFnFunction implements public DoFnFunction( GearpumpPipelineOptions pipelineOptions, - DoFn doFn, + OldDoFn doFn, WindowingStrategy windowingStrategy, SideInputReader sideInputReader) { this.doFnRunner = new GearpumpDoFnRunner<>( diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/utils/GearpumpDoFnRunner.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/utils/GearpumpDoFnRunner.java index 608ad7c85135b..be0d0256290d3 100644 --- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/utils/GearpumpDoFnRunner.java +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/utils/GearpumpDoFnRunner.java @@ -25,6 +25,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.OldDoFn; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.GlobalWindow; import org.apache.beam.sdk.transforms.windowing.GlobalWindows; @@ -64,7 +65,7 @@ public class GearpumpDoFnRunner implements DoFnRunner, Serializable { - private final DoFn fn; + private final OldDoFn fn; private final transient PipelineOptions options; private final SideInputReader sideInputReader; private final DoFnRunners.OutputManager outputManager; @@ -76,7 +77,7 @@ public class GearpumpDoFnRunner implements DoFnRunner doFn, + OldDoFn doFn, SideInputReader sideInputReader, DoFnRunners.OutputManager outputManager, TupleTag mainOutputTag, @@ -119,7 +120,7 @@ public void startBundle() { @Override public void processElement(WindowedValue elem) { if (elem.getWindows().size() <= 1 - || (!DoFn.RequiresWindowAccess.class.isAssignableFrom(fn.getClass()) + || (!OldDoFn.RequiresWindowAccess.class.isAssignableFrom(fn.getClass()) && context.sideInputReader.isEmpty())) { invokeProcessElement(elem); } else { @@ -144,7 +145,7 @@ public void finishBundle() { } private void invokeProcessElement(WindowedValue elem) { - final DoFn.ProcessContext processContext = + final OldDoFn.ProcessContext processContext = new DoFnProcessContext<>(fn, context, elem); // This can contain user code. Wrap it in case it throws an exception. try { @@ -169,11 +170,11 @@ private boolean isSystemDoFn() { * @param the type of the DoFn's (main) output elements */ private static class DoFnContext - extends DoFn.Context { + extends OldDoFn.Context { private static final int MAX_SIDE_OUTPUTS = 1000; final transient PipelineOptions options; - final DoFn fn; + final OldDoFn fn; final SideInputReader sideInputReader; final DoFnRunners.OutputManager outputManager; final TupleTag mainOutputTag; @@ -187,7 +188,7 @@ private static class DoFnContext private final Set> outputTags; public DoFnContext(PipelineOptions options, - DoFn fn, + OldDoFn fn, SideInputReader sideInputReader, DoFnRunners.OutputManager outputManager, TupleTag mainOutputTag, @@ -357,14 +358,14 @@ protected Aggregator createAggreg * @param the type of the DoFn's (main) output elements */ private static class DoFnProcessContext - extends DoFn.ProcessContext { + extends OldDoFn.ProcessContext { - final DoFn fn; + final OldDoFn fn; final DoFnContext context; final WindowedValue windowedValue; - public DoFnProcessContext(DoFn fn, + public DoFnProcessContext(OldDoFn fn, DoFnContext context, WindowedValue windowedValue) { fn.super(); @@ -409,7 +410,7 @@ public T sideInput(PCollectionView view) { @Override public BoundedWindow window() { - if (!(fn instanceof DoFn.RequiresWindowAccess)) { + if (!(fn instanceof OldDoFn.RequiresWindowAccess)) { throw new UnsupportedOperationException( "window() is only available in the context of a DoFn marked as RequiresWindow."); } From fb74c936ed92c7a8548c338cc03957794fc60902 Mon Sep 17 00:00:00 2001 From: Dan Halperin Date: Fri, 26 Aug 2016 16:25:58 -0700 Subject: [PATCH 006/346] gearpump: switch to stable version They have apparently deleted the SNAPSHOT jar and now builds are failing. --- runners/gearpump/pom.xml | 21 +-------------------- 1 file changed, 1 insertion(+), 20 deletions(-) diff --git a/runners/gearpump/pom.xml b/runners/gearpump/pom.xml index 8fc46357ca1e4..cc99a7a38489d 100644 --- a/runners/gearpump/pom.xml +++ b/runners/gearpump/pom.xml @@ -35,28 +35,9 @@ UTF-8 UTF-8 - 0.8.1-SNAPSHOT + 0.8.1 - - - apache.snapshots - Apache Development Snapshot Repository - https://repository.apache.org/content/repositories/snapshots/ - - false - - - true - - - - gearpump-shaded-repo - Vincent at Bintray - http://dl.bintray.com/fvunicorn/maven - - - local-runnable-on-service-tests From 0dfb8ff55d6f80264222fde4501ea3050d2e3911 Mon Sep 17 00:00:00 2001 From: gaurav gupta Date: Wed, 10 Aug 2016 16:43:03 -0700 Subject: [PATCH 007/346] Made byteArrayCoder final static --- .../build-tools/src/main/resources/beam/findbugs-filter.xml | 6 ------ .../src/main/java/org/apache/beam/sdk/util/BitSetCoder.java | 2 +- 2 files changed, 1 insertion(+), 7 deletions(-) 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 1ad79880772b6..d135cbb671d9b 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 @@ -109,12 +109,6 @@ - - - - - - diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/BitSetCoder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/BitSetCoder.java index 82c5f51c63d6b..b406b1a56cc52 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/BitSetCoder.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/BitSetCoder.java @@ -32,7 +32,7 @@ class BitSetCoder extends AtomicCoder { private static final BitSetCoder INSTANCE = new BitSetCoder(); - private transient ByteArrayCoder byteArrayCoder = ByteArrayCoder.of(); + private static final ByteArrayCoder byteArrayCoder = ByteArrayCoder.of(); private BitSetCoder() {} From b9f826366823003940805e6469b10df8819b0977 Mon Sep 17 00:00:00 2001 From: Dan Halperin Date: Wed, 10 Aug 2016 17:58:09 -0700 Subject: [PATCH 008/346] CompressedSource: CompressedReader is never splittable The only way it's safe to split a compressed file is if the file is not compressed. This can only happen when the source itself is splittable, and that in turn will result in the inner source's reader being returned. A CompressedReader will only be created in the event that the file is NOT splittable. So remove all the logic handling splittable compressed readers, and instead go with the logic when we know/assume the file is compressed. * TextIO: test compression with larger files It is important for correctness that we test with large files because otherwise the compressed file may be larger than the uncompressed file, which could mask bugs * TextIOTest: flesh out more * TextIOTest: add large uncompressed file --- .../apache/beam/sdk/io/CompressedSource.java | 80 ++---- .../beam/sdk/io/CompressedSourceTest.java | 2 +- .../org/apache/beam/sdk/io/TextIOTest.java | 251 ++++++++++++++---- 3 files changed, 227 insertions(+), 106 deletions(-) diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/CompressedSource.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/CompressedSource.java index ee4b84b52717c..11ff90f66af9e 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/CompressedSource.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/CompressedSource.java @@ -430,7 +430,6 @@ public static class CompressedReader extends FileBasedReader { private final FileBasedReader readerDelegate; private final CompressedSource source; - private final boolean splittable; private final Object progressLock = new Object(); @GuardedBy("progressLock") private int numRecordsRead; @@ -443,13 +442,6 @@ public static class CompressedReader extends FileBasedReader { public CompressedReader(CompressedSource source, FileBasedReader readerDelegate) { super(source); this.source = source; - boolean splittable; - try { - splittable = source.isSplittable(); - } catch (Exception e) { - throw new RuntimeException("Unable to tell whether source " + source + " is splittable", e); - } - this.splittable = splittable; this.readerDelegate = readerDelegate; } @@ -463,27 +455,19 @@ public T getCurrent() throws NoSuchElementException { @Override public boolean allowsDynamicSplitting() { - return splittable; + return false; } @Override public final long getSplitPointsConsumed() { - if (splittable) { - return readerDelegate.getSplitPointsConsumed(); - } else { - synchronized (progressLock) { - return (isDone() && numRecordsRead > 0) ? 1 : 0; - } + synchronized (progressLock) { + return (isDone() && numRecordsRead > 0) ? 1 : 0; } } @Override public final long getSplitPointsRemaining() { - if (splittable) { - return readerDelegate.getSplitPointsRemaining(); - } else { - return isDone() ? 0 : 1; - } + return isDone() ? 0 : 1; } /** @@ -491,18 +475,14 @@ public final long getSplitPointsRemaining() { */ @Override protected final boolean isAtSplitPoint() { - if (splittable) { - return readerDelegate.isAtSplitPoint(); - } else { - // We have to return true for the first record, but not for the state before reading it, - // and not for the state after reading any other record. Hence == rather than >= or <=. - // This is required because FileBasedReader is intended for readers that can read a range - // of offsets in a file and where the range can be split in parts. CompressedReader, - // however, is a degenerate case because it cannot be split, but it has to satisfy the - // semantics of offsets and split points anyway. - synchronized (progressLock) { - return numRecordsRead == 1; - } + // We have to return true for the first record, but not for the state before reading it, + // and not for the state after reading any other record. Hence == rather than >= or <=. + // This is required because FileBasedReader is intended for readers that can read a range + // of offsets in a file and where the range can be split in parts. CompressedReader, + // however, is a degenerate case because it cannot be split, but it has to satisfy the + // semantics of offsets and split points anyway. + synchronized (progressLock) { + return numRecordsRead == 1; } } @@ -546,14 +526,9 @@ public void close() throws IOException { */ @Override protected final void startReading(ReadableByteChannel channel) throws IOException { - if (splittable) { - // No-op. We will always delegate to the inner reader, so this.channel and this.progressLock - // will never be used. - } else { - synchronized (progressLock) { - this.channel = new CountingChannel(channel, getCurrentSource().getStartOffset()); - channel = this.channel; - } + synchronized (progressLock) { + this.channel = new CountingChannel(channel, getCurrentSource().getStartOffset()); + channel = this.channel; } if (source.getChannelFactory() instanceof FileNameBasedDecompressingChannelFactory) { @@ -582,30 +557,21 @@ protected final boolean readNextRecord() throws IOException { return true; } - // Splittable: simply delegates to the inner reader. - // // Unsplittable: returns the offset in the input stream that has been read by the input. // these positions are likely to be coarse-grained (in the event of buffering) and // over-estimates (because they reflect the number of bytes read to produce an element, not its // start) but both of these provide better data than e.g., reporting the start of the file. @Override protected final long getCurrentOffset() throws NoSuchElementException { - if (splittable) { - return readerDelegate.getCurrentOffset(); - } else { - synchronized (progressLock) { - if (numRecordsRead <= 1) { - // Since the first record is at a split point, it should start at the beginning of the - // file. This avoids the bad case where the decompressor read the entire file, which - // would cause the file to be treated as empty when returning channel.getCount() as it - // is outside the valid range. - return 0; - } - if (channel == null) { - throw new NoSuchElementException(); - } - return channel.getCount(); + synchronized (progressLock) { + if (numRecordsRead <= 1) { + // Since the first record is at a split point, it should start at the beginning of the + // file. This avoids the bad case where the decompressor read the entire file, which + // would cause the file to be treated as empty when returning channel.getCount() as it + // is outside the valid range. + return 0; } + return channel.getCount(); } } } 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 01e5fe5e9c6cb..4a9f95088804e 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 @@ -563,7 +563,7 @@ protected boolean readNextRecord() throws IOException { if (channel.read(buff) != 1) { return false; } - current = new Byte(buff.get(0)); + current = buff.get(0); offset += 1; return true; } 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 6ec3a717af6bf..6fd3093f2155b 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 @@ -25,6 +25,7 @@ import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasValue; import static org.hamcrest.Matchers.containsInAnyOrder; +import static org.hamcrest.Matchers.greaterThan; import static org.hamcrest.Matchers.hasItem; import static org.hamcrest.Matchers.startsWith; import static org.junit.Assert.assertEquals; @@ -61,6 +62,7 @@ import com.google.common.collect.ImmutableList; +import org.apache.commons.compress.compressors.bzip2.BZip2CompressorOutputStream; import org.junit.BeforeClass; import org.junit.Ignore; import org.junit.Rule; @@ -79,6 +81,7 @@ import java.io.FileOutputStream; import java.io.FileReader; import java.io.IOException; +import java.io.OutputStream; import java.io.PrintStream; import java.nio.channels.FileChannel; import java.nio.channels.SeekableByteChannel; @@ -102,8 +105,11 @@ @RunWith(JUnit4.class) @SuppressWarnings("unchecked") public class TextIOTest { - @Rule public TemporaryFolder tmpFolder = new TemporaryFolder(); - @Rule public ExpectedException expectedException = ExpectedException.none(); + + @Rule + public TemporaryFolder tmpFolder = new TemporaryFolder(); + @Rule + public ExpectedException expectedException = ExpectedException.none(); @BeforeClass public static void setupClass() { @@ -166,7 +172,7 @@ public void testReadEmptyInts() throws Exception { @Test @Category(NeedsRunner.class) public void testReadNulls() throws Exception { - runTestRead(new Void[]{ null, null, null }, VoidCoder.of()); + runTestRead(new Void[]{null, null, null}, VoidCoder.of()); } @Test @@ -419,32 +425,205 @@ public void testCompressionTypeIsSet() throws Exception { assertEquals(CompressionType.GZIP, read.getCompressionType()); } - @Test - @Category(NeedsRunner.class) - public void testCompressedRead() throws Exception { - String[] lines = {"Irritable eagle", "Optimistic jay", "Fanciful hawk"}; - File tmpFile = tmpFolder.newFile(); - String filename = tmpFile.getPath(); - - List expected = new ArrayList<>(); - try (PrintStream writer = - new PrintStream(new GZIPOutputStream(new FileOutputStream(tmpFile)))) { + /** + * Helper that writes the given lines (adding a newline in between) to a stream, then closes the + * stream. + */ + private static void writeToStreamAndClose(String[] lines, OutputStream outputStream) { + try (PrintStream writer = new PrintStream(outputStream)) { for (String line : lines) { writer.println(line); - expected.add(line); } } + } + /** + * Helper method that runs TextIO.Read.from(filename).withCompressionType(compressionType) + * and asserts that the results match the given expected output. + */ + private static void assertReadingCompressedFileMatchesExpected( + String filename, CompressionType compressionType, String[] expected) { Pipeline p = TestPipeline.create(); - TextIO.Read.Bound read = - TextIO.Read.from(filename).withCompressionType(CompressionType.GZIP); + TextIO.Read.from(filename).withCompressionType(compressionType); PCollection output = p.apply(read); PAssert.that(output).containsInAnyOrder(expected); p.run(); } + /** + * Helper to make an array of compressible strings. Returns ["word"i] for i in range(0,n). + */ + private static String[] makeLines(int n) { + String[] ret = new String[n]; + for (int i = 0; i < n; ++i) { + ret[i] = "word" + i; + } + return ret; + } + + /** + * Tests reading from a small, gzipped file with no .gz extension but GZIP compression set. + */ + @Test + @Category(NeedsRunner.class) + public void testSmallCompressedGzipReadNoExtension() throws Exception { + String[] lines = {"Irritable eagle", "Optimistic jay", "Fanciful hawk"}; + File tmpFile = tmpFolder.newFile(); // no GZ extension + String filename = tmpFile.getPath(); + + writeToStreamAndClose(lines, new GZIPOutputStream(new FileOutputStream(tmpFile))); + assertReadingCompressedFileMatchesExpected(filename, CompressionType.GZIP, lines); + } + + /** + * Tests reading from a small, gzipped file with .gz extension and AUTO or GZIP compression set. + */ + @Test + @Category(NeedsRunner.class) + public void testSmallCompressedGzipRead() throws Exception { + String[] lines = {"Irritable eagle", "Optimistic jay", "Fanciful hawk"}; + File tmpFile = tmpFolder.newFile("small_gzip.gz"); + String filename = tmpFile.getPath(); + + writeToStreamAndClose(lines, new GZIPOutputStream(new FileOutputStream(tmpFile))); + // Should work in AUTO mode. + assertReadingCompressedFileMatchesExpected(filename, CompressionType.AUTO, lines); + // Should work in GZIP mode. + assertReadingCompressedFileMatchesExpected(filename, CompressionType.GZIP, lines); + } + + /** + * Tests reading from a small, uncompressed file with .gz extension. + * This must work in AUTO or GZIP modes. This is needed because some network file systems / HTTP + * clients will transparently decompress gzipped content. + */ + @Test + @Category(NeedsRunner.class) + public void testSmallCompressedGzipReadActuallyUncompressed() throws Exception { + String[] lines = {"Irritable eagle", "Optimistic jay", "Fanciful hawk"}; + File tmpFile = tmpFolder.newFile("not_really_gzipped.gz"); // GZ file extension lies + String filename = tmpFile.getPath(); + + writeToStreamAndClose(lines, new FileOutputStream(tmpFile)); + // Should work with GZIP compression set. + assertReadingCompressedFileMatchesExpected(filename, CompressionType.GZIP, lines); + // Should also work with AUTO mode set. + assertReadingCompressedFileMatchesExpected(filename, CompressionType.AUTO, lines); + } + + /** + * Tests reading from a small, bzip2ed file with no .bz2 extension but BZIP2 compression set. + */ + @Test + @Category(NeedsRunner.class) + public void testSmallCompressedBzip2ReadNoExtension() throws Exception { + String[] lines = {"Irritable eagle", "Optimistic jay", "Fanciful hawk"}; + File tmpFile = tmpFolder.newFile(); // no BZ2 extension + String filename = tmpFile.getPath(); + + writeToStreamAndClose(lines, new BZip2CompressorOutputStream(new FileOutputStream(tmpFile))); + assertReadingCompressedFileMatchesExpected(filename, CompressionType.BZIP2, lines); + } + + /** + * Tests reading from a small, bzip2ed file with .bz2 extension and AUTO or BZIP2 compression set. + */ + @Test + @Category(NeedsRunner.class) + public void testSmallCompressedBzipRead() throws Exception { + String[] lines = {"Irritable eagle", "Optimistic jay", "Fanciful hawk"}; + File tmpFile = tmpFolder.newFile("small_bzip2.bz2"); + String filename = tmpFile.getPath(); + + writeToStreamAndClose(lines, new BZip2CompressorOutputStream(new FileOutputStream(tmpFile))); + // Should work in AUTO mode. + assertReadingCompressedFileMatchesExpected(filename, CompressionType.AUTO, lines); + // Should work in BZIP2 mode. + assertReadingCompressedFileMatchesExpected(filename, CompressionType.BZIP2, lines); + } + + /** + * Tests reading from a large, bzip2ed file with .bz2 extension and AUTO or BZIP2 compression set. + * It is important to test a large compressible file because using only small files may mask bugs + * from range tracking that can only occur if the file compression ratio is high -- small + * compressed files are usually as big as the uncompressed ones or bigger. + */ + @Test + @Category(NeedsRunner.class) + public void testLargeCompressedBzipRead() throws Exception { + String[] lines = makeLines(5000); + File bz2File = tmpFolder.newFile("large_bzip2.bz2"); + String bz2Filename = bz2File.getPath(); + + writeToStreamAndClose(lines, new BZip2CompressorOutputStream(new FileOutputStream(bz2File))); + // Should work in AUTO mode. + assertReadingCompressedFileMatchesExpected(bz2Filename, CompressionType.AUTO, lines); + // Should work in BZIP2 mode. + assertReadingCompressedFileMatchesExpected(bz2Filename, CompressionType.BZIP2, lines); + + // Confirm that the compressed file is smaller than the uncompressed file. + File txtFile = tmpFolder.newFile("large_bzip2.txt"); + writeToStreamAndClose(lines, new FileOutputStream(txtFile)); + assertThat(Files.size(txtFile.toPath()), greaterThan(Files.size(bz2File.toPath()))); + } + + /** + * Tests reading from a large, gzipped file with .gz extension and AUTO or GZIP compression set. + * It is important to test a large compressible file because using only small files may mask bugs + * from range tracking that can only occur if the file compression ratio is high -- small + * compressed files are usually as big as the uncompressed ones or bigger. + */ + @Test + @Category(NeedsRunner.class) + public void testLargeCompressedGzipRead() throws Exception { + String[] lines = makeLines(5000); + File gzFile = tmpFolder.newFile("large_gzip.gz"); + String gzFilename = gzFile.getPath(); + + writeToStreamAndClose(lines, new GZIPOutputStream(new FileOutputStream(gzFile))); + // Should work in AUTO mode. + assertReadingCompressedFileMatchesExpected(gzFilename, CompressionType.AUTO, lines); + // Should work in BZIP2 mode. + assertReadingCompressedFileMatchesExpected(gzFilename, CompressionType.GZIP, lines); + + // Confirm that the compressed file is smaller than the uncompressed file. + File txtFile = tmpFolder.newFile("large_gzip.txt"); + writeToStreamAndClose(lines, new FileOutputStream(txtFile)); + assertThat(Files.size(txtFile.toPath()), greaterThan(Files.size(gzFile.toPath()))); + } + + /** + * Tests reading from a large, uncompressed file. + */ + @Test + @Category(NeedsRunner.class) + public void testLargeUncompressedReadTxt() throws Exception { + String[] lines = makeLines(5000); + File txtFile = tmpFolder.newFile("large_file.txt"); + String txtFilename = txtFile.getPath(); + + writeToStreamAndClose(lines, new FileOutputStream(txtFile)); + // Should work in AUTO mode. + assertReadingCompressedFileMatchesExpected(txtFilename, CompressionType.AUTO, lines); + } + + /** + * Tests reading from a large, uncompressed file with a weird file extension. + */ + @Test + @Category(NeedsRunner.class) + public void testLargeUncompressedReadWeirdExtension() throws Exception { + String[] lines = makeLines(5000); + File txtFile = tmpFolder.newFile("large_file.bin.data.foo"); + String txtFilename = txtFile.getPath(); + + writeToStreamAndClose(lines, new FileOutputStream(txtFile)); + // Should work in AUTO mode. + assertReadingCompressedFileMatchesExpected(txtFilename, CompressionType.AUTO, lines); + } + /** * Create a zip file with the given lines. * @@ -553,7 +732,7 @@ public void testZipCompressedReadWithEmptyFile() throws Exception { @Test @Category(NeedsRunner.class) public void testZipCompressedReadWithEmptyEntry() throws Exception { - String filename = createZipFile(new ArrayList(), null, new String[]{ }); + String filename = createZipFile(new ArrayList(), null, new String[]{}); Pipeline p = TestPipeline.create(); @@ -571,9 +750,9 @@ public void testZipCompressedReadWithEmptyEntry() throws Exception { @Test @Category(NeedsRunner.class) public void testZipCompressedReadWithMultiEntriesFile() throws Exception { - String[] entry0 = new String[]{ "first", "second", "three" }; - String[] entry1 = new String[]{ "four", "five", "six" }; - String[] entry2 = new String[]{ "seven", "eight", "nine" }; + String[] entry0 = new String[]{"first", "second", "three"}; + String[] entry1 = new String[]{"four", "five", "six"}; + String[] entry2 = new String[]{"seven", "eight", "nine"}; List expected = new ArrayList<>(); @@ -599,10 +778,10 @@ public void testZipCompressedReadWithComplexEmptyAndPresentEntries() throws Exce String filename = createZipFile( new ArrayList(), null, - new String[] {"cat"}, - new String[] {}, - new String[] {}, - new String[] {"dog"}); + new String[]{"cat"}, + new String[]{}, + new String[]{}, + new String[]{"dog"}); List expected = ImmutableList.of("cat", "dog"); Pipeline p = TestPipeline.create(); @@ -614,30 +793,6 @@ public void testZipCompressedReadWithComplexEmptyAndPresentEntries() throws Exce p.run(); } - @Test - @Category(NeedsRunner.class) - public void testGZIPReadWhenUncompressed() throws Exception { - String[] lines = {"Meritorious condor", "Obnoxious duck"}; - File tmpFile = tmpFolder.newFile(); - String filename = tmpFile.getPath(); - - List expected = new ArrayList<>(); - try (PrintStream writer = new PrintStream(new FileOutputStream(tmpFile))) { - for (String line : lines) { - writer.println(line); - expected.add(line); - } - } - - Pipeline p = TestPipeline.create(); - TextIO.Read.Bound read = - TextIO.Read.from(filename).withCompressionType(CompressionType.GZIP); - PCollection output = p.apply(read); - - PAssert.that(output).containsInAnyOrder(expected); - p.run(); - } - @Test public void testTextIOGetName() { assertEquals("TextIO.Read", TextIO.Read.from("somefile").getName()); From 011bea9a83a828e0d8c6518ab83aa5cc4f75e146 Mon Sep 17 00:00:00 2001 From: David Rieber Date: Tue, 9 Aug 2016 14:05:25 -0700 Subject: [PATCH 009/346] Do not add DataDisks to windmill service jobs. --- .../beam/runners/dataflow/DataflowPipelineTranslator.java | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) 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 ac06b5275d4d8..0ca92c849f431 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 @@ -428,7 +428,9 @@ public Job translate(List packages) { workerPool.setPackages(packages); workerPool.setNumWorkers(options.getNumWorkers()); - if (options.isStreaming()) { + if (options.isStreaming() + && (options.getExperiments() == null + || !options.getExperiments().contains("enable_windmill_service"))) { // Use separate data disk for streaming. Disk disk = new Disk(); disk.setDiskType(options.getWorkerDiskType()); From 1d86335314685926d3eb0f9765d615a77cee75e6 Mon Sep 17 00:00:00 2001 From: Thomas Groh Date: Thu, 11 Aug 2016 09:16:55 -0700 Subject: [PATCH 010/346] Remove timeout in DirectRunnerTest If the test hangs due to bugs, the infrastructure should kill it. --- .../java/org/apache/beam/runners/direct/DirectRunnerTest.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 ddce4583a5d40..9739adb0f3662 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 @@ -155,7 +155,7 @@ public void updateChanged(ProcessContext c) { assertThat("Each element should have been processed twice", changed.get(), equalTo(6)); } - @Test(timeout = 5000L) + @Test public void byteArrayCountShouldSucceed() { Pipeline p = getPipeline(); From 36a9aa232ea56de449930194788becce585212ef Mon Sep 17 00:00:00 2001 From: Thomas Groh Date: Mon, 8 Aug 2016 19:09:58 -0700 Subject: [PATCH 011/346] Improve Write Error Message If provided with an Unbounded PCollection, Write will fail due to restriction of calling finalize only once. This error message fails in a deep stack trace based on it not being possible to apply a GroupByKey. Instead, throw immediately on application with a specific error message. --- .../direct/WriteWithShardingFactory.java | 4 ++++ .../java/org/apache/beam/sdk/io/Write.java | 5 ++++ .../org/apache/beam/sdk/io/WriteTest.java | 24 +++++++++++++++---- 3 files changed, 28 insertions(+), 5 deletions(-) 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 cee400102ba5d..c2157b8634ce0 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 @@ -34,6 +34,7 @@ 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.PCollection.IsBounded; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.PDone; import org.apache.beam.sdk.values.PInput; @@ -74,6 +75,9 @@ private DynamicallyReshardedWrite(Bound original) { @Override public PDone apply(PCollection input) { + checkArgument(IsBounded.BOUNDED == input.isBounded(), + "%s can only be applied to a Bounded PCollection", + getClass().getSimpleName()); PCollection records = input.apply("RewindowInputs", Window.into(new GlobalWindows()).triggering(DefaultTrigger.of()) .withAllowedLateness(Duration.ZERO) 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 a846b7c33ba0f..fea65ef233984 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 @@ -17,6 +17,7 @@ */ package org.apache.beam.sdk.io; +import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.base.Preconditions.checkNotNull; import org.apache.beam.sdk.Pipeline; @@ -40,6 +41,7 @@ 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.PCollection.IsBounded; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.PDone; @@ -106,6 +108,9 @@ private Bound(Sink sink, int numShards) { @Override public PDone apply(PCollection input) { + checkArgument(IsBounded.BOUNDED == input.isBounded(), + "%s can only be applied to a Bounded PCollection", + Write.class.getSimpleName()); PipelineOptions options = input.getPipeline().getOptions(); sink.validate(options); return createWrite(input, sink.createWriteOperation(options)); 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 705b77cb39868..6b44b6a027418 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 @@ -19,8 +19,6 @@ import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasDisplayItem; import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.includesDisplayDataFrom; -import static org.apache.beam.sdk.values.KV.of; - import static org.hamcrest.Matchers.anyOf; import static org.hamcrest.Matchers.containsInAnyOrder; import static org.hamcrest.Matchers.equalTo; @@ -30,8 +28,6 @@ import static org.junit.Assert.assertThat; import static org.junit.Assert.assertTrue; -import static java.util.concurrent.ThreadLocalRandom.current; - import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.SerializableCoder; @@ -56,14 +52,17 @@ 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.PCollection.IsBounded; import com.google.common.base.MoreObjects; import com.google.common.base.Optional; 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.rules.ExpectedException; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; @@ -76,6 +75,7 @@ import java.util.Objects; import java.util.Set; import java.util.UUID; +import java.util.concurrent.ThreadLocalRandom; import java.util.concurrent.atomic.AtomicInteger; /** @@ -83,6 +83,8 @@ */ @RunWith(JUnit4.class) public class WriteTest { + @Rule public ExpectedException thrown = ExpectedException.none(); + // Static store that can be accessed within the writer private static List sinkContents = new ArrayList<>(); // Static count of output shards @@ -108,7 +110,7 @@ private static class AddArbitraryKey extends DoFn> { @ProcessElement public void processElement(ProcessContext c) { - c.output(of(current().nextInt(), c.element())); + c.output(KV.of(ThreadLocalRandom.current().nextInt(), c.element())); } } @@ -289,6 +291,18 @@ public void populateDisplayData(DisplayData.Builder builder) { assertThat(displayData, hasDisplayItem("numShards", 1)); } + @Test + public void testWriteUnbounded() { + TestPipeline p = TestPipeline.create(); + PCollection unbounded = + p.apply(Create.of("foo")).setIsBoundedInternal(IsBounded.UNBOUNDED); + + TestSink sink = new TestSink(); + thrown.expect(IllegalArgumentException.class); + thrown.expectMessage("Write can only be applied to a Bounded PCollection"); + unbounded.apply(Write.to(sink)); + } + /** * Performs a Write transform and verifies the Write transform calls the appropriate methods on * a test sink in the correct order, as well as verifies that the elements of a PCollection are From d5641553cebb02f08ca7c1fe667948d39cb3962c Mon Sep 17 00:00:00 2001 From: Thomas Groh Date: Tue, 9 Aug 2016 10:47:09 -0700 Subject: [PATCH 012/346] Remove Streaming Write Overrides in DataflowRunner These writes should be forbidden based on the boundedness of the input PCollection. As Write explicitly forbids the application of the transform to an Unbounded PCollection, this will be equivalent in most cases; In cases where the input PCollection is Bounded, due to an UnboundedReadFromBoundedSource, the write will function as expected and does not need to be forbidden. --- .../beam/runners/dataflow/DataflowRunner.java | 72 +------------------ .../runners/dataflow/DataflowRunnerTest.java | 24 ------- .../org/apache/beam/sdk/io/WriteTest.java | 12 +++- 3 files changed, 11 insertions(+), 97 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 bea6264bd36f9..76dbecf433b2b 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 @@ -20,7 +20,6 @@ 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 com.google.common.base.Preconditions.checkArgument; import static com.google.common.base.Preconditions.checkState; import static com.google.common.base.Strings.isNullOrEmpty; @@ -58,14 +57,12 @@ import org.apache.beam.sdk.coders.StandardCoder; import org.apache.beam.sdk.coders.VarIntCoder; import org.apache.beam.sdk.coders.VarLongCoder; -import org.apache.beam.sdk.io.AvroIO; import org.apache.beam.sdk.io.BoundedSource; import org.apache.beam.sdk.io.FileBasedSink; import org.apache.beam.sdk.io.PubsubIO; import org.apache.beam.sdk.io.PubsubUnboundedSink; import org.apache.beam.sdk.io.PubsubUnboundedSource; import org.apache.beam.sdk.io.Read; -import org.apache.beam.sdk.io.TextIO; import org.apache.beam.sdk.io.UnboundedSource; import org.apache.beam.sdk.io.Write; import org.apache.beam.sdk.options.PipelineOptions; @@ -143,6 +140,7 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; + import org.joda.time.DateTimeUtils; import org.joda.time.DateTimeZone; import org.joda.time.format.DateTimeFormat; @@ -172,6 +170,7 @@ import java.util.Set; import java.util.SortedSet; import java.util.TreeSet; + import javax.annotation.Nullable; /** @@ -329,11 +328,8 @@ public static DataflowRunner fromOptions(PipelineOptions options) { builder.put(View.AsSingleton.class, StreamingViewAsSingleton.class); builder.put(View.AsList.class, StreamingViewAsList.class); builder.put(View.AsIterable.class, StreamingViewAsIterable.class); - builder.put(Write.Bound.class, StreamingWrite.class); builder.put(Read.Unbounded.class, StreamingUnboundedRead.class); builder.put(Read.Bounded.class, StreamingBoundedRead.class); - builder.put(AvroIO.Write.Bound.class, UnsupportedIO.class); - builder.put(TextIO.Write.Bound.class, UnsupportedIO.class); builder.put(Window.Bound.class, AssignWindows.class); // In streaming mode must use either the custom Pubsub unbounded source/sink or // defer to Windmill's built-in implementation. @@ -2045,30 +2041,6 @@ public PDone apply(PCollection input) { } } - /** - * Specialized (non-)implementation for - * {@link org.apache.beam.sdk.io.Write.Bound Write.Bound} - * for the Dataflow runner in streaming mode. - */ - private static class StreamingWrite extends PTransform, PDone> { - /** - * Builds an instance of this class from the overridden transform. - */ - @SuppressWarnings("unused") // used via reflection in DataflowRunner#apply() - public StreamingWrite(DataflowRunner runner, Write.Bound transform) { } - - @Override - public PDone apply(PCollection input) { - throw new UnsupportedOperationException( - "The Write transform is not supported by the Dataflow streaming runner."); - } - - @Override - protected String getKindString() { - return "StreamingWrite"; - } - } - // ================================================================================ // PubsubIO translations // ================================================================================ @@ -2723,30 +2695,6 @@ private static class UnsupportedIO doFn; - /** - * Builds an instance of this class from the overridden transform. - */ - @SuppressWarnings("unused") // used via reflection in DataflowRunner#apply() - public UnsupportedIO(DataflowRunner runner, AvroIO.Read.Bound transform) { - this.transform = transform; - } - - /** - * Builds an instance of this class from the overridden transform. - */ - @SuppressWarnings("unused") // used via reflection in DataflowRunner#apply() - public UnsupportedIO(DataflowRunner runner, TextIO.Read.Bound transform) { - this.transform = transform; - } - - /** - * Builds an instance of this class from the overridden transform. - */ - @SuppressWarnings("unused") // used via reflection in DataflowRunner#apply() - public UnsupportedIO(DataflowRunner runner, Read.Bounded transform) { - this.transform = transform; - } - /** * Builds an instance of this class from the overridden transform. */ @@ -2755,22 +2703,6 @@ public UnsupportedIO(DataflowRunner runner, Read.Unbounded transform) { this.transform = transform; } - /** - * Builds an instance of this class from the overridden transform. - */ - @SuppressWarnings("unused") // used via reflection in DataflowRunner#apply() - public UnsupportedIO(DataflowRunner runner, AvroIO.Write.Bound transform) { - this.transform = transform; - } - - /** - * Builds an instance of this class from the overridden transform. - */ - @SuppressWarnings("unused") // used via reflection in DataflowRunner#apply() - public UnsupportedIO(DataflowRunner runner, TextIO.Write.Bound transform) { - this.transform = transform; - } - /** * Builds an instance of this class from the overridden doFn. */ 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 704410def77c2..d7deffdb03238 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 @@ -55,7 +55,6 @@ import org.apache.beam.sdk.coders.BigEndianLongCoder; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.VarLongCoder; -import org.apache.beam.sdk.io.AvroIO; import org.apache.beam.sdk.io.Read; import org.apache.beam.sdk.io.TextIO; import org.apache.beam.sdk.options.PipelineOptions; @@ -82,7 +81,6 @@ import org.apache.beam.sdk.util.gcsfs.GcsPath; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; -import org.apache.beam.sdk.values.PDone; import org.apache.beam.sdk.values.PInput; import org.apache.beam.sdk.values.TimestampedValue; import org.apache.beam.sdk.values.TupleTag; @@ -956,28 +954,6 @@ public void testReadUnboundedUnsupportedInBatch() throws Exception { testUnsupportedSource(Read.from(new TestCountingSource(1)), "Read.Unbounded", false); } - private void testUnsupportedSink( - PTransform, PDone> sink, String name, boolean streaming) - throws Exception { - thrown.expect(UnsupportedOperationException.class); - thrown.expectMessage( - "The DataflowRunner in streaming mode does not support " + name); - - Pipeline p = Pipeline.create(makeOptions(streaming)); - p.apply(Create.of("foo")).apply(sink); - p.run(); - } - - @Test - public void testAvroIOSinkUnsupportedInStreaming() throws Exception { - testUnsupportedSink(AvroIO.Write.to("foo").withSchema(String.class), "AvroIO.Write", true); - } - - @Test - public void testTextIOSinkUnsupportedInStreaming() throws Exception { - testUnsupportedSink(TextIO.Write.to("foo"), "TextIO.Write", true); - } - @Test public void testBatchViewAsSingletonToIsmRecord() throws Exception { DoFnTester>>>, 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 6b44b6a027418..f9bf472eab9d5 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 @@ -52,7 +52,6 @@ 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.PCollection.IsBounded; import com.google.common.base.MoreObjects; import com.google.common.base.Optional; @@ -294,8 +293,8 @@ public void populateDisplayData(DisplayData.Builder builder) { @Test public void testWriteUnbounded() { TestPipeline p = TestPipeline.create(); - PCollection unbounded = - p.apply(Create.of("foo")).setIsBoundedInternal(IsBounded.UNBOUNDED); + PCollection unbounded = p.apply(CountingInput.unbounded()) + .apply(MapElements.via(new ToStringFn())); TestSink sink = new TestSink(); thrown.expect(IllegalArgumentException.class); @@ -303,6 +302,13 @@ public void testWriteUnbounded() { unbounded.apply(Write.to(sink)); } + private static class ToStringFn extends SimpleFunction { + @Override + public String apply(Long input) { + return Long.toString(input); + } + } + /** * Performs a Write transform and verifies the Write transform calls the appropriate methods on * a test sink in the correct order, as well as verifies that the elements of a PCollection are From 89921c41ca9d4c333af45efa32359a631214c1df Mon Sep 17 00:00:00 2001 From: bchambers Date: Fri, 29 Jul 2016 09:41:17 -0700 Subject: [PATCH 013/346] Remove Counter and associated code Aggregator is the model level concept. Counter was specific to the Dataflow Runner, and is now not needed as part of Beam. --- .../org/apache/beam/sdk/util/DoFnRunners.java | 78 - .../beam/runners/dataflow/DataflowRunner.java | 4 +- .../apache/beam/sdk/transforms/Combine.java | 13 - .../org/apache/beam/sdk/transforms/Max.java | 27 +- .../org/apache/beam/sdk/transforms/Min.java | 28 +- .../org/apache/beam/sdk/transforms/Sum.java | 27 +- .../beam/sdk/util/CounterAggregator.java | 128 -- .../apache/beam/sdk/util/common/Counter.java | 1287 ----------------- .../beam/sdk/util/common/CounterName.java | 153 -- .../beam/sdk/util/common/CounterProvider.java | 27 - .../beam/sdk/util/common/CounterSet.java | 179 --- .../util/common/ElementByteSizeObserver.java | 24 +- .../beam/sdk/util/CounterAggregatorTest.java | 256 ---- .../beam/sdk/util/common/CounterSetTest.java | 227 --- .../beam/sdk/util/common/CounterTest.java | 736 ---------- 15 files changed, 15 insertions(+), 3179 deletions(-) delete mode 100644 sdks/java/core/src/main/java/org/apache/beam/sdk/util/CounterAggregator.java delete mode 100644 sdks/java/core/src/main/java/org/apache/beam/sdk/util/common/Counter.java delete mode 100644 sdks/java/core/src/main/java/org/apache/beam/sdk/util/common/CounterName.java delete mode 100644 sdks/java/core/src/main/java/org/apache/beam/sdk/util/common/CounterProvider.java delete mode 100644 sdks/java/core/src/main/java/org/apache/beam/sdk/util/common/CounterSet.java delete mode 100644 sdks/java/core/src/test/java/org/apache/beam/sdk/util/CounterAggregatorTest.java delete mode 100644 sdks/java/core/src/test/java/org/apache/beam/sdk/util/common/CounterSetTest.java delete mode 100644 sdks/java/core/src/test/java/org/apache/beam/sdk/util/common/CounterTest.java diff --git a/runners/core-java/src/main/java/org/apache/beam/sdk/util/DoFnRunners.java b/runners/core-java/src/main/java/org/apache/beam/sdk/util/DoFnRunners.java index a9f3cf4304a16..60892281c7b78 100644 --- a/runners/core-java/src/main/java/org/apache/beam/sdk/util/DoFnRunners.java +++ b/runners/core-java/src/main/java/org/apache/beam/sdk/util/DoFnRunners.java @@ -23,8 +23,6 @@ import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.util.DoFnRunner.ReduceFnExecutor; import org.apache.beam.sdk.util.ExecutionContext.StepContext; -import org.apache.beam.sdk.util.common.CounterSet; -import org.apache.beam.sdk.util.common.CounterSet.AddCounterMutator; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.TupleTag; @@ -71,33 +69,6 @@ public static DoFnRunner simpleRunner( windowingStrategy); } - /** - * Returns a basic implementation of {@link DoFnRunner} that works for most - * {@link OldDoFn OldDoFns}. - * - *

It invokes {@link OldDoFn#processElement} for each input. - */ - public static DoFnRunner simpleRunner( - PipelineOptions options, - OldDoFn fn, - SideInputReader sideInputReader, - OutputManager outputManager, - TupleTag mainOutputTag, - List> sideOutputTags, - StepContext stepContext, - CounterSet.AddCounterMutator addCounterMutator, - WindowingStrategy windowingStrategy) { - return simpleRunner(options, - fn, - sideInputReader, - outputManager, - mainOutputTag, - sideOutputTags, - stepContext, - CounterAggregator.factoryFor(addCounterMutator), - windowingStrategy); - } - /** * Returns an implementation of {@link DoFnRunner} that handles late data dropping. * @@ -132,33 +103,6 @@ DoFnRunner, KV> lateDataDroppingRunner( reduceFnExecutor.getDroppedDueToLatenessAggregator()); } - /** - * Returns an implementation of {@link DoFnRunner} that handles late data dropping. - * - *

It drops elements from expired windows before they reach the underlying {@link OldDoFn}. - */ - public static - DoFnRunner, KV> lateDataDroppingRunner( - PipelineOptions options, - ReduceFnExecutor reduceFnExecutor, - SideInputReader sideInputReader, - OutputManager outputManager, - TupleTag> mainOutputTag, - List> sideOutputTags, - StepContext stepContext, - CounterSet.AddCounterMutator addCounterMutator, - WindowingStrategy windowingStrategy) { - return lateDataDroppingRunner( - options, - reduceFnExecutor, - sideInputReader, - outputManager, - mainOutputTag, - sideOutputTags, - stepContext, - CounterAggregator.factoryFor(addCounterMutator), - windowingStrategy); - } public static DoFnRunner createDefault( PipelineOptions options, @@ -197,26 +141,4 @@ public static DoFnRunner createDefault( aggregatorFactory, windowingStrategy); } - - public static DoFnRunner createDefault( - PipelineOptions options, - OldDoFn doFn, - SideInputReader sideInputReader, - OutputManager outputManager, - TupleTag mainOutputTag, - List> sideOutputTags, - StepContext stepContext, - AddCounterMutator addCounterMutator, - WindowingStrategy windowingStrategy) { - return createDefault( - options, - doFn, - sideInputReader, - outputManager, - mainOutputTag, - sideOutputTags, - stepContext, - CounterAggregator.factoryFor(addCounterMutator), - windowingStrategy); - } } 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 76dbecf433b2b..62222897184bc 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 @@ -211,9 +211,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-20160804-dofn"; + "dataflow.gcr.io/v1beta3/beam-java-batch:beam-master-20160810"; public static final String STREAMING_WORKER_HARNESS_CONTAINER_IMAGE = - "dataflow.gcr.io/v1beta3/beam-java-streaming:beam-master-20160804-dofn"; + "dataflow.gcr.io/v1beta3/beam-java-streaming:beam-master-20160810"; // The limit of CreateJob request size. private static final int CREATE_JOB_REQUEST_LIMIT_BYTES = 10 * 1024 * 1024; 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 a8258000b6144..6ba3f8a8964df 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 @@ -49,7 +49,6 @@ import org.apache.beam.sdk.util.PropertyNames; import org.apache.beam.sdk.util.SerializableUtils; import org.apache.beam.sdk.util.WindowingStrategy; -import org.apache.beam.sdk.util.common.Counter; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionList; @@ -736,10 +735,6 @@ private static int[] wrap(int value) { return new int[] { value }; } - public Counter getCounter(@SuppressWarnings("unused") String name) { - throw new UnsupportedOperationException("BinaryCombineIntegerFn does not support getCounter"); - } - private static final class ToIntegerCodingFunction implements DelegateCoder.CodingFunction { @Override @@ -839,10 +834,6 @@ private static long[] wrap(long value) { return new long[] { value }; } - public Counter getCounter(@SuppressWarnings("unused") String name) { - throw new UnsupportedOperationException("BinaryCombineLongFn does not support getCounter"); - } - private static final class ToLongCodingFunction implements DelegateCoder.CodingFunction { @Override @@ -944,10 +935,6 @@ private static double[] wrap(double value) { return new double[] { value }; } - public Counter getCounter(@SuppressWarnings("unused") String name) { - throw new UnsupportedOperationException("BinaryCombineDoubleFn does not support getCounter"); - } - private static final class ToDoubleCodingFunction implements DelegateCoder.CodingFunction { @Override diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Max.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Max.java index 52617b65705e8..eed13fbb70b4e 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Max.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Max.java @@ -19,9 +19,6 @@ import org.apache.beam.sdk.transforms.Combine.BinaryCombineFn; import org.apache.beam.sdk.transforms.display.DisplayData; -import org.apache.beam.sdk.util.common.Counter; -import org.apache.beam.sdk.util.common.Counter.AggregationKind; -import org.apache.beam.sdk.util.common.CounterProvider; import java.io.Serializable; import java.util.Comparator; @@ -218,8 +215,7 @@ public void populateDisplayData(DisplayData.Builder builder) { * A {@code CombineFn} that computes the maximum of a collection of {@code Integer}s, useful as an * argument to {@link Combine#globally} or {@link Combine#perKey}. */ - public static class MaxIntegerFn extends Combine.BinaryCombineIntegerFn - implements CounterProvider { + public static class MaxIntegerFn extends Combine.BinaryCombineIntegerFn { @Override public int apply(int left, int right) { return left >= right ? left : right; @@ -229,19 +225,13 @@ public int apply(int left, int right) { public int identity() { return Integer.MIN_VALUE; } - - @Override - public Counter getCounter(String name) { - return Counter.ints(name, AggregationKind.MAX); - } } /** * A {@code CombineFn} that computes the maximum of a collection of {@code Long}s, useful as an * argument to {@link Combine#globally} or {@link Combine#perKey}. */ - public static class MaxLongFn extends Combine.BinaryCombineLongFn - implements CounterProvider { + public static class MaxLongFn extends Combine.BinaryCombineLongFn { @Override public long apply(long left, long right) { return left >= right ? left : right; @@ -251,19 +241,13 @@ public long apply(long left, long right) { public long identity() { return Long.MIN_VALUE; } - - @Override - public Counter getCounter(String name) { - return Counter.longs(name, AggregationKind.MAX); - } } /** * A {@code CombineFn} that computes the maximum of a collection of {@code Double}s, useful as an * argument to {@link Combine#globally} or {@link Combine#perKey}. */ - public static class MaxDoubleFn extends Combine.BinaryCombineDoubleFn - implements CounterProvider { + public static class MaxDoubleFn extends Combine.BinaryCombineDoubleFn { @Override public double apply(double left, double right) { return left >= right ? left : right; @@ -273,10 +257,5 @@ public double apply(double left, double right) { public double identity() { return Double.NEGATIVE_INFINITY; } - - @Override - public Counter getCounter(String name) { - return Counter.doubles(name, AggregationKind.MAX); - } } } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Min.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Min.java index 31591348a56d6..9c9d14f3ad01a 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Min.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Min.java @@ -19,9 +19,6 @@ import org.apache.beam.sdk.transforms.Combine.BinaryCombineFn; import org.apache.beam.sdk.transforms.display.DisplayData; -import org.apache.beam.sdk.util.common.Counter; -import org.apache.beam.sdk.util.common.Counter.AggregationKind; -import org.apache.beam.sdk.util.common.CounterProvider; import java.io.Serializable; import java.util.Comparator; @@ -218,8 +215,7 @@ public void populateDisplayData(DisplayData.Builder builder) { * A {@code CombineFn} that computes the minimum of a collection of {@code Integer}s, useful as an * argument to {@link Combine#globally} or {@link Combine#perKey}. */ - public static class MinIntegerFn extends Combine.BinaryCombineIntegerFn - implements CounterProvider { + public static class MinIntegerFn extends Combine.BinaryCombineIntegerFn { @Override public int apply(int left, int right) { @@ -230,20 +226,13 @@ public int apply(int left, int right) { public int identity() { return Integer.MAX_VALUE; } - - @Override - public Counter getCounter(String name) { - return Counter.ints(name, AggregationKind.MIN); - } } /** * A {@code CombineFn} that computes the minimum of a collection of {@code Long}s, useful as an * argument to {@link Combine#globally} or {@link Combine#perKey}. */ - public static class MinLongFn extends Combine.BinaryCombineLongFn - implements CounterProvider { - + public static class MinLongFn extends Combine.BinaryCombineLongFn { @Override public long apply(long left, long right) { return left <= right ? left : right; @@ -253,19 +242,13 @@ public long apply(long left, long right) { public long identity() { return Long.MAX_VALUE; } - - @Override - public Counter getCounter(String name) { - return Counter.longs(name, AggregationKind.MIN); - } } /** * A {@code CombineFn} that computes the minimum of a collection of {@code Double}s, useful as an * argument to {@link Combine#globally} or {@link Combine#perKey}. */ - public static class MinDoubleFn extends Combine.BinaryCombineDoubleFn - implements CounterProvider { + public static class MinDoubleFn extends Combine.BinaryCombineDoubleFn { @Override public double apply(double left, double right) { @@ -276,10 +259,5 @@ public double apply(double left, double right) { public double identity() { return Double.POSITIVE_INFINITY; } - - @Override - public Counter getCounter(String name) { - return Counter.doubles(name, AggregationKind.MIN); - } } } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Sum.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Sum.java index 07f78b50655a0..27c5ced7a9d39 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Sum.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Sum.java @@ -17,10 +17,6 @@ */ package org.apache.beam.sdk.transforms; -import org.apache.beam.sdk.util.common.Counter; -import org.apache.beam.sdk.util.common.Counter.AggregationKind; -import org.apache.beam.sdk.util.common.CounterProvider; - /** * {@code PTransform}s for computing the sum of the elements in a * {@code PCollection}, or the sum of the values associated with @@ -123,8 +119,7 @@ public static Combine.PerKey doublesPerKey() { * {@code Iterable} of {@code Integer}s, useful as an argument to * {@link Combine#globally} or {@link Combine#perKey}. */ - public static class SumIntegerFn - extends Combine.BinaryCombineIntegerFn implements CounterProvider { + public static class SumIntegerFn extends Combine.BinaryCombineIntegerFn { @Override public int apply(int a, int b) { return a + b; @@ -134,11 +129,6 @@ public int apply(int a, int b) { public int identity() { return 0; } - - @Override - public Counter getCounter(String name) { - return Counter.ints(name, AggregationKind.SUM); - } } /** @@ -147,7 +137,7 @@ public Counter getCounter(String name) { * {@link Combine#globally} or {@link Combine#perKey}. */ public static class SumLongFn - extends Combine.BinaryCombineLongFn implements CounterProvider { + extends Combine.BinaryCombineLongFn { @Override public long apply(long a, long b) { return a + b; @@ -157,11 +147,6 @@ public long apply(long a, long b) { public long identity() { return 0; } - - @Override - public Counter getCounter(String name) { - return Counter.longs(name, AggregationKind.SUM); - } } /** @@ -169,8 +154,7 @@ public Counter getCounter(String name) { * {@code Iterable} of {@code Double}s, useful as an argument to * {@link Combine#globally} or {@link Combine#perKey}. */ - public static class SumDoubleFn - extends Combine.BinaryCombineDoubleFn implements CounterProvider { + public static class SumDoubleFn extends Combine.BinaryCombineDoubleFn { @Override public double apply(double a, double b) { return a + b; @@ -180,10 +164,5 @@ public double apply(double a, double b) { public double identity() { return 0; } - - @Override - public Counter getCounter(String name) { - return Counter.doubles(name, AggregationKind.SUM); - } } } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/CounterAggregator.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/CounterAggregator.java deleted file mode 100644 index 5bde8ef457bb9..0000000000000 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/CounterAggregator.java +++ /dev/null @@ -1,128 +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; - -import org.apache.beam.sdk.transforms.Aggregator; -import org.apache.beam.sdk.transforms.Combine.CombineFn; -import org.apache.beam.sdk.transforms.Max; -import org.apache.beam.sdk.transforms.Min; -import org.apache.beam.sdk.transforms.Sum; -import org.apache.beam.sdk.util.common.Counter; -import org.apache.beam.sdk.util.common.CounterProvider; -import org.apache.beam.sdk.util.common.CounterSet; -import org.apache.beam.sdk.util.common.CounterSet.AddCounterMutator; -import com.google.common.annotations.VisibleForTesting; - -/** - * An implementation of the {@code Aggregator} interface that uses a - * {@link Counter} as the underlying representation. Supports {@link CombineFn}s - * from the {@link Sum}, {@link Min} and {@link Max} classes. - * - * @param the type of input values - * @param the type of accumulator values - * @param the type of output value - */ -public class CounterAggregator - implements Aggregator { - - private static class CounterAggregatorFactory implements AggregatorFactory { - private final AddCounterMutator addCounterMutator; - - private CounterAggregatorFactory(CounterSet.AddCounterMutator addCounterMutator) { - this.addCounterMutator = addCounterMutator; - } - - @Override - public Aggregator createAggregatorForDoFn( - Class fnClass, ExecutionContext.StepContext stepContext, - String userName, CombineFn combine) { - boolean isSystem = fnClass.isAnnotationPresent(SystemDoFnInternal.class); - String mangledName = (isSystem ? "" : "user-") + stepContext.getStepName() + "-" + userName; - - return new CounterAggregator<>(mangledName, combine, addCounterMutator); - } - } - - private final Counter counter; - private final CombineFn combiner; - - /** - * Create a factory for producing {@link CounterAggregator CounterAggregators} backed by the given - * {@link CounterSet.AddCounterMutator}. - */ - public static AggregatorFactory factoryFor( - CounterSet.AddCounterMutator addCounterMutator) { - return new CounterAggregatorFactory(addCounterMutator); - } - - /** - * Constructs a new aggregator with the given name and aggregation logic - * specified in the CombineFn argument. The underlying counter is - * automatically added into the provided CounterSet. - * - *

If a counter with the same name already exists, it will be reused, as - * long as it has the same type. - */ - @VisibleForTesting CounterAggregator( - String name, CombineFn combiner, - CounterSet.AddCounterMutator addCounterMutator) { - // Safe contravariant cast - this(constructCounter(name, combiner), addCounterMutator, - (CombineFn) combiner); - } - - private CounterAggregator(Counter counter, - CounterSet.AddCounterMutator addCounterMutator, - CombineFn combiner) { - try { - this.counter = addCounterMutator.addCounter(counter); - } catch (IllegalArgumentException ex) { - throw new IllegalArgumentException( - "aggregator's name collides with an existing aggregator " - + "or system-provided counter of an incompatible type"); - } - this.combiner = combiner; - } - - private static Counter constructCounter(String name, - CombineFn combiner) { - if (combiner instanceof CounterProvider) { - @SuppressWarnings("unchecked") - CounterProvider counterProvider = (CounterProvider) combiner; - return counterProvider.getCounter(name); - } else { - throw new IllegalArgumentException("unsupported combiner in Aggregator: " - + combiner.getClass().getName()); - } - } - - @Override - public void addValue(InputT value) { - counter.addValue(value); - } - - @Override - public String getName() { - return counter.getFlatName(); - } - - @Override - public CombineFn getCombineFn() { - return combiner; - } -} diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/common/Counter.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/common/Counter.java deleted file mode 100644 index 550c648beb385..0000000000000 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/common/Counter.java +++ /dev/null @@ -1,1287 +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.common; - -import static org.apache.beam.sdk.util.common.Counter.AggregationKind.AND; -import static org.apache.beam.sdk.util.common.Counter.AggregationKind.MEAN; -import static org.apache.beam.sdk.util.common.Counter.AggregationKind.OR; - -import static com.google.common.base.Preconditions.checkArgument; - -import org.apache.beam.sdk.values.TypeDescriptor; - -import com.google.common.annotations.VisibleForTesting; -import com.google.common.util.concurrent.AtomicDouble; - -import java.util.Objects; -import java.util.concurrent.atomic.AtomicBoolean; -import java.util.concurrent.atomic.AtomicInteger; -import java.util.concurrent.atomic.AtomicLong; -import java.util.concurrent.atomic.AtomicReference; - -import javax.annotation.Nullable; - -/** - * A Counter enables the aggregation of a stream of values over time. The - * cumulative aggregate value is updated as new values are added, or it can be - * reset to a new value. Multiple kinds of aggregation are supported depending - * on the type of the counter. - * - *

Counters compare using value equality of their name, kind, and - * cumulative value. Equal counters should have equal toString()s. - * - *

After all possible mutations have completed, the reader should check - * {@link #isDirty} for every counter, otherwise updates may be lost. - * - *

A counter may become dirty without a corresponding update to the value. - * This generally will occur when the calls to {@code addValue()}, {@code committing()}, - * and {@code committed()} are interleaved such that the value is updated - * between the calls to committing and the read of the value. - * - * @param the type of values aggregated by this counter - */ -public abstract class Counter { - /** - * Possible kinds of counter aggregation. - */ - public static enum AggregationKind { - - /** - * Computes the sum of all added values. - * Applicable to {@link Integer}, {@link Long}, and {@link Double} values. - */ - SUM, - - /** - * Computes the maximum value of all added values. - * Applicable to {@link Integer}, {@link Long}, and {@link Double} values. - */ - MAX, - - /** - * Computes the minimum value of all added values. - * Applicable to {@link Integer}, {@link Long}, and {@link Double} values. - */ - MIN, - - /** - * Computes the arithmetic mean of all added values. Applicable to - * {@link Integer}, {@link Long}, and {@link Double} values. - */ - MEAN, - - /** - * Computes boolean AND over all added values. - * Applicable only to {@link Boolean} values. - */ - AND, - - /** - * Computes boolean OR over all added values. Applicable only to - * {@link Boolean} values. - */ - OR - // TODO: consider adding VECTOR_SUM, HISTOGRAM, KV_SET, PRODUCT, TOP. - } - - /** - * Constructs a new {@link Counter} that aggregates {@link Integer}, values - * according to the desired aggregation kind. The supported aggregation kinds - * are {@link AggregationKind#SUM}, {@link AggregationKind#MIN}, - * {@link AggregationKind#MAX}, and {@link AggregationKind#MEAN}. - * This is a convenience wrapper over a - * {@link Counter} implementation that aggregates {@link Long} values. This is - * useful when the application handles (boxed) {@link Integer} values that - * are not readily convertible to the (boxed) {@link Long} values otherwise - * expected by the {@link Counter} implementation aggregating {@link Long} - * values. - * - * @param name the name of the new counter - * @param kind the new counter's aggregation kind - * @return the newly constructed Counter - * @throws IllegalArgumentException if the aggregation kind is not supported - */ - public static Counter ints(CounterName name, AggregationKind kind) { - return new IntegerCounter(name, kind); - } - - /** - * Constructs a new {@code Counter} with an unstructured name. - * - * @deprecated use {@link #ints(CounterName, AggregationKind)}. - */ - @Deprecated - public static Counter ints(String name, AggregationKind kind) { - return new IntegerCounter(CounterName.named(name), kind); - } - - /** - * Constructs a new {@link Counter} that aggregates {@link Long} values - * according to the desired aggregation kind. The supported aggregation kinds - * are {@link AggregationKind#SUM}, {@link AggregationKind#MIN}, - * {@link AggregationKind#MAX}, and {@link AggregationKind#MEAN}. - * - * @param name the name of the new counter - * @param kind the new counter's aggregation kind - * @return the newly constructed Counter - * @throws IllegalArgumentException if the aggregation kind is not supported - */ - public static Counter longs(CounterName name, AggregationKind kind) { - return new LongCounter(name, kind); - } - - /** - * Constructs a new {@code Counter} with an unstructured name. - * - * @deprecated use {@link #longs(CounterName, AggregationKind)}. - */ - @Deprecated - public static Counter longs(String name, AggregationKind kind) { - return new LongCounter(CounterName.named(name), kind); - } - - /** - * Constructs a new {@link Counter} that aggregates {@link Double} values - * according to the desired aggregation kind. The supported aggregation kinds - * are {@link AggregationKind#SUM}, {@link AggregationKind#MIN}, - * {@link AggregationKind#MAX}, and {@link AggregationKind#MEAN}. - * - * @param name the name of the new counter - * @param kind the new counter's aggregation kind - * @return the newly constructed Counter - * @throws IllegalArgumentException if the aggregation kind is not supported - */ - public static Counter doubles(CounterName name, AggregationKind kind) { - return new DoubleCounter(name, kind); - } - - /** - * Constructs a new {@code Counter} with an unstructured name. - * - * @deprecated use {@link #doubles(CounterName, AggregationKind)}. - */ - @Deprecated - public static Counter doubles(String name, AggregationKind kind) { - return new DoubleCounter(CounterName.named(name), kind); - } - - /** - * Constructs a new {@link Counter} that aggregates {@link Boolean} values - * according to the desired aggregation kind. The only supported aggregation - * kinds are {@link AggregationKind#AND} and {@link AggregationKind#OR}. - * - * @param name the name of the new counter - * @param kind the new counter's aggregation kind - * @return the newly constructed Counter - * @throws IllegalArgumentException if the aggregation kind is not supported - */ - public static Counter booleans(CounterName name, AggregationKind kind) { - return new BooleanCounter(name, kind); - } - - /** - * Constructs a new {@code Counter} with an unstructured name. - * - * @deprecated use {@link #booleans(CounterName, AggregationKind)}. - */ - @Deprecated - public static Counter booleans(String name, AggregationKind kind) { - return new BooleanCounter(CounterName.named(name), kind); - } - - ////////////////////////////////////////////////////////////////////////////// - - /** - * Adds a new value to the aggregation stream. Returns this (to allow method - * chaining). - */ - public abstract Counter addValue(T value); - - /** - * Resets the aggregation stream to this new value. This aggregator must not - * be a MEAN aggregator. Returns this (to allow method chaining). - */ - public abstract Counter resetToValue(T value); - - /** - * Resets the aggregation stream to this new value. Returns this (to allow - * method chaining). The value of elementCount must be non-negative, and this - * aggregator must be a MEAN aggregator. - */ - public abstract Counter resetMeanToValue(long elementCount, T value); - - /** - * Resets the counter's delta value to have no values accumulated and returns - * the value of the delta prior to the reset. - * - * @return the aggregate delta at the time this method is called - */ - public abstract T getAndResetDelta(); - - /** - * Resets the counter's delta value to have no values accumulated and returns - * the value of the delta prior to the reset, for a MEAN counter. - * - * @return the mean delta t the time this method is called - */ - public abstract CounterMean getAndResetMeanDelta(); - - /** - * Returns the counter's flat name. - */ - public String getFlatName() { - return name.getFlatName(); - } - - /** - * Returns the counter's name. - * - * @deprecated use {@link #getFlatName}. - */ - @Deprecated - public String getName() { - return name.getFlatName(); - } - - /** - * Returns the counter's aggregation kind. - */ - public AggregationKind getKind() { - return kind; - } - - /** - * Returns the counter's type. - */ - public Class getType() { - return new TypeDescriptor(getClass()) {}.getRawType(); - } - - /** - * Returns the aggregated value, or the sum for MEAN aggregation, either - * total or, if delta, since the last update extraction or resetDelta. - */ - public abstract T getAggregate(); - - /** - * The mean value of a {@code Counter}, represented as an aggregate value and - * a count. - * - * @param the type of the aggregate - */ - public static interface CounterMean { - /** - * Gets the aggregate value of this {@code CounterMean}. - */ - T getAggregate(); - - /** - * Gets the count of this {@code CounterMean}. - */ - long getCount(); - } - - /** - * Returns the mean in the form of a CounterMean, or null if this is not a - * MEAN counter. - */ - @Nullable - public abstract CounterMean getMean(); - - /** - * Represents whether counters' values have been committed to the backend. - * - *

Runners can use this information to optimize counters updates. - * For example, if counters are committed, runners may choose to skip the updates. - * - *

Counters' state transition table: - * {@code - * Action\Current State COMMITTED DIRTY COMMITTING - * addValue() DIRTY DIRTY DIRTY - * committing() None COMMITTING None - * committed() None None COMMITTED - * } - */ - @VisibleForTesting - enum CommitState { - /** - * There are no local updates that haven't been committed to the backend. - */ - COMMITTED, - /** - * There are local updates that haven't been committed to the backend. - */ - DIRTY, - /** - * Local updates are committing to the backend, but are still pending. - */ - COMMITTING, - } - - /** - * Returns if the counter contains non-committed aggregate. - */ - public boolean isDirty() { - return commitState.get() != CommitState.COMMITTED; - } - - /** - * Changes the counter from {@code CommitState.DIRTY} to {@code CommitState.COMMITTING}. - * - * @return true if successful. False return indicates that the commit state - * was not in {@code CommitState.DIRTY}. - */ - public boolean committing() { - return commitState.compareAndSet(CommitState.DIRTY, CommitState.COMMITTING); - } - - /** - * Changes the counter from {@code CommitState.COMMITTING} to {@code CommitState.COMMITTED}. - * - * @return true if successful. - * - *

False return indicates that the counter was updated while the committing is pending. - * That counter update might or might not has been committed. The {@code commitState} has to - * stay in {@code CommitState.DIRTY}. - */ - public boolean committed() { - return commitState.compareAndSet(CommitState.COMMITTING, CommitState.COMMITTED); - } - - /** - * Sets the counter to {@code CommitState.DIRTY}. - * - *

Must be called at the end of {@link #addValue}, {@link #resetToValue}, - * {@link #resetMeanToValue}, and {@link #merge}. - */ - protected void setDirty() { - commitState.set(CommitState.DIRTY); - } - - /** - * Returns a string representation of the Counter. Useful for debugging logs. - * Example return value: "ElementCount:SUM(15)". - */ - @Override - public String toString() { - StringBuilder sb = new StringBuilder(); - sb.append(getFlatName()); - sb.append(":"); - sb.append(getKind()); - sb.append("("); - switch (kind) { - case SUM: - case MAX: - case MIN: - case AND: - case OR: - sb.append(getAggregate()); - break; - case MEAN: - sb.append(getMean()); - break; - default: - throw illegalArgumentException(); - } - sb.append(")"); - - return sb.toString(); - } - - @Override - public boolean equals(Object o) { - if (this == o) { - return true; - } else if (o instanceof Counter) { - Counter that = (Counter) o; - if (this.name.equals(that.name) && this.kind == that.kind - && this.getClass().equals(that.getClass())) { - if (kind == MEAN) { - CounterMean thisMean = this.getMean(); - CounterMean thatMean = that.getMean(); - return thisMean == thatMean - || (Objects.equals(thisMean.getAggregate(), thatMean.getAggregate()) - && thisMean.getCount() == thatMean.getCount()); - } else { - return Objects.equals(this.getAggregate(), that.getAggregate()); - } - } - } - return false; - } - - @Override - public int hashCode() { - if (kind == MEAN) { - CounterMean mean = getMean(); - return Objects.hash(getClass(), name, kind, mean.getAggregate(), mean.getCount()); - } else { - return Objects.hash(getClass(), name, kind, getAggregate()); - } - } - - /** - * Returns whether this Counter is compatible with that Counter. If - * so, they can be merged into a single Counter. - */ - public boolean isCompatibleWith(Counter that) { - return this.name.equals(that.name) - && this.kind == that.kind - && this.getClass().equals(that.getClass()); - } - - /** - * Merges this counter with the provided counter, returning this counter with the combined value - * of both counters. This may reset the delta of this counter. - * - * @throws IllegalArgumentException if the provided Counter is not compatible with this Counter - */ - public abstract Counter merge(Counter that); - - ////////////////////////////////////////////////////////////////////////////// - - /** The name and metadata of this counter. */ - protected final CounterName name; - - /** The kind of aggregation function to apply to this counter. */ - protected final AggregationKind kind; - - /** The commit state of this counter. */ - protected final AtomicReference commitState; - - protected Counter(CounterName name, AggregationKind kind) { - this.name = name; - this.kind = kind; - this.commitState = new AtomicReference<>(CommitState.COMMITTED); - } - - ////////////////////////////////////////////////////////////////////////////// - - /** - * Implements a {@link Counter} for {@link Long} values. - */ - private static class LongCounter extends Counter { - private final AtomicLong aggregate; - private final AtomicLong deltaAggregate; - private final AtomicReference mean; - private final AtomicReference deltaMean; - - /** Initializes a new {@link Counter} for {@link Long} values. */ - private LongCounter(CounterName name, AggregationKind kind) { - super(name, kind); - switch (kind) { - case MEAN: - mean = new AtomicReference<>(); - deltaMean = new AtomicReference<>(); - getAndResetMeanDelta(); - mean.set(deltaMean.get()); - aggregate = deltaAggregate = null; - break; - case SUM: - case MAX: - case MIN: - aggregate = new AtomicLong(); - deltaAggregate = new AtomicLong(); - getAndResetDelta(); - aggregate.set(deltaAggregate.get()); - mean = deltaMean = null; - break; - default: - throw illegalArgumentException(); - } - } - - @Override - public LongCounter addValue(Long value) { - try { - switch (kind) { - case SUM: - aggregate.addAndGet(value); - deltaAggregate.addAndGet(value); - break; - case MEAN: - addToMeanAndSet(value, mean); - addToMeanAndSet(value, deltaMean); - break; - case MAX: - maxAndSet(value, aggregate); - maxAndSet(value, deltaAggregate); - break; - case MIN: - minAndSet(value, aggregate); - minAndSet(value, deltaAggregate); - break; - default: - throw illegalArgumentException(); - } - return this; - } finally { - setDirty(); - } - } - - private void minAndSet(Long value, AtomicLong target) { - long current; - long update; - do { - current = target.get(); - update = Math.min(value, current); - } while (update < current && !target.compareAndSet(current, update)); - } - - private void maxAndSet(Long value, AtomicLong target) { - long current; - long update; - do { - current = target.get(); - update = Math.max(value, current); - } while (update > current && !target.compareAndSet(current, update)); - } - - private void addToMeanAndSet(Long value, AtomicReference target) { - LongCounterMean current; - LongCounterMean update; - do { - current = target.get(); - update = new LongCounterMean(current.getAggregate() + value, current.getCount() + 1L); - } while (!target.compareAndSet(current, update)); - } - - @Override - public Long getAggregate() { - if (kind != MEAN) { - return aggregate.get(); - } else { - return getMean().getAggregate(); - } - } - - @Override - public Long getAndResetDelta() { - switch (kind) { - case SUM: - return deltaAggregate.getAndSet(0L); - case MAX: - return deltaAggregate.getAndSet(Long.MIN_VALUE); - case MIN: - return deltaAggregate.getAndSet(Long.MAX_VALUE); - default: - throw illegalArgumentException(); - } - } - - @Override - public Counter resetToValue(Long value) { - try { - if (kind == MEAN) { - throw illegalArgumentException(); - } - aggregate.set(value); - deltaAggregate.set(value); - return this; - } finally { - setDirty(); - } - } - - @Override - public Counter resetMeanToValue(long elementCount, Long value) { - try { - if (kind != MEAN) { - throw illegalArgumentException(); - } - if (elementCount < 0) { - throw new IllegalArgumentException("elementCount must be non-negative"); - } - LongCounterMean counterMean = new LongCounterMean(value, elementCount); - mean.set(counterMean); - deltaMean.set(counterMean); - return this; - } finally { - setDirty(); - } - } - - @Override - public CounterMean getAndResetMeanDelta() { - if (kind != MEAN) { - throw illegalArgumentException(); - } - return deltaMean.getAndSet(new LongCounterMean(0L, 0L)); - } - - @Override - @Nullable - public CounterMean getMean() { - if (kind != MEAN) { - throw illegalArgumentException(); - } - return mean.get(); - } - - @Override - public Counter merge(Counter that) { - try { - checkArgument( - this.isCompatibleWith(that), "Counters %s and %s are incompatible", this, that); - switch (kind) { - case SUM: - case MIN: - case MAX: - return addValue(that.getAggregate()); - case MEAN: - CounterMean thisCounterMean = this.getMean(); - CounterMean thatCounterMean = that.getMean(); - return resetMeanToValue( - thisCounterMean.getCount() + thatCounterMean.getCount(), - thisCounterMean.getAggregate() + thatCounterMean.getAggregate()); - default: - throw illegalArgumentException(); - } - } finally { - setDirty(); - } - } - - private static class LongCounterMean implements CounterMean { - private final long aggregate; - private final long count; - - public LongCounterMean(long aggregate, long count) { - this.aggregate = aggregate; - this.count = count; - } - - @Override - public Long getAggregate() { - return aggregate; - } - - @Override - public long getCount() { - return count; - } - - @Override - public String toString() { - return aggregate + "/" + count; - } - } - } - - /** - * Implements a {@link Counter} for {@link Double} values. - */ - private static class DoubleCounter extends Counter { - AtomicDouble aggregate; - AtomicDouble deltaAggregate; - AtomicReference mean; - AtomicReference deltaMean; - - /** Initializes a new {@link Counter} for {@link Double} values. */ - private DoubleCounter(CounterName name, AggregationKind kind) { - super(name, kind); - switch (kind) { - case MEAN: - aggregate = deltaAggregate = null; - mean = new AtomicReference<>(); - deltaMean = new AtomicReference<>(); - getAndResetMeanDelta(); - mean.set(deltaMean.get()); - break; - case SUM: - case MAX: - case MIN: - mean = deltaMean = null; - aggregate = new AtomicDouble(); - deltaAggregate = new AtomicDouble(); - getAndResetDelta(); - aggregate.set(deltaAggregate.get()); - break; - default: - throw illegalArgumentException(); - } - } - - @Override - public DoubleCounter addValue(Double value) { - try { - switch (kind) { - case SUM: - aggregate.addAndGet(value); - deltaAggregate.addAndGet(value); - break; - case MEAN: - addToMeanAndSet(value, mean); - addToMeanAndSet(value, deltaMean); - break; - case MAX: - maxAndSet(value, aggregate); - maxAndSet(value, deltaAggregate); - break; - case MIN: - minAndSet(value, aggregate); - minAndSet(value, deltaAggregate); - break; - default: - throw illegalArgumentException(); - } - return this; - } finally { - setDirty(); - } - } - - private void addToMeanAndSet(Double value, AtomicReference target) { - DoubleCounterMean current; - DoubleCounterMean update; - do { - current = target.get(); - update = new DoubleCounterMean(current.getAggregate() + value, current.getCount() + 1); - } while (!target.compareAndSet(current, update)); - } - - private void maxAndSet(Double value, AtomicDouble target) { - double current; - double update; - do { - current = target.get(); - update = Math.max(current, value); - } while (update > current && !target.compareAndSet(current, update)); - } - - private void minAndSet(Double value, AtomicDouble target) { - double current; - double update; - do { - current = target.get(); - update = Math.min(current, value); - } while (update < current && !target.compareAndSet(current, update)); - } - - @Override - public Double getAndResetDelta() { - switch (kind) { - case SUM: - return deltaAggregate.getAndSet(0.0); - case MAX: - return deltaAggregate.getAndSet(Double.NEGATIVE_INFINITY); - case MIN: - return deltaAggregate.getAndSet(Double.POSITIVE_INFINITY); - default: - throw illegalArgumentException(); - } - } - - @Override - public Counter resetToValue(Double value) { - try { - if (kind == MEAN) { - throw illegalArgumentException(); - } - aggregate.set(value); - deltaAggregate.set(value); - return this; - } finally { - setDirty(); - } - } - - @Override - public Counter resetMeanToValue(long elementCount, Double value) { - try { - if (kind != MEAN) { - throw illegalArgumentException(); - } - if (elementCount < 0) { - throw new IllegalArgumentException("elementCount must be non-negative"); - } - DoubleCounterMean counterMean = new DoubleCounterMean(value, elementCount); - mean.set(counterMean); - deltaMean.set(counterMean); - return this; - } finally { - setDirty(); - } - } - - @Override - public CounterMean getAndResetMeanDelta() { - if (kind != MEAN) { - throw illegalArgumentException(); - } - return deltaMean.getAndSet(new DoubleCounterMean(0.0, 0L)); - } - - @Override - public Double getAggregate() { - if (kind != MEAN) { - return aggregate.get(); - } else { - return getMean().getAggregate(); - } - } - - @Override - @Nullable - public CounterMean getMean() { - if (kind != MEAN) { - throw illegalArgumentException(); - } - return mean.get(); - } - - @Override - public Counter merge(Counter that) { - try { - checkArgument( - this.isCompatibleWith(that), "Counters %s and %s are incompatible", this, that); - switch (kind) { - case SUM: - case MIN: - case MAX: - return addValue(that.getAggregate()); - case MEAN: - CounterMean thisCounterMean = this.getMean(); - CounterMean thatCounterMean = that.getMean(); - return resetMeanToValue( - thisCounterMean.getCount() + thatCounterMean.getCount(), - thisCounterMean.getAggregate() + thatCounterMean.getAggregate()); - default: - throw illegalArgumentException(); - } - } finally { - setDirty(); - } - } - - private static class DoubleCounterMean implements CounterMean { - private final double aggregate; - private final long count; - - public DoubleCounterMean(double aggregate, long count) { - this.aggregate = aggregate; - this.count = count; - } - - @Override - public Double getAggregate() { - return aggregate; - } - - @Override - public long getCount() { - return count; - } - - @Override - public String toString() { - return aggregate + "/" + count; - } - } - } - - /** - * Implements a {@link Counter} for {@link Boolean} values. - */ - private static class BooleanCounter extends Counter { - private final AtomicBoolean aggregate; - private final AtomicBoolean deltaAggregate; - - /** Initializes a new {@link Counter} for {@link Boolean} values. */ - private BooleanCounter(CounterName name, AggregationKind kind) { - super(name, kind); - aggregate = new AtomicBoolean(); - deltaAggregate = new AtomicBoolean(); - getAndResetDelta(); - aggregate.set(deltaAggregate.get()); - } - - @Override - public BooleanCounter addValue(Boolean value) { - try { - if (kind.equals(AND) && !value) { - aggregate.set(value); - deltaAggregate.set(value); - } else if (kind.equals(OR) && value) { - aggregate.set(value); - deltaAggregate.set(value); - } - return this; - } finally { - setDirty(); - } - } - - @Override - public Boolean getAndResetDelta() { - switch (kind) { - case AND: - return deltaAggregate.getAndSet(true); - case OR: - return deltaAggregate.getAndSet(false); - default: - throw illegalArgumentException(); - } - } - - @Override - public Counter resetToValue(Boolean value) { - try { - aggregate.set(value); - deltaAggregate.set(value); - return this; - } finally { - setDirty(); - } - } - - @Override - public Counter resetMeanToValue(long elementCount, Boolean value) { - throw illegalArgumentException(); - } - - @Override - public CounterMean getAndResetMeanDelta() { - throw illegalArgumentException(); - } - - @Override - public Boolean getAggregate() { - return aggregate.get(); - } - - @Override - @Nullable - public CounterMean getMean() { - throw illegalArgumentException(); - } - - @Override - public Counter merge(Counter that) { - try { - checkArgument( - this.isCompatibleWith(that), "Counters %s and %s are incompatible", this, that); - return addValue(that.getAggregate()); - } finally { - setDirty(); - } - } - } - - /** - * Implements a {@link Counter} for {@link String} values. - */ - private static class StringCounter extends Counter { - /** Initializes a new {@link Counter} for {@link String} values. */ - private StringCounter(CounterName name, AggregationKind kind) { - super(name, kind); - // TODO: Support MIN, MAX of Strings. - throw illegalArgumentException(); - } - - @Override - public StringCounter addValue(String value) { - switch (kind) { - default: - throw illegalArgumentException(); - } - } - - @Override - public Counter resetToValue(String value) { - switch (kind) { - default: - throw illegalArgumentException(); - } - } - - @Override - public Counter resetMeanToValue(long elementCount, String value) { - switch (kind) { - default: - throw illegalArgumentException(); - } - } - - @Override - public String getAndResetDelta() { - switch (kind) { - default: - throw illegalArgumentException(); - } - } - - @Override - public CounterMean getAndResetMeanDelta() { - switch (kind) { - default: - throw illegalArgumentException(); - } - } - - @Override - public String getAggregate() { - switch (kind) { - default: - throw illegalArgumentException(); - } - } - - @Override - @Nullable - public CounterMean getMean() { - switch (kind) { - default: - throw illegalArgumentException(); - } - } - - @Override - public Counter merge(Counter that) { - checkArgument(this.isCompatibleWith(that), "Counters %s and %s are incompatible", this, that); - switch (kind) { - default: - throw illegalArgumentException(); - } - } - } - - /** - * Implements a {@link Counter} for {@link Integer} values. - */ - private static class IntegerCounter extends Counter { - private final AtomicInteger aggregate; - private final AtomicInteger deltaAggregate; - private final AtomicReference mean; - private final AtomicReference deltaMean; - - /** Initializes a new {@link Counter} for {@link Integer} values. */ - private IntegerCounter(CounterName name, AggregationKind kind) { - super(name, kind); - switch (kind) { - case MEAN: - aggregate = deltaAggregate = null; - mean = new AtomicReference<>(); - deltaMean = new AtomicReference<>(); - getAndResetMeanDelta(); - mean.set(deltaMean.get()); - break; - case SUM: - case MAX: - case MIN: - mean = deltaMean = null; - aggregate = new AtomicInteger(); - deltaAggregate = new AtomicInteger(); - getAndResetDelta(); - aggregate.set(deltaAggregate.get()); - break; - default: - throw illegalArgumentException(); - } - } - - @Override - public IntegerCounter addValue(Integer value) { - try { - switch (kind) { - case SUM: - aggregate.getAndAdd(value); - deltaAggregate.getAndAdd(value); - break; - case MEAN: - addToMeanAndSet(value, mean); - addToMeanAndSet(value, deltaMean); - break; - case MAX: - maxAndSet(value, aggregate); - maxAndSet(value, deltaAggregate); - break; - case MIN: - minAndSet(value, aggregate); - minAndSet(value, deltaAggregate); - break; - default: - throw illegalArgumentException(); - } - return this; - } finally { - setDirty(); - } - } - - private void addToMeanAndSet(int value, AtomicReference target) { - IntegerCounterMean current; - IntegerCounterMean update; - do { - current = target.get(); - update = new IntegerCounterMean(current.getAggregate() + value, current.getCount() + 1); - } while (!target.compareAndSet(current, update)); - } - - private void maxAndSet(int value, AtomicInteger target) { - int current; - int update; - do { - current = target.get(); - update = Math.max(value, current); - } while (update > current && !target.compareAndSet(current, update)); - } - - private void minAndSet(int value, AtomicInteger target) { - int current; - int update; - do { - current = target.get(); - update = Math.min(value, current); - } while (update < current && !target.compareAndSet(current, update)); - } - - @Override - public Integer getAndResetDelta() { - switch (kind) { - case SUM: - return deltaAggregate.getAndSet(0); - case MAX: - return deltaAggregate.getAndSet(Integer.MIN_VALUE); - case MIN: - return deltaAggregate.getAndSet(Integer.MAX_VALUE); - default: - throw illegalArgumentException(); - } - } - - @Override - public Counter resetToValue(Integer value) { - try { - if (kind == MEAN) { - throw illegalArgumentException(); - } - aggregate.set(value); - deltaAggregate.set(value); - return this; - } finally { - setDirty(); - } - } - - @Override - public Counter resetMeanToValue(long elementCount, Integer value) { - try { - if (kind != MEAN) { - throw illegalArgumentException(); - } - if (elementCount < 0) { - throw new IllegalArgumentException("elementCount must be non-negative"); - } - IntegerCounterMean counterMean = new IntegerCounterMean(value, elementCount); - mean.set(counterMean); - deltaMean.set(counterMean); - return this; - } finally { - setDirty(); - } - } - - @Override - public CounterMean getAndResetMeanDelta() { - if (kind != MEAN) { - throw illegalArgumentException(); - } - return deltaMean.getAndSet(new IntegerCounterMean(0, 0L)); - } - - @Override - public Integer getAggregate() { - if (kind != MEAN) { - return aggregate.get(); - } else { - return getMean().getAggregate(); - } - } - - @Override - @Nullable - public CounterMean getMean() { - if (kind != MEAN) { - throw illegalArgumentException(); - } - return mean.get(); - } - - @Override - public Counter merge(Counter that) { - try { - checkArgument( - this.isCompatibleWith(that), "Counters %s and %s are incompatible", this, that); - switch (kind) { - case SUM: - case MIN: - case MAX: - return addValue(that.getAggregate()); - case MEAN: - CounterMean thisCounterMean = this.getMean(); - CounterMean thatCounterMean = that.getMean(); - return resetMeanToValue( - thisCounterMean.getCount() + thatCounterMean.getCount(), - thisCounterMean.getAggregate() + thatCounterMean.getAggregate()); - default: - throw illegalArgumentException(); - } - } finally { - setDirty(); - } - } - - private static class IntegerCounterMean implements CounterMean { - private final int aggregate; - private final long count; - - public IntegerCounterMean(int aggregate, long count) { - this.aggregate = aggregate; - this.count = count; - } - - @Override - public Integer getAggregate() { - return aggregate; - } - - @Override - public long getCount() { - return count; - } - - @Override - public String toString() { - return aggregate + "/" + count; - } - } - } - - ////////////////////////////////////////////////////////////////////////////// - - /** - * Constructs an {@link IllegalArgumentException} explaining that this - * {@link Counter}'s aggregation kind is not supported by its value type. - */ - protected IllegalArgumentException illegalArgumentException() { - return new IllegalArgumentException("Cannot compute " + kind - + " aggregation over " + getType().getSimpleName() + " values."); - } -} diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/common/CounterName.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/common/CounterName.java deleted file mode 100644 index b46be98af0ed1..0000000000000 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/common/CounterName.java +++ /dev/null @@ -1,153 +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.common; - -import static com.google.common.base.Preconditions.checkNotNull; - -import com.google.common.base.Strings; - -import java.util.concurrent.atomic.AtomicReference; - -/** - * The name of a counter identifies the user-specified name, as well as the origin, - * the step the counter is associated with, and a prefix to add to the name. - * - *

For backwards compatibility, the {@link CounterName} will be converted to - * a flat name (string) during the migration. - */ -public class CounterName { - /** - * Returns a {@link CounterName} with the given name. - */ - public static CounterName named(String name) { - return new CounterName(name, "", "", ""); - } - - /** - * Returns a msecs {@link CounterName}. - */ - public static CounterName msecs(String name) { - return named(name + "-msecs"); - } - - /** - * Returns a {@link CounterName} identical to this, but with the given origin. - */ - public CounterName withOrigin(String origin) { - return new CounterName(this.name, origin, this.stepName, this.prefix); - } - - /** - * Returns a {@link CounterName} identical to this, but with the given step name. - */ - public CounterName withStepName(String stepName) { - return new CounterName(this.name, this.origin, stepName, this.prefix); - } - - /** - * Returns a {@link CounterName} identical to this, but with the given prefix. - */ - public CounterName withPrefix(String prefix) { - return new CounterName(this.name, this.origin, this.stepName, prefix); - } - - /** - * Name of the counter. - * - *

For example, process-msecs, ElementCount. - */ - private final String name; - - /** - * Origin (namespace) of counter name. - * - *

For example, "user" for user-defined counters. - * It is empty for counters defined by the SDK or the runner. - */ - private final String origin; - - /** - * System defined step name or the named-output of a step. - * - *

For example, {@code s1} or {@code s2.out}. - * It may be empty when counters don't associate with step names. - */ - private final String stepName; - - /** - * Prefix of group of counters. - * - *

It is empty when counters don't have general prefixes. - */ - private final String prefix; - - /** - * Flat name is the equivalent unstructured name. - * - *

It is null before {@link #getFlatName()} is called. - */ - private AtomicReference flatName; - - private CounterName(String name, String origin, String stepName, String prefix) { - this.name = checkNotNull(name, "name"); - this.origin = checkNotNull(origin, "origin"); - this.stepName = checkNotNull(stepName, "stepName"); - this.prefix = checkNotNull(prefix, "prefix"); - this.flatName = new AtomicReference(); - } - - /** - * Returns the flat name of a structured counter. - */ - public String getFlatName() { - String ret = flatName.get(); - if (ret == null) { - StringBuilder sb = new StringBuilder(); - if (!Strings.isNullOrEmpty(prefix)) { - // Not all runner versions use "-" to concatenate prefix, it may already have it in it. - sb.append(prefix); - } - if (!Strings.isNullOrEmpty(origin)) { - sb.append(origin + "-"); - } - if (!Strings.isNullOrEmpty(stepName)) { - sb.append(stepName + "-"); - } - sb.append(name); - flatName.compareAndSet(null, sb.toString()); - ret = flatName.get(); - } - return ret; - } - - @Override - public boolean equals(Object o) { - if (this == o) { - return true; - } else if (o instanceof CounterName) { - CounterName that = (CounterName) o; - return this.getFlatName().equals(that.getFlatName()); - } - return false; - } - - @Override - public int hashCode() { - return getFlatName().hashCode(); - } -} diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/common/CounterProvider.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/common/CounterProvider.java deleted file mode 100644 index c2550cd216498..0000000000000 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/common/CounterProvider.java +++ /dev/null @@ -1,27 +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.common; - -/** - * A counter provider can provide {@link Counter} instances. - * - * @param the input type of the counter. - */ -public interface CounterProvider { - Counter getCounter(String name); -} diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/common/CounterSet.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/common/CounterSet.java deleted file mode 100644 index cb0ffe5aa7f3a..0000000000000 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/common/CounterSet.java +++ /dev/null @@ -1,179 +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.common; - -import static com.google.common.base.Preconditions.checkArgument; - -import java.util.AbstractSet; -import java.util.HashMap; -import java.util.Iterator; - -/** - * A CounterSet maintains a set of {@link Counter}s. - * - *

Thread-safe. - */ -public class CounterSet extends AbstractSet> { - - /** Registered counters. */ - private final HashMap> counters = new HashMap<>(); - - private final AddCounterMutator addCounterMutator = new AddCounterMutator(); - - /** - * Constructs a CounterSet containing the given Counters. - */ - public CounterSet(Counter... counters) { - for (Counter counter : counters) { - addNewCounter(counter); - } - } - - /** - * Returns an object that supports adding additional counters into - * this CounterSet. - */ - public AddCounterMutator getAddCounterMutator() { - return addCounterMutator; - } - - /** - * Adds a new counter, throwing an exception if a counter of the - * same name already exists. - */ - public void addNewCounter(Counter counter) { - if (!addCounter(counter)) { - throw new IllegalArgumentException( - "Counter " + counter + " duplicates an existing counter in " + this); - } - } - - /** - * Adds the given Counter to this CounterSet. - * - *

If a counter with the same name already exists, it will be - * reused, as long as it is compatible. - * - * @return the Counter that was reused, or added - * @throws IllegalArgumentException if a counter with the same - * name but an incompatible kind had already been added - */ - public synchronized Counter addOrReuseCounter(Counter counter) { - String flatName = counter.getFlatName(); - Counter oldCounter = counters.get(flatName); - if (oldCounter == null) { - // A new counter. - counters.put(flatName, counter); - return counter; - } - if (counter.isCompatibleWith(oldCounter)) { - // Return the counter to reuse. - @SuppressWarnings("unchecked") - Counter compatibleCounter = (Counter) oldCounter; - return compatibleCounter; - } - throw new IllegalArgumentException( - "Counter " + counter + " duplicates incompatible counter " - + oldCounter + " in " + this); - } - - /** - * Adds a counter. Returns {@code true} if the counter was added to the set - * and false if the given counter was {@code null} or it already existed in - * the set. - * - * @param counter to register - */ - public boolean addCounter(Counter counter) { - return add(counter); - } - - /** - * Returns the Counter with the given name in this CounterSet; - * returns null if no such Counter exists. - */ - public synchronized Counter getExistingCounter(String name) { - return counters.get(name); - } - - @Override - public synchronized Iterator> iterator() { - return counters.values().iterator(); - } - - @Override - public synchronized int size() { - return counters.size(); - } - - @Override - public synchronized boolean add(Counter e) { - if (null == e) { - return false; - } - if (counters.containsKey(e.getFlatName())) { - return false; - } - counters.put(e.getFlatName(), e); - return true; - } - - public synchronized void merge(CounterSet that) { - for (Counter theirCounter : that) { - Counter myCounter = counters.get(theirCounter.getFlatName()); - if (myCounter != null) { - mergeCounters(myCounter, theirCounter); - } else { - addCounter(theirCounter); - } - } - } - - private void mergeCounters(Counter mine, Counter theirCounter) { - checkArgument( - mine.isCompatibleWith(theirCounter), - "Can't merge CounterSets containing incompatible counters with the same name: " - + "%s (existing) and %s (merged)", - mine, - theirCounter); - @SuppressWarnings("unchecked") - Counter theirs = (Counter) theirCounter; - mine.merge(theirs); - } - - /** - * A nested class that supports adding additional counters into the - * enclosing CounterSet. This is useful as a mutator, hiding other - * public methods of the CounterSet. - */ - public class AddCounterMutator { - /** - * Adds the given Counter into the enclosing CounterSet. - * - *

If a counter with the same name already exists, it will be - * reused, as long as it has the same type. - * - * @return the Counter that was reused, or added - * @throws IllegalArgumentException if a counter with the same - * name but an incompatible kind had already been added - */ - public Counter addCounter(Counter counter) { - return addOrReuseCounter(counter); - } - } -} diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/common/ElementByteSizeObserver.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/common/ElementByteSizeObserver.java index 3e7011bb22bdf..388355e8a899e 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/common/ElementByteSizeObserver.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/common/ElementByteSizeObserver.java @@ -19,37 +19,21 @@ import java.util.Observable; import java.util.Observer; -import javax.annotation.Nullable; /** - * An observer that gets notified when additional bytes are read - * and/or used. It adds all bytes into a local counter. When the - * observer gets advanced via the next() call, it adds the total byte - * count to the specified counter, and prepares for the next element. + * An observer that gets notified when additional bytes are read and/or used. */ -public class ElementByteSizeObserver implements Observer { - @Nullable - private final Counter counter; +public abstract class ElementByteSizeObserver implements Observer { private boolean isLazy = false; private long totalSize = 0; private double scalingFactor = 1.0; - public ElementByteSizeObserver() { - this.counter = null; - } - - public ElementByteSizeObserver(Counter counter) { - this.counter = counter; - } + public ElementByteSizeObserver() {} /** * Called to report element byte size. */ - protected void reportElementSize(long elementByteSize) { - if (counter != null) { - counter.addValue(elementByteSize); - } - } + protected abstract void reportElementSize(long elementByteSize); /** * Sets byte counting for the current element as lazy. That is, the diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/CounterAggregatorTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/CounterAggregatorTest.java deleted file mode 100644 index 3f96cf2095eda..0000000000000 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/CounterAggregatorTest.java +++ /dev/null @@ -1,256 +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; - -import static org.apache.beam.sdk.util.common.Counter.AggregationKind.MAX; -import static org.apache.beam.sdk.util.common.Counter.AggregationKind.MIN; -import static org.apache.beam.sdk.util.common.Counter.AggregationKind.SUM; - -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 static org.mockito.Mockito.withSettings; - -import org.apache.beam.sdk.transforms.Aggregator; -import org.apache.beam.sdk.transforms.Combine; -import org.apache.beam.sdk.transforms.Combine.CombineFn; -import org.apache.beam.sdk.transforms.Combine.IterableCombineFn; -import org.apache.beam.sdk.transforms.Max; -import org.apache.beam.sdk.transforms.Min; -import org.apache.beam.sdk.transforms.SerializableFunction; -import org.apache.beam.sdk.transforms.Sum; -import org.apache.beam.sdk.util.common.Counter; -import org.apache.beam.sdk.util.common.CounterProvider; -import org.apache.beam.sdk.util.common.CounterSet; -import org.apache.beam.sdk.util.common.CounterSet.AddCounterMutator; - -import com.google.common.collect.Iterables; - -import org.hamcrest.Matchers; -import org.junit.Assert; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.ExpectedException; -import org.junit.runner.RunWith; -import org.junit.runners.JUnit4; - -import java.util.Arrays; -import java.util.List; - -/** - * Unit tests for the {@link Aggregator} API. - */ -@RunWith(JUnit4.class) -public class CounterAggregatorTest { - @Rule - public final ExpectedException expectedEx = ExpectedException.none(); - - private static final String AGGREGATOR_NAME = "aggregator_name"; - - @SuppressWarnings("rawtypes") - private void testAggregator(List items, - Combine.CombineFn combiner, - Counter expectedCounter) { - CounterSet counters = new CounterSet(); - Aggregator aggregator = new CounterAggregator<>( - AGGREGATOR_NAME, combiner, counters.getAddCounterMutator()); - for (V item : items) { - aggregator.addValue(item); - } - - assertEquals(Iterables.getOnlyElement(counters), expectedCounter); - } - - @Test - public void testGetName() { - String name = "testAgg"; - CounterAggregator aggregator = new CounterAggregator<>( - name, new Sum.SumLongFn(), - new CounterSet().getAddCounterMutator()); - - assertEquals(name, aggregator.getName()); - } - - @Test - public void testGetCombineFn() { - CombineFn combineFn = new Min.MinLongFn(); - - CounterAggregator aggregator = new CounterAggregator<>("foo", - combineFn, new CounterSet().getAddCounterMutator()); - - assertEquals(combineFn, aggregator.getCombineFn()); - } - - @Test - - public void testSumInteger() throws Exception { - testAggregator(Arrays.asList(2, 4, 1, 3), new Sum.SumIntegerFn(), - Counter.ints(AGGREGATOR_NAME, SUM).resetToValue(10)); - } - - @Test - public void testSumLong() throws Exception { - testAggregator(Arrays.asList(2L, 4L, 1L, 3L), new Sum.SumLongFn(), - Counter.longs(AGGREGATOR_NAME, SUM).resetToValue(10L)); - } - - @Test - public void testSumDouble() throws Exception { - testAggregator(Arrays.asList(2.0, 4.1, 1.0, 3.1), new Sum.SumDoubleFn(), - Counter.doubles(AGGREGATOR_NAME, SUM).resetToValue(10.2)); - } - - @Test - public void testMinInteger() throws Exception { - testAggregator(Arrays.asList(2, 4, 1, 3), new Min.MinIntegerFn(), - Counter.ints(AGGREGATOR_NAME, MIN).resetToValue(1)); - } - - @Test - public void testMinLong() throws Exception { - testAggregator(Arrays.asList(2L, 4L, 1L, 3L), new Min.MinLongFn(), - Counter.longs(AGGREGATOR_NAME, MIN).resetToValue(1L)); - } - - @Test - public void testMinDouble() throws Exception { - testAggregator(Arrays.asList(2.0, 4.1, 1.0, 3.1), new Min.MinDoubleFn(), - Counter.doubles(AGGREGATOR_NAME, MIN).resetToValue(1.0)); - } - - @Test - public void testMaxInteger() throws Exception { - testAggregator(Arrays.asList(2, 4, 1, 3), new Max.MaxIntegerFn(), - Counter.ints(AGGREGATOR_NAME, MAX).resetToValue(4)); - } - - @Test - public void testMaxLong() throws Exception { - testAggregator(Arrays.asList(2L, 4L, 1L, 3L), new Max.MaxLongFn(), - Counter.longs(AGGREGATOR_NAME, MAX).resetToValue(4L)); - } - - @Test - public void testMaxDouble() throws Exception { - testAggregator(Arrays.asList(2.0, 4.1, 1.0, 3.1), new Max.MaxDoubleFn(), - Counter.doubles(AGGREGATOR_NAME, MAX).resetToValue(4.1)); - } - - @Test - public void testCounterProviderCallsProvidedCounterAddValue() { - @SuppressWarnings("unchecked") - CombineFn combiner = mock(CombineFn.class, - withSettings().extraInterfaces(CounterProvider.class)); - @SuppressWarnings("unchecked") - CounterProvider provider = (CounterProvider) combiner; - - @SuppressWarnings("unchecked") - Counter mockCounter = mock(Counter.class); - String name = "foo"; - when(provider.getCounter(name)).thenReturn(mockCounter); - - AddCounterMutator addCounterMutator = mock(AddCounterMutator.class); - when(addCounterMutator.addCounter(mockCounter)).thenReturn(mockCounter); - - Aggregator aggregator = - new CounterAggregator<>(name, combiner, addCounterMutator); - - aggregator.addValue("bar_baz"); - - verify(mockCounter).addValue("bar_baz"); - verify(addCounterMutator).addCounter(mockCounter); - } - - - @Test - public void testCompatibleDuplicateNames() throws Exception { - CounterSet counters = new CounterSet(); - Aggregator aggregator1 = new CounterAggregator<>( - AGGREGATOR_NAME, new Sum.SumIntegerFn(), - counters.getAddCounterMutator()); - - Aggregator aggregator2 = new CounterAggregator<>( - AGGREGATOR_NAME, new Sum.SumIntegerFn(), - counters.getAddCounterMutator()); - - // The duplicate aggregators should update the same counter. - aggregator1.addValue(3); - aggregator2.addValue(4); - Assert.assertEquals( - new CounterSet(Counter.ints(AGGREGATOR_NAME, SUM).resetToValue(7)), - counters); - } - - @Test - public void testIncompatibleDuplicateNames() throws Exception { - CounterSet counters = new CounterSet(); - new CounterAggregator<>( - AGGREGATOR_NAME, new Sum.SumIntegerFn(), - counters.getAddCounterMutator()); - - expectedEx.expect(IllegalArgumentException.class); - expectedEx.expectMessage(Matchers.containsString( - "aggregator's name collides with an existing aggregator or " - + "system-provided counter of an incompatible type")); - new CounterAggregator<>( - AGGREGATOR_NAME, new Sum.SumLongFn(), - counters.getAddCounterMutator()); - } - - @Test - public void testUnsupportedCombineFn() throws Exception { - expectedEx.expect(IllegalArgumentException.class); - expectedEx.expectMessage(Matchers.containsString("unsupported combiner")); - new CounterAggregator<>( - AGGREGATOR_NAME, - new Combine.CombineFn, Integer>() { - @Override - public List createAccumulator() { - return null; - } - @Override - public List addInput(List accumulator, Integer input) { - return null; - } - @Override - public List mergeAccumulators(Iterable> accumulators) { - return null; - } - @Override - public Integer extractOutput(List accumulator) { - return null; - } - }, (new CounterSet()).getAddCounterMutator()); - } - - @Test - public void testUnsupportedSerializableFunction() throws Exception { - expectedEx.expect(IllegalArgumentException.class); - expectedEx.expectMessage(Matchers.containsString("unsupported combiner")); - CombineFn, Integer> combiner = IterableCombineFn - .of(new SerializableFunction, Integer>() { - @Override - public Integer apply(Iterable input) { - return null; - } - }); - new CounterAggregator<>(AGGREGATOR_NAME, combiner, - (new CounterSet()).getAddCounterMutator()); - } -} diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/common/CounterSetTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/common/CounterSetTest.java deleted file mode 100644 index bce0a888ff4f8..0000000000000 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/common/CounterSetTest.java +++ /dev/null @@ -1,227 +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.common; - -import static org.apache.beam.sdk.util.common.Counter.AggregationKind.MAX; -import static org.apache.beam.sdk.util.common.Counter.AggregationKind.SUM; - -import static org.hamcrest.Matchers.containsInAnyOrder; -import static org.hamcrest.Matchers.equalTo; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertSame; -import static org.junit.Assert.assertThat; -import static org.junit.Assert.assertTrue; - -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; - -/** - * Unit tests for {@link CounterSet}. - */ -@RunWith(JUnit4.class) -public class CounterSetTest { - private CounterSet set; - - @Rule - public ExpectedException thrown = ExpectedException.none(); - - @Before - public void setup() { - set = new CounterSet(); - } - - @Test - public void testAddWithDifferentNamesAddsAll() { - Counter c1 = Counter.longs("c1", SUM); - Counter c2 = Counter.ints("c2", MAX); - - boolean c1Add = set.add(c1); - boolean c2Add = set.add(c2); - - assertTrue(c1Add); - assertTrue(c2Add); - assertThat(set, containsInAnyOrder(c1, c2)); - } - - @Test - public void testAddWithAlreadyPresentNameReturnsFalse() { - Counter c1 = Counter.longs("c1", SUM); - Counter c1Dup = Counter.longs("c1", SUM); - - boolean c1Add = set.add(c1); - boolean c1DupAdd = set.add(c1Dup); - - assertTrue(c1Add); - assertFalse(c1DupAdd); - assertThat(set, containsInAnyOrder((Counter) c1)); - } - - @Test - public void testAddOrReuseWithAlreadyPresentReturnsPresent() { - Counter c1 = Counter.longs("c1", SUM); - Counter c1Dup = Counter.longs("c1", SUM); - - Counter c1AddResult = set.addOrReuseCounter(c1); - Counter c1DupAddResult = set.addOrReuseCounter(c1Dup); - - assertSame(c1, c1AddResult); - assertSame(c1AddResult, c1DupAddResult); - assertThat(set, containsInAnyOrder((Counter) c1)); - } - - @Test - public void testAddOrReuseWithNoCounterReturnsProvided() { - Counter c1 = Counter.longs("c1", SUM); - - Counter c1AddResult = set.addOrReuseCounter(c1); - - assertSame(c1, c1AddResult); - assertThat(set, containsInAnyOrder((Counter) c1)); - } - - @Test - public void testAddOrReuseWithIncompatibleTypesThrowsException() { - Counter c1 = Counter.longs("c1", SUM); - Counter c1Incompatible = Counter.ints("c1", MAX); - - set.addOrReuseCounter(c1); - - thrown.expect(IllegalArgumentException.class); - thrown.expectMessage("Counter " + c1Incompatible - + " duplicates incompatible counter " + c1 + " in " + set); - - set.addOrReuseCounter(c1Incompatible); - } - - @Test - public void testMergeWithDifferentNamesAddsAll() { - Counter c1 = Counter.longs("c1", SUM); - Counter c2 = Counter.ints("c2", MAX); - - set.add(c1); - set.add(c2); - - CounterSet newSet = new CounterSet(); - newSet.merge(set); - - assertThat(newSet, containsInAnyOrder(c1, c2)); - } - - @SuppressWarnings("unchecked") - @Test - public void testMergeWithSameNamesMerges() { - Counter c1 = Counter.longs("c1", SUM); - Counter c2 = Counter.ints("c2", MAX); - - set.add(c1); - set.add(c2); - - c1.addValue(3L); - c2.addValue(22); - - CounterSet newSet = new CounterSet(); - Counter c1Prime = Counter.longs("c1", SUM); - Counter c2Prime = Counter.ints("c2", MAX); - - c1Prime.addValue(7L); - c2Prime.addValue(14); - - newSet.add(c1Prime); - newSet.add(c2Prime); - - newSet.merge(set); - - assertThat((Counter) newSet.getExistingCounter("c1"), equalTo(c1Prime)); - assertThat((Long) newSet.getExistingCounter("c1").getAggregate(), equalTo(10L)); - - assertThat((Counter) newSet.getExistingCounter("c2"), equalTo(c2Prime)); - assertThat((Integer) newSet.getExistingCounter("c2").getAggregate(), equalTo(22)); - } - - @SuppressWarnings("unchecked") - @Test - public void testMergeWithIncompatibleTypesThrowsException() { - Counter c1 = Counter.longs("c1", SUM); - - set.add(c1); - - CounterSet newSet = new CounterSet(); - Counter c1Prime = Counter.longs("c1", MAX); - - newSet.add(c1Prime); - - thrown.expect(IllegalArgumentException.class); - thrown.expectMessage("c1"); - thrown.expectMessage("incompatible counters with the same name"); - - newSet.merge(set); - } - - @Test - public void testAddCounterMutatorAddCounterAddsCounter() { - Counter c1 = Counter.longs("c1", SUM); - - Counter addC1Result = set.getAddCounterMutator().addCounter(c1); - - assertSame(c1, addC1Result); - assertThat(set, containsInAnyOrder((Counter) c1)); - } - - @Test - public void testAddCounterMutatorAddEqualCounterReusesCounter() { - Counter c1 = Counter.longs("c1", SUM); - Counter c1dup = Counter.longs("c1", SUM); - - Counter addC1Result = set.getAddCounterMutator().addCounter(c1); - Counter addC1DupResult = set.getAddCounterMutator().addCounter(c1dup); - - assertThat(set, containsInAnyOrder((Counter) c1)); - assertSame(c1, addC1Result); - assertSame(c1, addC1DupResult); - } - - @Test - public void testAddCounterMutatorIncompatibleTypesThrowsException() { - Counter c1 = Counter.longs("c1", SUM); - Counter c1Incompatible = Counter.longs("c1", MAX); - - set.getAddCounterMutator().addCounter(c1); - - thrown.expect(IllegalArgumentException.class); - thrown.expectMessage("Counter " + c1Incompatible - + " duplicates incompatible counter " + c1 + " in " + set); - - set.getAddCounterMutator().addCounter(c1Incompatible); - } - - @Test - public void testAddCounterMutatorAddMultipleCounters() { - Counter c1 = Counter.longs("c1", SUM); - Counter c2 = Counter.longs("c2", MAX); - - set.getAddCounterMutator().addCounter(c1); - set.getAddCounterMutator().addCounter(c2); - - assertThat(set, containsInAnyOrder(c1, c2)); - } - -} diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/common/CounterTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/common/CounterTest.java deleted file mode 100644 index 79f0cb78b3bda..0000000000000 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/common/CounterTest.java +++ /dev/null @@ -1,736 +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.common; - -import static org.apache.beam.sdk.util.Values.asDouble; -import static org.apache.beam.sdk.util.common.Counter.AggregationKind.AND; -import static org.apache.beam.sdk.util.common.Counter.AggregationKind.MAX; -import static org.apache.beam.sdk.util.common.Counter.AggregationKind.MEAN; -import static org.apache.beam.sdk.util.common.Counter.AggregationKind.MIN; -import static org.apache.beam.sdk.util.common.Counter.AggregationKind.OR; -import static org.apache.beam.sdk.util.common.Counter.AggregationKind.SUM; - -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertTrue; - -import org.apache.beam.sdk.util.common.Counter.CommitState; -import org.apache.beam.sdk.util.common.Counter.CounterMean; - -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.ExpectedException; -import org.junit.runner.RunWith; -import org.junit.runners.JUnit4; - -import java.util.Arrays; -import java.util.List; - -/** - * Unit tests for the {@link Counter} API. - */ -@RunWith(JUnit4.class) -public class CounterTest { - - @Rule - public ExpectedException thrown = ExpectedException.none(); - - private static void flush(Counter c) { - switch (c.getKind()) { - case SUM: - case MAX: - case MIN: - case AND: - case OR: - c.getAndResetDelta(); - break; - case MEAN: - c.getAndResetMeanDelta(); - break; - default: - throw new IllegalArgumentException("Unknown counter kind " + c.getKind()); - } - } - - private static final double EPSILON = 0.00000000001; - - @Test - public void testCompatibility() { - // Equal counters are compatible, of all kinds. - assertTrue( - Counter.longs("c", SUM).isCompatibleWith(Counter.longs("c", SUM))); - assertTrue( - Counter.ints("c", SUM).isCompatibleWith(Counter.ints("c", SUM))); - assertTrue( - Counter.doubles("c", SUM).isCompatibleWith(Counter.doubles("c", SUM))); - assertTrue( - Counter.booleans("c", OR).isCompatibleWith( - Counter.booleans("c", OR))); - - // The name, kind, and type of the counter must match. - assertFalse( - Counter.longs("c", SUM).isCompatibleWith(Counter.longs("c2", SUM))); - assertFalse( - Counter.longs("c", SUM).isCompatibleWith(Counter.longs("c", MAX))); - assertFalse( - Counter.longs("c", SUM).isCompatibleWith(Counter.ints("c", SUM))); - - // The value of the counters are ignored. - assertTrue( - Counter.longs("c", SUM).resetToValue(666L).isCompatibleWith( - Counter.longs("c", SUM).resetToValue(42L))); - } - - - private void assertOK(long total, long delta, Counter c) { - assertEquals(total, c.getAggregate().longValue()); - assertEquals(delta, c.getAndResetDelta().longValue()); - } - - private void assertOK(double total, double delta, Counter c) { - assertEquals(total, asDouble(c.getAggregate()), EPSILON); - assertEquals(delta, asDouble(c.getAndResetDelta()), EPSILON); - } - - - // Tests for SUM. - - @Test - public void testSumLong() { - Counter c = Counter.longs("sum-long", SUM); - long expectedTotal = 0; - long expectedDelta = 0; - assertOK(expectedTotal, expectedDelta, c); - - c.addValue(13L).addValue(42L).addValue(0L); - expectedTotal += 55; - expectedDelta += 55; - assertOK(expectedTotal, expectedDelta, c); - - c.resetToValue(120L).addValue(17L).addValue(37L); - expectedTotal = expectedDelta = 174; - assertOK(expectedTotal, expectedDelta, c); - - flush(c); - expectedDelta = 0; - assertOK(expectedTotal, expectedDelta, c); - - c.addValue(15L).addValue(42L); - expectedTotal += 57; - expectedDelta += 57; - assertOK(expectedTotal, expectedDelta, c); - - c.resetToValue(100L).addValue(17L).addValue(49L); - expectedTotal = expectedDelta = 166; - assertOK(expectedTotal, expectedDelta, c); - - Counter other = Counter.longs("sum-long", SUM); - other.addValue(12L); - expectedDelta = 12L; - expectedTotal += 12L; - c.merge(other); - assertOK(expectedTotal, expectedDelta, c); - } - - @Test - public void testSumDouble() { - Counter c = Counter.doubles("sum-double", SUM); - double expectedTotal = 0.0; - double expectedDelta = 0.0; - assertOK(expectedTotal, expectedDelta, c); - - c.addValue(Math.E).addValue(Math.PI).addValue(0.0); - expectedTotal += Math.E + Math.PI; - expectedDelta += Math.E + Math.PI; - assertOK(expectedTotal, expectedDelta, c); - - c.resetToValue(Math.sqrt(2)).addValue(2 * Math.PI).addValue(3 * Math.E); - expectedTotal = expectedDelta = Math.sqrt(2) + 2 * Math.PI + 3 * Math.E; - assertOK(expectedTotal, expectedDelta, c); - - flush(c); - expectedDelta = 0.0; - assertOK(expectedTotal, expectedDelta, c); - - c.addValue(7 * Math.PI).addValue(5 * Math.E); - expectedTotal += 7 * Math.PI + 5 * Math.E; - expectedDelta += 7 * Math.PI + 5 * Math.E; - assertOK(expectedTotal, expectedDelta, c); - - c.resetToValue(Math.sqrt(17)).addValue(17.0).addValue(49.0); - expectedTotal = expectedDelta = Math.sqrt(17.0) + 17.0 + 49.0; - assertOK(expectedTotal, expectedDelta, c); - - Counter other = Counter.doubles("sum-double", SUM); - other.addValue(12 * Math.PI); - expectedDelta = 12 * Math.PI; - expectedTotal += 12 * Math.PI; - c.merge(other); - assertOK(expectedTotal, expectedDelta, c); - } - - - // Tests for MAX. - - @Test - public void testMaxLong() { - Counter c = Counter.longs("max-long", MAX); - long expectedTotal = Long.MIN_VALUE; - long expectedDelta = Long.MIN_VALUE; - assertOK(expectedTotal, expectedDelta, c); - - c.addValue(13L).addValue(42L).addValue(Long.MIN_VALUE); - expectedTotal = expectedDelta = 42; - assertOK(expectedTotal, expectedDelta, c); - - c.resetToValue(120L).addValue(17L).addValue(37L); - expectedTotal = expectedDelta = 120; - assertOK(expectedTotal, expectedDelta, c); - - flush(c); - expectedDelta = Long.MIN_VALUE; - assertOK(expectedTotal, expectedDelta, c); - - c.addValue(42L).addValue(15L); - expectedDelta = 42; - assertOK(expectedTotal, expectedDelta, c); - - c.resetToValue(100L).addValue(171L).addValue(49L); - expectedTotal = expectedDelta = 171; - assertOK(expectedTotal, expectedDelta, c); - - Counter other = Counter.longs("max-long", MAX); - other.addValue(12L); - expectedDelta = 12L; - c.merge(other); - assertOK(expectedTotal, expectedDelta, c); - } - - @Test - public void testMaxDouble() { - Counter c = Counter.doubles("max-double", MAX); - double expectedTotal = Double.NEGATIVE_INFINITY; - double expectedDelta = Double.NEGATIVE_INFINITY; - assertOK(expectedTotal, expectedDelta, c); - - c.addValue(Math.E).addValue(Math.PI).addValue(Double.NEGATIVE_INFINITY); - expectedTotal = expectedDelta = Math.PI; - assertOK(expectedTotal, expectedDelta, c); - - c.resetToValue(Math.sqrt(12345)).addValue(2 * Math.PI).addValue(3 * Math.E); - expectedTotal = expectedDelta = Math.sqrt(12345); - assertOK(expectedTotal, expectedDelta, c); - - flush(c); - expectedDelta = Double.NEGATIVE_INFINITY; - assertOK(expectedTotal, expectedDelta, c); - - c.addValue(7 * Math.PI).addValue(5 * Math.E); - expectedDelta = 7 * Math.PI; - assertOK(expectedTotal, expectedDelta, c); - - c.resetToValue(Math.sqrt(17)).addValue(171.0).addValue(49.0); - expectedTotal = expectedDelta = 171.0; - assertOK(expectedTotal, expectedDelta, c); - - Counter other = Counter.doubles("max-double", MAX); - other.addValue(12 * Math.PI); - expectedDelta = 12 * Math.PI; - c.merge(other); - assertOK(expectedTotal, expectedDelta, c); - } - - - // Tests for MIN. - - @Test - public void testMinLong() { - Counter c = Counter.longs("min-long", MIN); - long expectedTotal = Long.MAX_VALUE; - long expectedDelta = Long.MAX_VALUE; - assertOK(expectedTotal, expectedDelta, c); - - c.addValue(13L).addValue(42L).addValue(Long.MAX_VALUE); - expectedTotal = expectedDelta = 13; - assertOK(expectedTotal, expectedDelta, c); - - c.resetToValue(120L).addValue(17L).addValue(37L); - expectedTotal = expectedDelta = 17; - assertOK(expectedTotal, expectedDelta, c); - - flush(c); - expectedDelta = Long.MAX_VALUE; - assertOK(expectedTotal, expectedDelta, c); - - c.addValue(42L).addValue(18L); - expectedDelta = 18; - assertOK(expectedTotal, expectedDelta, c); - - c.resetToValue(100L).addValue(171L).addValue(49L); - expectedTotal = expectedDelta = 49; - assertOK(expectedTotal, expectedDelta, c); - - Counter other = Counter.longs("min-long", MIN); - other.addValue(42L); - expectedTotal = expectedDelta = 42L; - c.merge(other); - assertOK(expectedTotal, expectedDelta, c); - } - - @Test - public void testMinDouble() { - Counter c = Counter.doubles("min-double", MIN); - double expectedTotal = Double.POSITIVE_INFINITY; - double expectedDelta = Double.POSITIVE_INFINITY; - assertOK(expectedTotal, expectedDelta, c); - - c.addValue(Math.E).addValue(Math.PI).addValue(Double.POSITIVE_INFINITY); - expectedTotal = expectedDelta = Math.E; - assertOK(expectedTotal, expectedDelta, c); - - c.resetToValue(Math.sqrt(12345)).addValue(2 * Math.PI).addValue(3 * Math.E); - expectedTotal = expectedDelta = 2 * Math.PI; - assertOK(expectedTotal, expectedDelta, c); - - flush(c); - expectedDelta = Double.POSITIVE_INFINITY; - assertOK(expectedTotal, expectedDelta, c); - - c.addValue(7 * Math.PI).addValue(5 * Math.E); - expectedDelta = 5 * Math.E; - assertOK(expectedTotal, expectedDelta, c); - - c.resetToValue(Math.sqrt(17)).addValue(171.0).addValue(0.0); - expectedTotal = expectedDelta = 0.0; - assertOK(expectedTotal, expectedDelta, c); - - Counter other = Counter.doubles("min-double", MIN); - other.addValue(42 * Math.E); - expectedDelta = 42 * Math.E; - c.merge(other); - assertOK(expectedTotal, expectedDelta, c); - } - - - // Tests for MEAN. - - private void assertMean(long s, long sd, long c, long cd, Counter cn) { - CounterMean mean = cn.getMean(); - CounterMean deltaMean = cn.getAndResetMeanDelta(); - assertEquals(s, mean.getAggregate().longValue()); - assertEquals(sd, deltaMean.getAggregate().longValue()); - assertEquals(c, mean.getCount()); - assertEquals(cd, deltaMean.getCount()); - } - - private void assertMean(double s, double sd, long c, long cd, - Counter cn) { - CounterMean mean = cn.getMean(); - CounterMean deltaMean = cn.getAndResetMeanDelta(); - assertEquals(s, mean.getAggregate().doubleValue(), EPSILON); - assertEquals(sd, deltaMean.getAggregate().doubleValue(), EPSILON); - assertEquals(c, mean.getCount()); - assertEquals(cd, deltaMean.getCount()); - } - - @Test - public void testMeanLong() { - Counter c = Counter.longs("mean-long", MEAN); - long expTotal = 0; - long expDelta = 0; - long expCountTotal = 0; - long expCountDelta = 0; - assertMean(expTotal, expDelta, expCountTotal, expCountDelta, c); - - c.addValue(13L).addValue(42L).addValue(0L); - expTotal += 55; - expDelta += 55; - expCountTotal += 3; - expCountDelta += 3; - assertMean(expTotal, expDelta, expCountTotal, expCountDelta, c); - - c.resetMeanToValue(1L, 120L).addValue(17L).addValue(37L); - expTotal = expDelta = 174; - assertMean(expTotal, expDelta, expCountTotal, expCountDelta, c); - - flush(c); - expDelta = 0; - expCountDelta = 0; - assertMean(expTotal, expDelta, expCountTotal, expCountDelta, c); - - c.addValue(15L).addValue(42L); - expTotal += 57; - expDelta += 57; - expCountTotal += 2; - expCountDelta += 2; - assertMean(expTotal, expDelta, expCountTotal, expCountDelta, c); - - c.resetMeanToValue(3L, 100L).addValue(17L).addValue(49L); - expTotal = expDelta = 166; - expCountTotal = expCountDelta = 5; - assertMean(expTotal, expDelta, expCountTotal, expCountDelta, c); - - Counter other = Counter.longs("mean-long", MEAN); - other.addValue(12L).addValue(44L).addValue(-5L); - expTotal += 12L + 44L - 5L; - expDelta += 12L + 44L - 5L; - expCountTotal += 3; - expCountDelta += 3; - c.merge(other); - assertMean(expTotal, expDelta, expCountTotal, expCountDelta, c); - } - - @Test - public void testMeanDouble() { - Counter c = Counter.doubles("mean-double", MEAN); - double expTotal = 0.0; - double expDelta = 0.0; - long expCountTotal = 0; - long expCountDelta = 0; - assertMean(expTotal, expDelta, expCountTotal, expCountDelta, c); - - c.addValue(Math.E).addValue(Math.PI).addValue(0.0); - expTotal += Math.E + Math.PI; - expDelta += Math.E + Math.PI; - expCountTotal += 3; - expCountDelta += 3; - assertMean(expTotal, expDelta, expCountTotal, expCountDelta, c); - - c.resetMeanToValue(1L, Math.sqrt(2)).addValue(2 * Math.PI) - .addValue(3 * Math.E); - expTotal = expDelta = Math.sqrt(2) + 2 * Math.PI + 3 * Math.E; - assertMean(expTotal, expDelta, expCountTotal, expCountDelta, c); - - flush(c); - expDelta = 0.0; - expCountDelta = 0; - assertMean(expTotal, expDelta, expCountTotal, expCountDelta, c); - - c.addValue(7 * Math.PI).addValue(5 * Math.E); - expTotal += 7 * Math.PI + 5 * Math.E; - expDelta += 7 * Math.PI + 5 * Math.E; - expCountTotal += 2; - expCountDelta += 2; - assertMean(expTotal, expDelta, expCountTotal, expCountDelta, c); - - c.resetMeanToValue(3L, Math.sqrt(17)).addValue(17.0).addValue(49.0); - expTotal = expDelta = Math.sqrt(17.0) + 17.0 + 49.0; - expCountTotal = expCountDelta = 5; - assertMean(expTotal, expDelta, expCountTotal, expCountDelta, c); - - Counter other = Counter.doubles("mean-double", MEAN); - other.addValue(3 * Math.PI).addValue(12 * Math.E); - expTotal += 3 * Math.PI + 12 * Math.E; - expDelta += 3 * Math.PI + 12 * Math.E; - expCountTotal += 2; - expCountDelta += 2; - c.merge(other); - assertMean(expTotal, expDelta, expCountTotal, expCountDelta, c); - } - - - // Test for AND and OR. - private void assertBool(boolean total, boolean delta, Counter c) { - assertEquals(total, c.getAggregate().booleanValue()); - assertEquals(delta, c.getAndResetDelta().booleanValue()); - } - - @Test - public void testBoolAnd() { - Counter c = Counter.booleans("bool-and", AND); - boolean expectedTotal = true; - boolean expectedDelta = true; - assertBool(expectedTotal, expectedDelta, c); - - c.addValue(true); - assertBool(expectedTotal, expectedDelta, c); - - c.addValue(false); - expectedTotal = expectedDelta = false; - assertBool(expectedTotal, expectedDelta, c); - - c.resetToValue(true).addValue(true); - expectedTotal = expectedDelta = true; - assertBool(expectedTotal, expectedDelta, c); - - c.addValue(false); - expectedTotal = expectedDelta = false; - assertBool(expectedTotal, expectedDelta, c); - - flush(c); - expectedDelta = true; - assertBool(expectedTotal, expectedDelta, c); - - c.addValue(false); - expectedDelta = false; - assertBool(expectedTotal, expectedDelta, c); - } - - @Test - public void testBoolOr() { - Counter c = Counter.booleans("bool-or", OR); - boolean expectedTotal = false; - boolean expectedDelta = false; - assertBool(expectedTotal, expectedDelta, c); - - c.addValue(false); - assertBool(expectedTotal, expectedDelta, c); - - c.addValue(true); - expectedTotal = expectedDelta = true; - assertBool(expectedTotal, expectedDelta, c); - - c.resetToValue(false).addValue(false); - expectedTotal = expectedDelta = false; - assertBool(expectedTotal, expectedDelta, c); - - c.addValue(true); - expectedTotal = expectedDelta = true; - assertBool(expectedTotal, expectedDelta, c); - - flush(c); - expectedDelta = false; - assertBool(expectedTotal, expectedDelta, c); - - c.addValue(true); - expectedDelta = true; - assertBool(expectedTotal, expectedDelta, c); - } - - // Incompatibility tests. - - @Test(expected = IllegalArgumentException.class) - public void testSumBool() { - Counter.booleans("counter", SUM); - } - - @Test(expected = IllegalArgumentException.class) - public void testMinBool() { - Counter.booleans("counter", MIN); - } - - @Test(expected = IllegalArgumentException.class) - public void testMaxBool() { - Counter.booleans("counter", MAX); - } - - @Test(expected = IllegalArgumentException.class) - public void testMeanBool() { - Counter.booleans("counter", MEAN); - } - - @Test(expected = IllegalArgumentException.class) - public void testAndLong() { - Counter.longs("counter", AND); - } - - @Test(expected = IllegalArgumentException.class) - public void testAndDouble() { - Counter.doubles("counter", AND); - } - - @Test(expected = IllegalArgumentException.class) - public void testOrLong() { - Counter.longs("counter", OR); - } - - @Test(expected = IllegalArgumentException.class) - public void testOrDouble() { - Counter.doubles("counter", OR); - } - - @Test - public void testMergeIncompatibleCounters() { - Counter longSums = Counter.longs("longsums", SUM); - Counter longMean = Counter.longs("longmean", MEAN); - Counter longMin = Counter.longs("longmin", MIN); - - Counter otherLongSums = Counter.longs("othersums", SUM); - Counter otherLongMean = Counter.longs("otherlongmean", MEAN); - - Counter doubleSums = Counter.doubles("doublesums", SUM); - Counter doubleMean = Counter.doubles("doublemean", MEAN); - - Counter boolAnd = Counter.booleans("and", AND); - Counter boolOr = Counter.booleans("or", OR); - - List> longCounters = - Arrays.asList(longSums, longMean, longMin, otherLongSums, otherLongMean); - for (Counter left : longCounters) { - for (Counter right : longCounters) { - if (left != right) { - assertIncompatibleMerge(left, right); - } - } - } - - assertIncompatibleMerge(doubleSums, doubleMean); - assertIncompatibleMerge(boolAnd, boolOr); - } - - private void assertIncompatibleMerge(Counter left, Counter right) { - thrown.expect(IllegalArgumentException.class); - thrown.expectMessage("Counters"); - thrown.expectMessage("are incompatible"); - left.merge(right); - } - - @Test - public void testDirtyBit() { - Counter longSum = Counter.longs("long-sum", SUM); - Counter longMean = Counter.longs("long-mean", MEAN); - Counter doubleSum = Counter.doubles("double-sum", SUM); - Counter doubleMean = Counter.doubles("double-sum", MEAN); - Counter intSum = Counter.ints("int-sum", SUM); - Counter intMean = Counter.ints("int-sum", MEAN); - Counter boolAnd = Counter.booleans("and", AND); - - // Test counters are not dirty and are COMMITTED initially. - assertFalse(longSum.isDirty()); - assertFalse(longMean.isDirty()); - assertFalse(doubleSum.isDirty()); - assertFalse(doubleMean.isDirty()); - assertFalse(intSum.isDirty()); - assertFalse(intMean.isDirty()); - assertFalse(boolAnd.isDirty()); - - assertEquals(CommitState.COMMITTED, longSum.commitState.get()); - assertEquals(CommitState.COMMITTED, longMean.commitState.get()); - assertEquals(CommitState.COMMITTED, doubleSum.commitState.get()); - assertEquals(CommitState.COMMITTED, doubleMean.commitState.get()); - assertEquals(CommitState.COMMITTED, intSum.commitState.get()); - assertEquals(CommitState.COMMITTED, intMean.commitState.get()); - assertEquals(CommitState.COMMITTED, boolAnd.commitState.get()); - - // Test counters are dirty after mutating. - longSum.addValue(1L); - longMean.resetMeanToValue(1L, 1L); - doubleSum.addValue(1.0); - doubleMean.resetMeanToValue(1L, 1.0); - intSum.addValue(1); - intMean.resetMeanToValue(1, 1); - boolAnd.addValue(true); - - assertTrue(longSum.isDirty()); - assertTrue(longMean.isDirty()); - assertTrue(doubleSum.isDirty()); - assertTrue(doubleMean.isDirty()); - assertTrue(intSum.isDirty()); - assertTrue(intMean.isDirty()); - assertTrue(boolAnd.isDirty()); - - assertEquals(CommitState.DIRTY, longSum.commitState.get()); - assertEquals(CommitState.DIRTY, longMean.commitState.get()); - assertEquals(CommitState.DIRTY, doubleSum.commitState.get()); - assertEquals(CommitState.DIRTY, doubleMean.commitState.get()); - assertEquals(CommitState.DIRTY, intSum.commitState.get()); - assertEquals(CommitState.DIRTY, intMean.commitState.get()); - assertEquals(CommitState.DIRTY, boolAnd.commitState.get()); - - // Test counters are dirty and are COMMITTING. - assertTrue(longSum.committing()); - assertTrue(longMean.committing()); - assertTrue(doubleSum.committing()); - assertTrue(doubleMean.committing()); - assertTrue(intSum.committing()); - assertTrue(intMean.committing()); - assertTrue(boolAnd.committing()); - - assertTrue(longSum.isDirty()); - assertTrue(longMean.isDirty()); - assertTrue(doubleSum.isDirty()); - assertTrue(doubleMean.isDirty()); - assertTrue(intSum.isDirty()); - assertTrue(intMean.isDirty()); - assertTrue(boolAnd.isDirty()); - - assertEquals(CommitState.COMMITTING, longSum.commitState.get()); - assertEquals(CommitState.COMMITTING, longMean.commitState.get()); - assertEquals(CommitState.COMMITTING, doubleSum.commitState.get()); - assertEquals(CommitState.COMMITTING, doubleMean.commitState.get()); - assertEquals(CommitState.COMMITTING, intSum.commitState.get()); - assertEquals(CommitState.COMMITTING, intMean.commitState.get()); - assertEquals(CommitState.COMMITTING, boolAnd.commitState.get()); - - // Test counters are dirty again after mutating. - longSum.addValue(1L); - longMean.resetMeanToValue(1L, 1L); - doubleSum.addValue(1.0); - doubleMean.resetMeanToValue(1L, 1.0); - intSum.addValue(1); - intMean.resetMeanToValue(1, 1); - boolAnd.addValue(true); - - assertFalse(longSum.committed()); - assertFalse(longMean.committed()); - assertFalse(doubleSum.committed()); - assertFalse(doubleMean.committed()); - assertFalse(intSum.committed()); - assertFalse(intMean.committed()); - assertFalse(boolAnd.committed()); - - assertTrue(longSum.isDirty()); - assertTrue(longMean.isDirty()); - assertTrue(doubleSum.isDirty()); - assertTrue(doubleMean.isDirty()); - assertTrue(intSum.isDirty()); - assertTrue(intMean.isDirty()); - assertTrue(boolAnd.isDirty()); - - assertEquals(CommitState.DIRTY, longSum.commitState.get()); - assertEquals(CommitState.DIRTY, longMean.commitState.get()); - assertEquals(CommitState.DIRTY, doubleSum.commitState.get()); - assertEquals(CommitState.DIRTY, doubleMean.commitState.get()); - assertEquals(CommitState.DIRTY, intSum.commitState.get()); - assertEquals(CommitState.DIRTY, intMean.commitState.get()); - assertEquals(CommitState.DIRTY, boolAnd.commitState.get()); - - // Test counters are not dirty and are COMMITTED. - assertTrue(longSum.committing()); - assertTrue(longMean.committing()); - assertTrue(doubleSum.committing()); - assertTrue(doubleMean.committing()); - assertTrue(intSum.committing()); - assertTrue(intMean.committing()); - assertTrue(boolAnd.committing()); - - assertTrue(longSum.committed()); - assertTrue(longMean.committed()); - assertTrue(doubleSum.committed()); - assertTrue(doubleMean.committed()); - assertTrue(intSum.committed()); - assertTrue(intMean.committed()); - assertTrue(boolAnd.committed()); - - assertFalse(longSum.isDirty()); - assertFalse(longMean.isDirty()); - assertFalse(doubleSum.isDirty()); - assertFalse(doubleMean.isDirty()); - assertFalse(intSum.isDirty()); - assertFalse(intMean.isDirty()); - assertFalse(boolAnd.isDirty()); - - assertEquals(CommitState.COMMITTED, longSum.commitState.get()); - assertEquals(CommitState.COMMITTED, longMean.commitState.get()); - assertEquals(CommitState.COMMITTED, doubleSum.commitState.get()); - assertEquals(CommitState.COMMITTED, doubleMean.commitState.get()); - assertEquals(CommitState.COMMITTED, intSum.commitState.get()); - assertEquals(CommitState.COMMITTED, intMean.commitState.get()); - assertEquals(CommitState.COMMITTED, boolAnd.commitState.get()); - } -} From 7fc2c6848f002ac8b2ccbe35e6b5a576777a7af9 Mon Sep 17 00:00:00 2001 From: Mark Liu Date: Tue, 2 Aug 2016 17:25:14 -0700 Subject: [PATCH 014/346] [BEAM-495] Create General Verifier for File Checksum --- .../org/apache/beam/examples/WordCountIT.java | 112 +------------- .../beam/sdk/testing/FileChecksumMatcher.java | 137 ++++++++++++++++++ .../sdk/testing/FileChecksumMatcherTest.java | 78 ++++++++++ 3 files changed, 219 insertions(+), 108 deletions(-) create mode 100644 sdks/java/core/src/main/java/org/apache/beam/sdk/testing/FileChecksumMatcher.java create mode 100644 sdks/java/core/src/test/java/org/apache/beam/sdk/testing/FileChecksumMatcherTest.java diff --git a/examples/java/src/test/java/org/apache/beam/examples/WordCountIT.java b/examples/java/src/test/java/org/apache/beam/examples/WordCountIT.java index c087c67d8391a..de6615c95ea59 100644 --- a/examples/java/src/test/java/org/apache/beam/examples/WordCountIT.java +++ b/examples/java/src/test/java/org/apache/beam/examples/WordCountIT.java @@ -18,38 +18,18 @@ package org.apache.beam.examples; -import static com.google.common.base.Preconditions.checkArgument; - import org.apache.beam.examples.WordCount.WordCountOptions; -import org.apache.beam.sdk.PipelineResult; import org.apache.beam.sdk.options.PipelineOptionsFactory; -import org.apache.beam.sdk.testing.SerializableMatcher; +import org.apache.beam.sdk.testing.FileChecksumMatcher; import org.apache.beam.sdk.testing.TestPipeline; import org.apache.beam.sdk.testing.TestPipelineOptions; -import org.apache.beam.sdk.util.IOChannelFactory; import org.apache.beam.sdk.util.IOChannelUtils; -import com.google.common.base.Strings; -import com.google.common.hash.HashCode; -import com.google.common.hash.Hashing; -import com.google.common.io.CharStreams; - -import org.hamcrest.Description; -import org.hamcrest.TypeSafeMatcher; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; -import java.io.IOException; -import java.io.Reader; -import java.nio.channels.Channels; -import java.nio.charset.StandardCharsets; -import java.util.ArrayList; -import java.util.Collection; import java.util.Date; -import java.util.List; /** * End-to-end tests of WordCount. @@ -73,94 +53,10 @@ public void testE2EWordCount() throws Exception { String.format("WordCountIT-%tF-% - implements SerializableMatcher { - - private static final Logger LOG = LoggerFactory.getLogger(WordCountOnSuccessMatcher.class); - - private static final String EXPECTED_CHECKSUM = "c04722202dee29c442b55ead54c6000693e85e77"; - private String actualChecksum; - - private final String outputPath; - - WordCountOnSuccessMatcher(String outputPath) { - checkArgument( - !Strings.isNullOrEmpty(outputPath), - "Expected valid output path, but received %s", outputPath); - - this.outputPath = outputPath; - } - - @Override - protected boolean matchesSafely(PipelineResult pResult) { - try { - // Load output data - List outputs = readLines(outputPath); - - // Verify outputs. Checksum is computed using SHA-1 algorithm - actualChecksum = hashing(outputs); - LOG.info("Generated checksum for output data: {}", actualChecksum); - - return actualChecksum.equals(EXPECTED_CHECKSUM); - } catch (IOException e) { - throw new RuntimeException( - String.format("Failed to read from path: %s", outputPath)); - } - } - - 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); - - // Read data from file paths - int i = 0; - for (String file : files) { - try (Reader reader = - 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); - } - i++; - } - return readData; - } - - private String hashing(List strs) { - List hashCodes = new ArrayList<>(); - for (String str : strs) { - hashCodes.add(Hashing.sha1().hashString(str, StandardCharsets.UTF_8)); - } - return Hashing.combineUnordered(hashCodes).toString(); - } - - @Override - public void describeTo(Description description) { - description - .appendText("Expected checksum is (") - .appendText(EXPECTED_CHECKSUM) - .appendText(")"); - } - - @Override - protected void describeMismatchSafely(PipelineResult pResult, Description description) { - description - .appendText("was (") - .appendText(actualChecksum) - .appendText(")"); - } - } } 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 new file mode 100644 index 0000000000000..f4bd0626cbbbc --- /dev/null +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/FileChecksumMatcher.java @@ -0,0 +1,137 @@ +/* + * 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 org.apache.beam.sdk.PipelineResult; +import org.apache.beam.sdk.util.IOChannelFactory; +import org.apache.beam.sdk.util.IOChannelUtils; + +import com.google.common.base.Strings; +import com.google.common.hash.HashCode; +import com.google.common.hash.Hashing; +import com.google.common.io.CharStreams; + +import org.hamcrest.Description; +import org.hamcrest.TypeSafeMatcher; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.io.Reader; +import java.nio.channels.Channels; +import java.nio.charset.StandardCharsets; +import java.util.ArrayList; +import java.util.Collection; +import java.util.List; + +/** + * Matcher to verify file checksum in E2E test. + * + *

For example: + *

{@code [
+ *   assertTrue(job, new FileChecksumMatcher(checksumString, filePath));
+ * ]}
+ */ +public class FileChecksumMatcher extends TypeSafeMatcher + implements SerializableMatcher { + + private static final Logger LOG = LoggerFactory.getLogger(FileChecksumMatcher.class); + + private final String expectedChecksum; + private final String filePath; + private String actualChecksum; + + public FileChecksumMatcher(String checksum, String filePath) { + checkArgument( + !Strings.isNullOrEmpty(checksum), + "Expected valid checksum, but received %s", checksum); + checkArgument( + !Strings.isNullOrEmpty(filePath), + "Expected valid file path, but received %s", filePath); + + this.expectedChecksum = checksum; + this.filePath = filePath; + } + + @Override + public boolean matchesSafely(PipelineResult pipelineResult) { + try { + // Load output data + List outputs = readLines(filePath); + + // Verify outputs. Checksum is computed using SHA-1 algorithm + actualChecksum = hashing(outputs); + LOG.info("Generated checksum for output data: {}", actualChecksum); + + return actualChecksum.equals(expectedChecksum); + } catch (IOException e) { + throw new RuntimeException( + String.format("Failed to read from path: %s", filePath)); + } + } + + 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); + + // Read data from file paths + int i = 0; + for (String file : files) { + try (Reader reader = + 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); + } + i++; + } + return readData; + } + + private String hashing(List strs) { + List hashCodes = new ArrayList<>(); + for (String str : strs) { + hashCodes.add(Hashing.sha1().hashString(str, StandardCharsets.UTF_8)); + } + return Hashing.combineUnordered(hashCodes).toString(); + } + + @Override + public void describeTo(Description description) { + description + .appendText("Expected checksum is (") + .appendText(expectedChecksum) + .appendText(")"); + } + + @Override + public void describeMismatchSafely(PipelineResult pResult, Description description) { + description + .appendText("was (") + .appendText(actualChecksum) + .appendText(")"); + } +} 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 new file mode 100644 index 0000000000000..00417c746b0a5 --- /dev/null +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/FileChecksumMatcherTest.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.sdk.testing; + +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.containsString; + +import org.apache.beam.sdk.PipelineResult; + +import com.google.common.io.Files; + +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; + +import java.io.File; +import java.io.IOException; +import java.nio.charset.StandardCharsets; + +/** Tests for {@link FileChecksumMatcher}. */ +@RunWith(JUnit4.class) +public class FileChecksumMatcherTest { + @Rule + public TemporaryFolder tmpFolder = new TemporaryFolder(); + @Rule + public ExpectedException thrown = ExpectedException.none(); + + @Mock + private PipelineResult pResult = Mockito.mock(PipelineResult.class); + + @Test + public void testPreconditionValidChecksumString() throws IOException{ + String tmpPath = tmpFolder.newFile().getPath(); + + thrown.expect(IllegalArgumentException.class); + thrown.expectMessage(containsString("Expected valid checksum, but received")); + new FileChecksumMatcher(null, tmpPath); + new FileChecksumMatcher("", tmpPath); + } + + @Test + public void testPreconditionValidFilePath() throws IOException { + thrown.expect(IllegalArgumentException.class); + thrown.expectMessage(containsString("Expected valid file path, but received")); + new FileChecksumMatcher("checksumString", null); + new FileChecksumMatcher("checksumString", ""); + } + + @Test + public void testChecksumVerify() throws IOException{ + File tmpFile = tmpFolder.newFile(); + Files.write("Test for file checksum verifier.", tmpFile, StandardCharsets.UTF_8); + FileChecksumMatcher matcher = + new FileChecksumMatcher("a8772322f5d7b851777f820fc79d050f9d302915", tmpFile.getPath()); + + assertThat(pResult, matcher); + } + } From b47549e4893a6d487c00ea0ba02619168a3f19f3 Mon Sep 17 00:00:00 2001 From: Mark Liu Date: Tue, 2 Aug 2016 17:47:46 -0700 Subject: [PATCH 015/346] Add output checksum to WordCountITOptions --- .../test/java/org/apache/beam/examples/WordCountIT.java | 9 +++++++-- 1 file changed, 7 insertions(+), 2 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 de6615c95ea59..e438faf4f6069 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 @@ -19,6 +19,7 @@ package org.apache.beam.examples; import org.apache.beam.examples.WordCount.WordCountOptions; +import org.apache.beam.sdk.options.Default; import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.testing.FileChecksumMatcher; import org.apache.beam.sdk.testing.TestPipeline; @@ -39,8 +40,13 @@ public class WordCountIT { /** * Options for the WordCount Integration Test. + * + * Define expected output file checksum to verify WordCount pipeline result with customized input. */ public interface WordCountITOptions extends TestPipelineOptions, WordCountOptions { + @Default.String("c04722202dee29c442b55ead54c6000693e85e77") + String getOutputChecksum(); + void setOutputChecksum(String value); } @Test @@ -54,8 +60,7 @@ public void testE2EWordCount() throws Exception { "output", "results")); options.setOnSuccessMatcher( - new FileChecksumMatcher("c04722202dee29c442b55ead54c6000693e85e77", - options.getOutput() + "*")); + new FileChecksumMatcher(options.getOutputChecksum(), options.getOutput() + "*")); WordCount.main(TestPipeline.convertToArgs(options)); } From 37ce2a3e75cb96a4b3fdcd4938fb7fda95122724 Mon Sep 17 00:00:00 2001 From: Mark Liu Date: Thu, 11 Aug 2016 11:26:28 -0700 Subject: [PATCH 016/346] More unit test and code style fix --- .../beam/sdk/testing/FileChecksumMatcher.java | 17 +++++----- .../sdk/testing/FileChecksumMatcherTest.java | 34 +++++++++++++++++-- 2 files changed, 39 insertions(+), 12 deletions(-) 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 f4bd0626cbbbc..303efcb392b8a 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 @@ -84,13 +84,12 @@ public boolean matchesSafely(PipelineResult pipelineResult) { return actualChecksum.equals(expectedChecksum); } catch (IOException e) { throw new RuntimeException( - String.format("Failed to read from path: %s", filePath)); + String.format("Failed to read from path: %s", filePath)); } } private List readLines(String path) throws IOException { List readData = new ArrayList<>(); - IOChannelFactory factory = IOChannelUtils.getFactory(path); // Match inputPath which may contains glob @@ -100,7 +99,7 @@ private List readLines(String path) throws IOException { int i = 0; 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( @@ -122,16 +121,16 @@ private String hashing(List strs) { @Override public void describeTo(Description description) { description - .appendText("Expected checksum is (") - .appendText(expectedChecksum) - .appendText(")"); + .appendText("Expected checksum is (") + .appendText(expectedChecksum) + .appendText(")"); } @Override public void describeMismatchSafely(PipelineResult pResult, Description description) { description - .appendText("was (") - .appendText(actualChecksum) - .appendText(")"); + .appendText("was (") + .appendText(actualChecksum) + .appendText(")"); } } 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 00417c746b0a5..eebb73e33212d 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 @@ -49,25 +49,39 @@ public class FileChecksumMatcherTest { private PipelineResult pResult = Mockito.mock(PipelineResult.class); @Test - public void testPreconditionValidChecksumString() throws IOException{ + public void testPreconditionChecksumIsNull() throws IOException { String tmpPath = tmpFolder.newFile().getPath(); thrown.expect(IllegalArgumentException.class); thrown.expectMessage(containsString("Expected valid checksum, but received")); new FileChecksumMatcher(null, tmpPath); + } + + @Test + public void testPreconditionChecksumIsEmpty() throws IOException { + String tmpPath = tmpFolder.newFile().getPath(); + + thrown.expect(IllegalArgumentException.class); + thrown.expectMessage(containsString("Expected valid checksum, but received")); new FileChecksumMatcher("", tmpPath); } @Test - public void testPreconditionValidFilePath() throws IOException { + 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); + thrown.expectMessage(containsString("Expected valid file path, but received")); new FileChecksumMatcher("checksumString", ""); } @Test - public void testChecksumVerify() throws IOException{ + public void testMatcherVerifySingleFile() throws IOException{ File tmpFile = tmpFolder.newFile(); Files.write("Test for file checksum verifier.", tmpFile, StandardCharsets.UTF_8); FileChecksumMatcher matcher = @@ -75,4 +89,18 @@ public void testChecksumVerify() throws IOException{ assertThat(pResult, matcher); } + + @Test + public void testMatcherVerifyMultipleFilesInOneDir() throws IOException { + File tmpFile1 = tmpFolder.newFile(); + File tmpFile2 = tmpFolder.newFile(); + Files.write("To be or not to be, ", tmpFile1, StandardCharsets.UTF_8); + Files.write("it is not a question.", tmpFile2, StandardCharsets.UTF_8); + FileChecksumMatcher matcher = + new FileChecksumMatcher( + "90552392c28396935fe4f123bd0b5c2d0f6260c8", + tmpFolder.getRoot().getPath() + "/*"); + + assertThat(pResult, matcher); + } } From 046e36eaa41659ae43866bc6fbce4f122889f286 Mon Sep 17 00:00:00 2001 From: Mark Liu Date: Thu, 11 Aug 2016 11:55:17 -0700 Subject: [PATCH 017/346] Using IOChannelUtils to resolve file path --- .../test/java/org/apache/beam/examples/WordCountIT.java | 3 ++- .../apache/beam/sdk/testing/FileChecksumMatcherTest.java | 7 +++---- 2 files changed, 5 insertions(+), 5 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 e438faf4f6069..f93dc2b2f7792 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 @@ -41,7 +41,8 @@ public class WordCountIT { /** * Options for the WordCount Integration Test. * - * Define expected output file checksum to verify WordCount pipeline result with customized input. + *

Define expected output file checksum to verify WordCount pipeline result + * with customized input. */ public interface WordCountITOptions extends TestPipelineOptions, WordCountOptions { @Default.String("c04722202dee29c442b55ead54c6000693e85e77") 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 eebb73e33212d..d94ffe2c47615 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 @@ -21,9 +21,8 @@ import static org.hamcrest.Matchers.containsString; import org.apache.beam.sdk.PipelineResult; - +import org.apache.beam.sdk.util.IOChannelUtils; import com.google.common.io.Files; - import org.junit.Rule; import org.junit.Test; import org.junit.rules.ExpectedException; @@ -99,8 +98,8 @@ public void testMatcherVerifyMultipleFilesInOneDir() throws IOException { FileChecksumMatcher matcher = new FileChecksumMatcher( "90552392c28396935fe4f123bd0b5c2d0f6260c8", - tmpFolder.getRoot().getPath() + "/*"); + IOChannelUtils.resolve(tmpFolder.getRoot().getPath(), "*")); assertThat(pResult, matcher); } - } +} From 58cd781c82fa728f34f5ab0641f8f9b6edcf449c Mon Sep 17 00:00:00 2001 From: Ian Zhou Date: Fri, 5 Aug 2016 15:31:59 -0700 Subject: [PATCH 018/346] Added unit tests and error handling in removeTemporaryTables --- .../beam/sdk/io/gcp/bigquery/BigQueryIO.java | 34 +-- .../sdk/io/gcp/bigquery/BigQueryIOTest.java | 193 ++++++++++++------ 2 files changed, 148 insertions(+), 79 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 36e09f1dbee0a..46c2935942ee3 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 @@ -2062,7 +2062,7 @@ public void processElement(ProcessContext c) throws Exception { createDisposition); c.output(toJsonString(ref)); - removeTemporaryFiles(c.getPipelineOptions(), partition); + removeTemporaryFiles(c.getPipelineOptions(), tempFilePrefix, partition); } private void load( @@ -2108,16 +2108,17 @@ private void load( + "retries: %d", jobIdPrefix, Bound.MAX_RETRY_JOBS)); } - private void removeTemporaryFiles(PipelineOptions options, Collection matches) + static void removeTemporaryFiles( + PipelineOptions options, + String tempFilePrefix, + Collection files) throws IOException { - String pattern = tempFilePrefix + "*"; - LOG.debug("Finding temporary files matching {}.", pattern); - IOChannelFactory factory = IOChannelUtils.getFactory(pattern); + IOChannelFactory factory = IOChannelUtils.getFactory(tempFilePrefix); if (factory instanceof GcsIOChannelFactory) { GcsUtil gcsUtil = new GcsUtil.GcsUtilFactory().create(options); - gcsUtil.remove(matches); + gcsUtil.remove(files); } else if (factory instanceof FileIOChannelFactory) { - for (String filename : matches) { + for (String filename : files) { LOG.debug("Removing file {}", filename); boolean exists = Files.deleteIfExists(Paths.get(filename)); if (!exists) { @@ -2175,7 +2176,7 @@ public WriteRename( public void processElement(ProcessContext c) throws Exception { List tempTablesJson = Lists.newArrayList(c.sideInput(tempTablesView)); - // Do not copy if not temp tables are provided + // Do not copy if no temp tables are provided if (tempTablesJson.size() == 0) { return; } @@ -2237,13 +2238,18 @@ private void copy( + "retries: %d", jobIdPrefix, Bound.MAX_RETRY_JOBS)); } - private void removeTemporaryTables(DatasetService tableService, - List tempTables) throws Exception { + static void removeTemporaryTables(DatasetService tableService, + List tempTables) { for (TableReference tableRef : tempTables) { - tableService.deleteTable( - tableRef.getProjectId(), - tableRef.getDatasetId(), - tableRef.getTableId()); + try { + LOG.debug("Deleting table {}", toJsonString(tableRef)); + tableService.deleteTable( + tableRef.getProjectId(), + tableRef.getDatasetId(), + tableRef.getTableId()); + } catch (Exception e) { + LOG.warn("Failed to delete the table {}", toJsonString(tableRef), 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 6d6eb60defe6a..fcaa054da4565 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 @@ -30,6 +30,8 @@ import static org.junit.Assert.assertTrue; import static org.mockito.Matchers.anyString; import static org.mockito.Matchers.eq; +import static org.mockito.Mockito.doNothing; +import static org.mockito.Mockito.doThrow; import static org.mockito.Mockito.when; import org.apache.beam.sdk.Pipeline; @@ -47,6 +49,7 @@ import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.Status; import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.TransformingSource; import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.Write.CreateDisposition; +import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.Write.TableRowWriter; import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.Write.WriteDisposition; import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.Write.WritePartition; import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.Write.WriteRename; @@ -140,8 +143,6 @@ @RunWith(JUnit4.class) public class BigQueryIOTest implements Serializable { - @Rule public transient TemporaryFolder tmpFolder = new TemporaryFolder(); - // Status.UNKNOWN maps to null private static final Map JOB_STATUS_MAP = ImmutableMap.of( Status.SUCCEEDED, new Job().setStatus(new JobStatus()), @@ -586,12 +587,11 @@ public void processElement(ProcessContext c) throws Exception { @Test @Category(NeedsRunner.class) - public void testCustomSink() throws Exception { + public void testWrite() throws Exception { FakeBigQueryServices fakeBqServices = new FakeBigQueryServices() .withJobService(new FakeJobService() .startJobReturns("done", "done", "done") - .pollJobReturns(Status.FAILED, Status.FAILED, Status.SUCCEEDED)) - .withDatasetService(mockDatasetService); + .pollJobReturns(Status.FAILED, Status.FAILED, Status.SUCCEEDED)); Pipeline p = TestPipeline.create(bqOptions); p.apply(Create.of( @@ -610,17 +610,18 @@ public void testCustomSink() throws Exception { p.run(); logged.verifyInfo("Starting BigQuery load job"); + logged.verifyInfo("BigQuery load job failed"); + logged.verifyInfo("try 0/" + BigQueryIO.Write.Bound.MAX_RETRY_JOBS); + logged.verifyInfo("try 1/" + BigQueryIO.Write.Bound.MAX_RETRY_JOBS); + logged.verifyInfo("try 2/" + BigQueryIO.Write.Bound.MAX_RETRY_JOBS); + logged.verifyNotLogged("try 3/" + BigQueryIO.Write.Bound.MAX_RETRY_JOBS); File tempDir = new File(bqOptions.getTempLocation()); - assertEquals(0, tempDir.listFiles(new FileFilter() { - @Override - public boolean accept(File pathname) { - return pathname.isFile(); - }}).length); + testNumFiles(tempDir, 0); } @Test @Category(NeedsRunner.class) - public void testCustomSinkUnknown() throws Exception { + public void testWriteUnknown() throws Exception { FakeBigQueryServices fakeBqServices = new FakeBigQueryServices() .withJobService(new FakeJobService() .startJobReturns("done", "done") @@ -642,11 +643,7 @@ public void testCustomSinkUnknown() throws Exception { p.run(); File tempDir = new File(bqOptions.getTempLocation()); - assertEquals(0, tempDir.listFiles(new FileFilter() { - @Override - public boolean accept(File pathname) { - return pathname.isFile(); - }}).length); + testNumFiles(tempDir, 0); } @Test @@ -703,7 +700,7 @@ public void testQuerySourcePrimitiveDisplayData() throws IOException, Interrupte @Test - public void testBuildSink() { + public void testBuildWrite() { BigQueryIO.Write.Bound bound = BigQueryIO.Write.to("foo.com:project:somedataset.sometable"); checkWriteObject( bound, "foo.com:project", "somedataset", "sometable", @@ -713,18 +710,18 @@ public void testBuildSink() { @Test @Category(RunnableOnService.class) @Ignore("[BEAM-436] DirectRunner RunnableOnService tempLocation configuration insufficient") - public void testBatchSinkPrimitiveDisplayData() throws IOException, InterruptedException { - testSinkPrimitiveDisplayData(/* streaming: */ false); + public void testBatchWritePrimitiveDisplayData() throws IOException, InterruptedException { + testWritePrimitiveDisplayData(/* streaming: */ false); } @Test @Category(RunnableOnService.class) @Ignore("[BEAM-436] DirectRunner RunnableOnService tempLocation configuration insufficient") - public void testStreamingSinkPrimitiveDisplayData() throws IOException, InterruptedException { - testSinkPrimitiveDisplayData(/* streaming: */ true); + public void testStreamingWritePrimitiveDisplayData() throws IOException, InterruptedException { + testWritePrimitiveDisplayData(/* streaming: */ true); } - private void testSinkPrimitiveDisplayData(boolean streaming) throws IOException, + private void testWritePrimitiveDisplayData(boolean streaming) throws IOException, InterruptedException { PipelineOptions options = TestPipeline.testingPipelineOptions(); options.as(StreamingOptions.class).setStreaming(streaming); @@ -747,7 +744,7 @@ private void testSinkPrimitiveDisplayData(boolean streaming) throws IOException, } @Test - public void testBuildSinkwithoutValidation() { + public void testBuildWriteWithoutValidation() { // This test just checks that using withoutValidation will not trigger object // construction errors. BigQueryIO.Write.Bound bound = @@ -758,7 +755,7 @@ public void testBuildSinkwithoutValidation() { } @Test - public void testBuildSinkDefaultProject() { + public void testBuildWriteDefaultProject() { BigQueryIO.Write.Bound bound = BigQueryIO.Write.to("somedataset.sometable"); checkWriteObject( bound, null, "somedataset", "sometable", @@ -766,7 +763,7 @@ public void testBuildSinkDefaultProject() { } @Test - public void testBuildSinkWithTableReference() { + public void testBuildWriteWithTableReference() { TableReference table = new TableReference() .setProjectId("foo.com:project") .setDatasetId("somedataset") @@ -779,7 +776,7 @@ public void testBuildSinkWithTableReference() { @Test @Category(RunnableOnService.class) - public void testBuildSinkWithoutTable() { + public void testBuildWriteWithoutTable() { Pipeline p = TestPipeline.create(); thrown.expect(IllegalStateException.class); thrown.expectMessage("must set the table reference"); @@ -788,7 +785,7 @@ public void testBuildSinkWithoutTable() { } @Test - public void testBuildSinkWithSchema() { + public void testBuildWriteWithSchema() { TableSchema schema = new TableSchema(); BigQueryIO.Write.Bound bound = BigQueryIO.Write.to("foo.com:project:somedataset.sometable").withSchema(schema); @@ -798,7 +795,7 @@ public void testBuildSinkWithSchema() { } @Test - public void testBuildSinkWithCreateDispositionNever() { + public void testBuildWriteWithCreateDispositionNever() { BigQueryIO.Write.Bound bound = BigQueryIO.Write .to("foo.com:project:somedataset.sometable") .withCreateDisposition(CreateDisposition.CREATE_NEVER); @@ -808,7 +805,7 @@ public void testBuildSinkWithCreateDispositionNever() { } @Test - public void testBuildSinkWithCreateDispositionIfNeeded() { + public void testBuildWriteWithCreateDispositionIfNeeded() { BigQueryIO.Write.Bound bound = BigQueryIO.Write .to("foo.com:project:somedataset.sometable") .withCreateDisposition(CreateDisposition.CREATE_IF_NEEDED); @@ -818,7 +815,7 @@ public void testBuildSinkWithCreateDispositionIfNeeded() { } @Test - public void testBuildSinkWithWriteDispositionTruncate() { + public void testBuildWriteWithWriteDispositionTruncate() { BigQueryIO.Write.Bound bound = BigQueryIO.Write .to("foo.com:project:somedataset.sometable") .withWriteDisposition(WriteDisposition.WRITE_TRUNCATE); @@ -828,7 +825,7 @@ public void testBuildSinkWithWriteDispositionTruncate() { } @Test - public void testBuildSinkWithWriteDispositionAppend() { + public void testBuildWriteWithWriteDispositionAppend() { BigQueryIO.Write.Bound bound = BigQueryIO.Write .to("foo.com:project:somedataset.sometable") .withWriteDisposition(WriteDisposition.WRITE_APPEND); @@ -838,7 +835,7 @@ public void testBuildSinkWithWriteDispositionAppend() { } @Test - public void testBuildSinkWithWriteDispositionEmpty() { + public void testBuildWriteWithWriteDispositionEmpty() { BigQueryIO.Write.Bound bound = BigQueryIO.Write .to("foo.com:project:somedataset.sometable") .withWriteDisposition(WriteDisposition.WRITE_EMPTY); @@ -848,7 +845,7 @@ public void testBuildSinkWithWriteDispositionEmpty() { } @Test - public void testBuildSinkDisplayData() { + public void testBuildWriteDisplayData() { String tableSpec = "project:dataset.table"; TableSchema schema = new TableSchema().set("col1", "type1").set("col2", "type2"); @@ -1256,53 +1253,53 @@ void cleanup(PipelineOptions options) throws Exception { @Test public void testWritePartitionEmptyData() throws Exception { - final long numFiles = 0; - final long fileSize = 0; + long numFiles = 0; + long fileSize = 0; // An empty file is created for no input data. One partition is needed. - final long expectedNumPartitions = 1; + long expectedNumPartitions = 1; testWritePartition(numFiles, fileSize, expectedNumPartitions); } @Test public void testWritePartitionSinglePartition() throws Exception { - final long numFiles = BigQueryIO.Write.Bound.MAX_NUM_FILES; - final long fileSize = 1; + long numFiles = BigQueryIO.Write.Bound.MAX_NUM_FILES; + long fileSize = 1; // One partition is needed. - final long expectedNumPartitions = 1; + long expectedNumPartitions = 1; testWritePartition(numFiles, fileSize, expectedNumPartitions); } @Test public void testWritePartitionManyFiles() throws Exception { - final long numFiles = BigQueryIO.Write.Bound.MAX_NUM_FILES * 3; - final long fileSize = 1; + long numFiles = BigQueryIO.Write.Bound.MAX_NUM_FILES * 3; + long fileSize = 1; // One partition is needed for each group of BigQueryWrite.MAX_NUM_FILES files. - final long expectedNumPartitions = 3; + long expectedNumPartitions = 3; testWritePartition(numFiles, fileSize, expectedNumPartitions); } @Test public void testWritePartitionLargeFileSize() throws Exception { - final long numFiles = 10; - final long fileSize = BigQueryIO.Write.Bound.MAX_SIZE_BYTES / 3; + long numFiles = 10; + long fileSize = BigQueryIO.Write.Bound.MAX_SIZE_BYTES / 3; // One partition is needed for each group of three files. - final long expectedNumPartitions = 4; + long expectedNumPartitions = 4; testWritePartition(numFiles, fileSize, expectedNumPartitions); } private void testWritePartition(long numFiles, long fileSize, long expectedNumPartitions) throws Exception { - final List expectedPartitionIds = Lists.newArrayList(); + List expectedPartitionIds = Lists.newArrayList(); for (long i = 1; i <= expectedNumPartitions; ++i) { expectedPartitionIds.add(i); } - final List> files = Lists.newArrayList(); - final List fileNames = Lists.newArrayList(); + List> files = Lists.newArrayList(); + List fileNames = Lists.newArrayList(); for (int i = 0; i < numFiles; ++i) { String fileName = String.format("files%05d", i); fileNames.add(fileName); @@ -1314,7 +1311,7 @@ private void testWritePartition(long numFiles, long fileSize, long expectedNumPa TupleTag>> singlePartitionTag = new TupleTag>>("singlePartitionTag") {}; - final PCollectionView>> filesView = PCollectionViews.iterableView( + PCollectionView>> filesView = PCollectionViews.iterableView( TestPipeline.create(), WindowingStrategy.globalDefault(), KvCoder.of(StringUtf8Coder.of(), VarLongCoder.of())); @@ -1324,7 +1321,7 @@ private void testWritePartition(long numFiles, long fileSize, long expectedNumPa DoFnTester>> tester = DoFnTester.of(writePartition); tester.setSideInput(filesView, GlobalWindow.INSTANCE, files); - tester.processElement(tmpFolder.getRoot().getAbsolutePath()); + tester.processElement(bqOptions.getTempLocation()); List>> partitions; if (expectedNumPartitions > 1) { @@ -1357,18 +1354,17 @@ public void testWriteTables() throws Exception { FakeBigQueryServices fakeBqServices = new FakeBigQueryServices() .withJobService(new FakeJobService() .startJobReturns("done", "done", "done", "done") - .pollJobReturns(Status.FAILED, Status.SUCCEEDED, Status.SUCCEEDED, Status.SUCCEEDED)) - .withDatasetService(mockDatasetService); + .pollJobReturns(Status.FAILED, Status.SUCCEEDED, Status.SUCCEEDED, Status.SUCCEEDED)); - final long numPartitions = 3; - final long numFilesPerPartition = 10; - final String jobIdToken = "jobIdToken"; - final String tempFilePrefix = "tempFilePrefix"; - final String jsonTable = "{}"; - final String jsonSchema = "{}"; - final List expectedTempTables = Lists.newArrayList(); + long numPartitions = 3; + long numFilesPerPartition = 10; + String jobIdToken = "jobIdToken"; + String tempFilePrefix = "tempFilePrefix"; + String jsonTable = "{}"; + String jsonSchema = "{}"; + List expectedTempTables = Lists.newArrayList(); - final List>>> partitions = Lists.newArrayList(); + List>>> partitions = Lists.newArrayList(); for (long i = 0; i < numPartitions; ++i) { List filesPerPartition = Lists.newArrayList(); for (int j = 0; j < numFilesPerPartition; ++j) { @@ -1396,10 +1392,40 @@ public void testWriteTables() throws Exception { List tempTables = tester.takeOutputElements(); logged.verifyInfo("Starting BigQuery load job"); + logged.verifyInfo("BigQuery load job failed"); + logged.verifyInfo("try 0/" + BigQueryIO.Write.Bound.MAX_RETRY_JOBS); + logged.verifyInfo("try 1/" + BigQueryIO.Write.Bound.MAX_RETRY_JOBS); + logged.verifyNotLogged("try 2/" + BigQueryIO.Write.Bound.MAX_RETRY_JOBS); assertEquals(expectedTempTables, tempTables); } + @Test + public void testRemoveTemporaryFiles() throws Exception { + int numFiles = 10; + List fileNames = Lists.newArrayList(); + String tempFilePrefix = bqOptions.getTempLocation() + "/"; + TableRowWriter writer = new TableRowWriter(tempFilePrefix); + for (int i = 0; i < numFiles; ++i) { + String fileName = String.format("files%05d", i); + writer.open(fileName); + fileNames.add(writer.close().getKey()); + } + fileNames.add(tempFilePrefix + String.format("files%05d", numFiles)); + + File tempDir = new File(bqOptions.getTempLocation()); + testNumFiles(tempDir, 10); + + WriteTables.removeTemporaryFiles(bqOptions, tempFilePrefix, fileNames); + + testNumFiles(tempDir, 0); + + for (String fileName : fileNames) { + logged.verifyDebug("Removing file " + fileName); + } + logged.verifyDebug(fileNames.get(numFiles) + " does not exist."); + } + @Test public void testWriteRename() throws Exception { FakeBigQueryServices fakeBqServices = new FakeBigQueryServices() @@ -1408,15 +1434,15 @@ public void testWriteRename() throws Exception { .pollJobReturns(Status.FAILED, Status.SUCCEEDED)) .withDatasetService(mockDatasetService); - final long numTempTables = 3; - final String jobIdToken = "jobIdToken"; - final String jsonTable = "{}"; - final List tempTables = Lists.newArrayList(); + long numTempTables = 3; + String jobIdToken = "jobIdToken"; + String jsonTable = "{}"; + List tempTables = Lists.newArrayList(); for (long i = 0; i < numTempTables; ++i) { tempTables.add(String.format("{\"tableId\":\"%s_%05d\"}", jobIdToken, i)); } - final PCollectionView> tempTablesView = PCollectionViews.iterableView( + PCollectionView> tempTablesView = PCollectionViews.iterableView( TestPipeline.create(), WindowingStrategy.globalDefault(), StringUtf8Coder.of()); @@ -1434,5 +1460,42 @@ public void testWriteRename() throws Exception { tester.processElement(null); logged.verifyInfo("Starting BigQuery copy job"); + logged.verifyInfo("BigQuery copy job failed"); + logged.verifyInfo("try 0/" + BigQueryIO.Write.Bound.MAX_RETRY_JOBS); + logged.verifyInfo("try 1/" + BigQueryIO.Write.Bound.MAX_RETRY_JOBS); + logged.verifyNotLogged("try 2/" + BigQueryIO.Write.Bound.MAX_RETRY_JOBS); + } + + @Test + public void testRemoveTemporaryTables() throws Exception { + String projectId = "someproject"; + String datasetId = "somedataset"; + List tables = Lists.newArrayList("table1", "table2", "table3"); + List tableRefs = Lists.newArrayList( + BigQueryIO.parseTableSpec(String.format("%s:%s.%s", projectId, datasetId, tables.get(0))), + BigQueryIO.parseTableSpec(String.format("%s:%s.%s", projectId, datasetId, tables.get(1))), + BigQueryIO.parseTableSpec(String.format("%s:%s.%s", projectId, datasetId, tables.get(2)))); + + doThrow(new IOException("Unable to delete table")) + .when(mockDatasetService).deleteTable(projectId, datasetId, tables.get(0)); + doNothing().when(mockDatasetService).deleteTable(projectId, datasetId, tables.get(1)); + doNothing().when(mockDatasetService).deleteTable(projectId, datasetId, tables.get(2)); + + WriteRename.removeTemporaryTables(mockDatasetService, tableRefs); + + for (TableReference ref : tableRefs) { + logged.verifyDebug("Deleting table " + toJsonString(ref)); + } + logged.verifyWarn("Failed to delete the table " + toJsonString(tableRefs.get(0))); + logged.verifyNotLogged("Failed to delete the table " + toJsonString(tableRefs.get(1))); + logged.verifyNotLogged("Failed to delete the table " + toJsonString(tableRefs.get(2))); + } + + private static void testNumFiles(File tempDir, int expectedNumFiles) { + assertEquals(expectedNumFiles, tempDir.listFiles(new FileFilter() { + @Override + public boolean accept(File pathname) { + return pathname.isFile(); + }}).length); } } From d99a652f96f12bcc235caa038ffa741906336b1f Mon Sep 17 00:00:00 2001 From: Maximilian Michels Date: Fri, 12 Aug 2016 17:51:02 +0200 Subject: [PATCH 019/346] [flink] add missing maven config to example pom --- runners/flink/README.md | 17 +++++++++++++++++ 1 file changed, 17 insertions(+) diff --git a/runners/flink/README.md b/runners/flink/README.md index aeb16922a96b8..8361112028c9a 100644 --- a/runners/flink/README.md +++ b/runners/flink/README.md @@ -171,6 +171,14 @@ The contents of the root `pom.xml` should be slightly changed aftewards (explana beam-runners-flink_2.10 0.2.0-incubating-SNAPSHOT + + + + @@ -196,6 +204,15 @@ The contents of the root `pom.xml` should be slightly changed aftewards (explana + + org.apache.maven.plugins + maven-compiler-plugin + + 1.7 + 1.7 + + + From 39f763e16182e33019a1805d6210549934998856 Mon Sep 17 00:00:00 2001 From: Pei He Date: Mon, 1 Aug 2016 13:41:59 -0700 Subject: [PATCH 020/346] Remove DataflowPipelineJob from examples --- examples/java/pom.xml | 7 +- .../beam/examples/common/ExampleUtils.java | 74 +++++-------------- .../runners/dataflow/DataflowPipelineJob.java | 14 +++- 3 files changed, 31 insertions(+), 64 deletions(-) diff --git a/examples/java/pom.xml b/examples/java/pom.xml index cdf80cb8297f1..6efbc54406d91 100644 --- a/examples/java/pom.xml +++ b/examples/java/pom.xml @@ -219,11 +219,6 @@ google-api-client - - com.google.apis - google-api-services-dataflow - - com.google.apis google-api-services-bigquery @@ -286,6 +281,8 @@ org.apache.beam beam-runners-google-cloud-dataflow-java ${project.version} + runtime + true diff --git a/examples/java/src/main/java/org/apache/beam/examples/common/ExampleUtils.java b/examples/java/src/main/java/org/apache/beam/examples/common/ExampleUtils.java index 8b66861107ad3..7f03fc0d1c6f6 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/common/ExampleUtils.java +++ b/examples/java/src/main/java/org/apache/beam/examples/common/ExampleUtils.java @@ -17,17 +17,10 @@ */ package org.apache.beam.examples.common; -import org.apache.beam.runners.dataflow.BlockingDataflowRunner; -import org.apache.beam.runners.dataflow.DataflowPipelineJob; -import org.apache.beam.runners.dataflow.DataflowRunner; -import org.apache.beam.runners.dataflow.options.DataflowPipelineOptions; -import org.apache.beam.runners.dataflow.util.MonitoringUtil; 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.options.StreamingOptions; -import org.apache.beam.sdk.runners.PipelineRunner; import org.apache.beam.sdk.util.AttemptBoundedExponentialBackOff; import org.apache.beam.sdk.util.Transport; @@ -44,7 +37,6 @@ 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.dataflow.Dataflow; import com.google.api.services.pubsub.Pubsub; import com.google.api.services.pubsub.model.Subscription; import com.google.api.services.pubsub.model.Topic; @@ -71,8 +63,7 @@ public class ExampleUtils { private final PipelineOptions options; private Bigquery bigQueryClient = null; private Pubsub pubsubClient = null; - private Dataflow dataflowClient = null; - private Set jobsToCancel = Sets.newHashSet(); + private Set pipelinesToCancel = Sets.newHashSet(); private List pendingMessages = Lists.newArrayList(); /** @@ -80,7 +71,6 @@ public class ExampleUtils { */ public ExampleUtils(PipelineOptions options) { this.options = options; - setupRunner(); } /** @@ -280,72 +270,47 @@ private void deletePubsubSubscription(String subscription) throws IOException { } } - /** - * Do some runner setup: check that the DirectRunner is not used in conjunction with - * streaming, and if streaming is specified, use the DataflowRunner. - */ - private void setupRunner() { - Class> runner = options.getRunner(); - if (options.as(StreamingOptions.class).isStreaming() - && runner.equals(BlockingDataflowRunner.class)) { - // In order to cancel the pipelines automatically, - // {@literal DataflowRunner} is forced to be used. - options.setRunner(DataflowRunner.class); - } - } - /** * If {@literal DataflowRunner} or {@literal BlockingDataflowRunner} is used, * waits for the pipeline to finish and cancels it (and the injector) before the program exists. */ public void waitToFinish(PipelineResult result) { - if (result instanceof DataflowPipelineJob) { - final DataflowPipelineJob job = (DataflowPipelineJob) result; - jobsToCancel.add(job); - if (!options.as(ExampleOptions.class).getKeepJobsRunning()) { - addShutdownHook(jobsToCancel); - } - try { - job.waitUntilFinish(); - } catch (Exception e) { - throw new RuntimeException("Failed to wait for job to finish: " + job.getJobId()); - } - } else { + 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 jobs) { - if (dataflowClient == null) { - dataflowClient = options.as(DataflowPipelineOptions.class).getDataflowClient(); - } - + private void addShutdownHook(final Collection pipelineResults) { Runtime.getRuntime().addShutdownHook(new Thread() { @Override public void run() { tearDown(); printPendingMessages(); - for (DataflowPipelineJob job : jobs) { - System.out.println("Canceling example pipeline: " + job.getJobId()); + for (PipelineResult pipelineResult : pipelineResults) { try { - job.cancel(); + pipelineResult.cancel(); } catch (IOException e) { - System.out.println("Failed to cancel the job," - + " please go to the Developers Console to cancel it manually"); - System.out.println( - MonitoringUtil.getJobMonitoringPageURL(job.getProjectId(), job.getJobId())); + System.out.println("Failed to cancel the job."); + System.out.println(e.getMessage()); } } - for (DataflowPipelineJob job : jobs) { + for (PipelineResult pipelineResult : pipelineResults) { boolean cancellationVerified = false; for (int retryAttempts = 6; retryAttempts > 0; retryAttempts--) { - if (job.getState().isTerminal()) { + if (pipelineResult.getState().isTerminal()) { cancellationVerified = true; - System.out.println("Canceled example pipeline: " + job.getJobId()); break; } else { System.out.println( @@ -354,10 +319,7 @@ public void run() { Uninterruptibles.sleepUninterruptibly(10, TimeUnit.SECONDS); } if (!cancellationVerified) { - System.out.println("Failed to verify the cancellation for job: " + job.getJobId()); - System.out.println("Please go to the Developers Console to verify manually:"); - System.out.println( - MonitoringUtil.getJobMonitoringPageURL(job.getProjectId(), job.getJobId())); + System.out.println("Failed to verify the cancellation for job: " + pipelineResult); } } } 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 e043e23ad162c..3d0f1458fbae2 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 @@ -271,9 +271,17 @@ public State cancel() throws IOException { content.setProjectId(projectId); content.setId(jobId); content.setRequestedState("JOB_STATE_CANCELLED"); - dataflowOptions.getDataflowClient().projects().jobs() - .update(projectId, jobId, content) - .execute(); + try { + dataflowOptions.getDataflowClient().projects().jobs() + .update(projectId, jobId, content) + .execute(); + } catch (IOException e) { + String errorMsg = String.format( + "Failed to cancel the job, please go to the Developers Console to cancel it manually: %s", + MonitoringUtil.getJobMonitoringPageURL(getProjectId(), getJobId())); + LOG.warn(errorMsg); + throw new IOException(errorMsg, e); + } return State.CANCELLED; } From 424c4c492965e5a93b1c020c8d52805e3a9a9088 Mon Sep 17 00:00:00 2001 From: mariusz89016 Date: Sun, 14 Aug 2016 00:35:19 +0200 Subject: [PATCH 021/346] [BEAM-432] Corrected BigQueryIO javadoc --- .../java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 46c2935942ee3..aa168bdab0db0 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 @@ -177,7 +177,7 @@ * *

{@code
  * PCollection shakespeare = pipeline.apply(
- *     BigQueryIO.Read.fromQuery("SELECT year, mean_temp FROM samples.weather_stations"));
+ *     BigQueryIO.Read.fromQuery("SELECT year, mean_temp FROM [samples.weather_stations]"));
  * }
* *

When creating a BigQuery input transform, users should provide either a query or a table. From b80d96748dcb71f93697126489c924020ebbd4a9 Mon Sep 17 00:00:00 2001 From: Thomas Groh Date: Fri, 15 Jul 2016 11:27:00 -0700 Subject: [PATCH 022/346] Add TransformEvaluatorFactory#cleanup This cleans up any state stored within the Transform Evaluator Factory. --- .../direct/BoundedReadEvaluatorFactory.java | 4 ++ .../ExecutorServiceParallelExecutor.java | 9 +++- .../direct/FlattenEvaluatorFactory.java | 3 ++ .../GroupAlsoByWindowEvaluatorFactory.java | 6 ++- .../GroupByKeyOnlyEvaluatorFactory.java | 4 +- .../direct/ParDoMultiEvaluatorFactory.java | 5 +++ .../direct/ParDoSingleEvaluatorFactory.java | 5 +++ .../direct/TransformEvaluatorFactory.java | 8 ++++ .../direct/TransformEvaluatorRegistry.java | 41 +++++++++++++++++++ .../direct/UnboundedReadEvaluatorFactory.java | 3 ++ .../runners/direct/ViewEvaluatorFactory.java | 3 ++ .../direct/WindowEvaluatorFactory.java | 3 ++ 12 files changed, 90 insertions(+), 4 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 2f4f86c1ee18c..0c4b7fd9bc800 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 @@ -60,6 +60,10 @@ public TransformEvaluator forApplication( return getTransformEvaluator((AppliedPTransform) application, evaluationContext); } + @Override + public void cleanup() { + } + /** * Get a {@link TransformEvaluator} that produces elements for the provided application of * {@link Bounded Read.Bounded}, initializing the queue of evaluators if required. 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 a0a5ec0e5df30..8c6c6eddb3b31 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 @@ -447,13 +447,18 @@ private void fireTimers() throws Exception { private boolean shouldShutdown() { boolean shouldShutdown = exceptionThrown || evaluationContext.isDone(); if (shouldShutdown) { + LOG.debug("Pipeline has terminated. Shutting down."); + executorService.shutdown(); + try { + registry.cleanup(); + } catch (Exception e) { + visibleUpdates.add(VisibleExecutorUpdate.fromThrowable(e)); + } if (evaluationContext.isDone()) { - LOG.debug("Pipeline is finished. Shutting down. {}"); while (!visibleUpdates.offer(VisibleExecutorUpdate.finished())) { visibleUpdates.poll(); } } - executorService.shutdown(); } return shouldShutdown; } 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 c84f62035a86e..5a0d31dee27de 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 @@ -43,6 +43,9 @@ public TransformEvaluator forApplication( return evaluator; } + @Override + public void cleanup() throws Exception {} + private TransformEvaluator createInMemoryEvaluator( final AppliedPTransform< PCollectionList, PCollection, FlattenPCollectionList> 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 e052226c7e5bb..d16ffa0bf5171 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 @@ -61,11 +61,15 @@ public TransformEvaluator forApplication( return evaluator; } + @Override + public void cleanup() {} + private TransformEvaluator> createEvaluator( AppliedPTransform< PCollection>, PCollection>>, - DirectGroupAlsoByWindow> application, + DirectGroupAlsoByWindow> + application, CommittedBundle> inputBundle, EvaluationContext evaluationContext) { return new GroupAlsoByWindowEvaluator<>( 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 0e419c3959e28..dbdbdaf4ea348 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 @@ -18,7 +18,6 @@ package org.apache.beam.runners.direct; import static org.apache.beam.sdk.util.CoderUtils.encodeToByteArray; - import static com.google.common.base.Preconditions.checkState; import org.apache.beam.runners.direct.DirectGroupByKey.DirectGroupByKeyOnly; @@ -61,6 +60,9 @@ public TransformEvaluator forApplication( return evaluator; } + @Override + public void cleanup() {} + private TransformEvaluator>> createEvaluator( final AppliedPTransform< PCollection>>, diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoMultiEvaluatorFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoMultiEvaluatorFactory.java index ce770cae37ca1..40533c0072af4 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoMultiEvaluatorFactory.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoMultiEvaluatorFactory.java @@ -70,6 +70,11 @@ public TransformEvaluator forApplication( return evaluator; } + @Override + public void cleanup() { + + } + private TransformEvaluator createMultiEvaluator( AppliedPTransform, PCollectionTuple, BoundMulti> application, CommittedBundle inputBundle, diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoSingleEvaluatorFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoSingleEvaluatorFactory.java index 53af6af7cdf53..201fb46b7453d 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoSingleEvaluatorFactory.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoSingleEvaluatorFactory.java @@ -69,6 +69,11 @@ public TransformEvaluator forApplication( return evaluator; } + @Override + public void cleanup() { + + } + private TransformEvaluator createSingleEvaluator( AppliedPTransform, PCollection, Bound> application, diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TransformEvaluatorFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TransformEvaluatorFactory.java index d021b43037e5b..3655d26dd553c 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TransformEvaluatorFactory.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TransformEvaluatorFactory.java @@ -51,4 +51,12 @@ public interface TransformEvaluatorFactory { @Nullable TransformEvaluator forApplication( AppliedPTransform application, @Nullable CommittedBundle inputBundle, EvaluationContext evaluationContext) throws Exception; + + /** + * Cleans up any state maintained by this {@link TransformEvaluatorFactory}. Called after a + * {@link Pipeline} is shut down. No more calls to + * {@link #forApplication(AppliedPTransform, CommittedBundle, EvaluationContext)} will be made + * after a call to {@link #cleanup()}. + */ + void cleanup() throws Exception; } 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 f0afc3be5d7a8..b46923775c484 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 @@ -17,6 +17,8 @@ */ package org.apache.beam.runners.direct; +import static com.google.common.base.Preconditions.checkState; + import org.apache.beam.runners.direct.DirectGroupByKey.DirectGroupAlsoByWindow; import org.apache.beam.runners.direct.DirectGroupByKey.DirectGroupByKeyOnly; import org.apache.beam.runners.direct.DirectRunner.CommittedBundle; @@ -29,7 +31,13 @@ import com.google.common.collect.ImmutableMap; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.ArrayList; +import java.util.Collection; import java.util.Map; +import java.util.concurrent.atomic.AtomicBoolean; import javax.annotation.Nullable; @@ -38,6 +46,7 @@ * implementations based on the type of {@link PTransform} of the application. */ class TransformEvaluatorRegistry implements TransformEvaluatorFactory { + private static final Logger LOG = LoggerFactory.getLogger(TransformEvaluatorRegistry.class); public static TransformEvaluatorRegistry defaultRegistry() { @SuppressWarnings("rawtypes") ImmutableMap, TransformEvaluatorFactory> primitives = @@ -61,6 +70,8 @@ public static TransformEvaluatorRegistry defaultRegistry() { @SuppressWarnings("rawtypes") private final Map, TransformEvaluatorFactory> factories; + private final AtomicBoolean finished = new AtomicBoolean(false); + private TransformEvaluatorRegistry( @SuppressWarnings("rawtypes") Map, TransformEvaluatorFactory> factories) { @@ -73,7 +84,37 @@ public TransformEvaluator forApplication( @Nullable CommittedBundle inputBundle, EvaluationContext evaluationContext) throws Exception { + checkState( + !finished.get(), "Tried to get an evaluator for a finished TransformEvaluatorRegistry"); TransformEvaluatorFactory factory = factories.get(application.getTransform().getClass()); return factory.forApplication(application, inputBundle, evaluationContext); } + + @Override + public void cleanup() throws Exception { + Collection thrownInCleanup = new ArrayList<>(); + for (TransformEvaluatorFactory factory : factories.values()) { + try { + factory.cleanup(); + } catch (Exception e) { + if (e instanceof InterruptedException) { + Thread.currentThread().interrupt(); + } + thrownInCleanup.add(e); + } + } + finished.set(true); + if (!thrownInCleanup.isEmpty()) { + LOG.error("Exceptions {} thrown while cleaning up evaluators", thrownInCleanup); + Exception toThrow = null; + for (Exception e : thrownInCleanup) { + if (toThrow == null) { + toThrow = e; + } else { + toThrow.addSuppressed(e); + } + } + throw toThrow; + } + } } 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 0e2745b8bde3b..c4d408b232f04 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 @@ -113,6 +113,9 @@ TransformEvaluator getTransformEvaluator( return evaluatorQueue.poll(); } + @Override + public void cleanup() {} + /** * A {@link UnboundedReadEvaluator} produces elements from an underlying {@link UnboundedSource}, * discarding all input elements. Within the call to {@link #finishBundle()}, the evaluator 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 362e903a06305..3b0de4b6f8df0 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 @@ -59,6 +59,9 @@ public TransformEvaluator forApplication( return evaluator; } + @Override + public void cleanup() throws Exception {} + private TransformEvaluator> createEvaluator( final AppliedPTransform>, PCollectionView, WriteView> application, 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 67c2f173f0edb..f2e62cb0445d8 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 @@ -66,6 +66,9 @@ private TransformEvaluator createTransformEvaluator( return new WindowIntoEvaluator<>(transform, fn, outputBundle); } + @Override + public void cleanup() {} + private static class WindowIntoEvaluator implements TransformEvaluator { private final AppliedPTransform, PCollection, Window.Bound> transform; From 77c90d00ae715795e73efec8f8e85e3917cf8d80 Mon Sep 17 00:00:00 2001 From: Thomas Groh Date: Tue, 19 Jul 2016 11:03:15 -0700 Subject: [PATCH 023/346] Replace CloningThreadLocal with DoFnLifecycleManager This is a more focused interface that interacts with a DoFn before it is available for use and after it has completed and the reference is lost. It is required to properly support setup and teardown, as the fields in a ThreadLocal cannot all be cleaned up without additional tracking. Part of BEAM-452. --- .../runners/direct/CloningThreadLocal.java | 43 ------- .../runners/direct/DoFnLifecycleManager.java | 78 ++++++++++++ ...cleManagerRemovingTransformEvaluator.java} | 41 ++++-- .../direct/ParDoMultiEvaluatorFactory.java | 56 +++++---- .../direct/ParDoSingleEvaluatorFactory.java | 43 ++++--- .../direct/CloningThreadLocalTest.java | 92 -------------- ...anagerRemovingTransformEvaluatorTest.java} | 41 +++--- .../direct/DoFnLifecycleManagerTest.java | 119 ++++++++++++++++++ 8 files changed, 305 insertions(+), 208 deletions(-) delete mode 100644 runners/direct-java/src/main/java/org/apache/beam/runners/direct/CloningThreadLocal.java create mode 100644 runners/direct-java/src/main/java/org/apache/beam/runners/direct/DoFnLifecycleManager.java rename runners/direct-java/src/main/java/org/apache/beam/runners/direct/{ThreadLocalInvalidatingTransformEvaluator.java => DoFnLifecycleManagerRemovingTransformEvaluator.java} (53%) delete mode 100644 runners/direct-java/src/test/java/org/apache/beam/runners/direct/CloningThreadLocalTest.java rename runners/direct-java/src/test/java/org/apache/beam/runners/direct/{ThreadLocalInvalidatingTransformEvaluatorTest.java => DoFnLifecycleManagerRemovingTransformEvaluatorTest.java} (73%) create mode 100644 runners/direct-java/src/test/java/org/apache/beam/runners/direct/DoFnLifecycleManagerTest.java diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/CloningThreadLocal.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/CloningThreadLocal.java deleted file mode 100644 index b9dc4ca715ac3..0000000000000 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/CloningThreadLocal.java +++ /dev/null @@ -1,43 +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.direct; - -import org.apache.beam.sdk.util.SerializableUtils; - -import java.io.Serializable; - -/** - * A {@link ThreadLocal} that obtains the initial value by cloning an original value. - */ -class CloningThreadLocal extends ThreadLocal { - public static CloningThreadLocal of(T original) { - return new CloningThreadLocal<>(original); - } - - private final T original; - - private CloningThreadLocal(T original) { - this.original = original; - } - - @Override - public T initialValue() { - return SerializableUtils.clone(original); - } -} 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 new file mode 100644 index 0000000000000..27836577b3a7d --- /dev/null +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DoFnLifecycleManager.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.direct; + +import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.transforms.OldDoFn; +import org.apache.beam.sdk.util.SerializableUtils; + +import com.google.common.cache.CacheBuilder; +import com.google.common.cache.CacheLoader; +import com.google.common.cache.LoadingCache; + +/** + * Manages {@link DoFn} setup, teardown, and serialization. + * + *

{@link DoFnLifecycleManager} is similar to a {@link ThreadLocal} storing a {@link DoFn}, but + * calls the {@link DoFn} {@link Setup} the first time the {@link DoFn} is obtained and {@link + * Teardown} whenever the {@link DoFn} is removed, and provides a method for clearing all cached + * {@link DoFn DoFns}. + */ +class DoFnLifecycleManager { + public static DoFnLifecycleManager of(OldDoFn original) { + return new DoFnLifecycleManager(original); + } + + private final LoadingCache> outstanding; + + private DoFnLifecycleManager(OldDoFn original) { + this.outstanding = CacheBuilder.newBuilder().build(new DeserializingCacheLoader(original)); + } + + public OldDoFn get() throws Exception { + Thread currentThread = Thread.currentThread(); + return outstanding.get(currentThread); + } + + public void remove() throws Exception { + Thread currentThread = Thread.currentThread(); + outstanding.invalidate(currentThread); + } + + /** + * Remove all {@link DoFn DoFns} from this {@link DoFnLifecycleManager}. + */ + public void removeAll() throws Exception { + outstanding.invalidateAll(); + } + + private class DeserializingCacheLoader extends CacheLoader> { + private final byte[] original; + + public DeserializingCacheLoader(OldDoFn original) { + this.original = SerializableUtils.serializeToByteArray(original); + } + + @Override + public OldDoFn load(Thread key) throws Exception { + return (OldDoFn) SerializableUtils.deserializeFromByteArray(original, + "DoFn Copy in thread " + key.getName()); + } + } +} diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ThreadLocalInvalidatingTransformEvaluator.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DoFnLifecycleManagerRemovingTransformEvaluator.java similarity index 53% rename from runners/direct-java/src/main/java/org/apache/beam/runners/direct/ThreadLocalInvalidatingTransformEvaluator.java rename to runners/direct-java/src/main/java/org/apache/beam/runners/direct/DoFnLifecycleManagerRemovingTransformEvaluator.java index d8a6bf9e79980..f3d1d4f3d5842 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ThreadLocalInvalidatingTransformEvaluator.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DoFnLifecycleManagerRemovingTransformEvaluator.java @@ -20,25 +20,28 @@ import org.apache.beam.sdk.util.WindowedValue; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + /** * A {@link TransformEvaluator} which delegates calls to an underlying {@link TransformEvaluator}, - * clearing the value of a {@link ThreadLocal} if any call throws an exception. + * clearing the value of a {@link DoFnLifecycleManager} if any call throws an exception. */ -class ThreadLocalInvalidatingTransformEvaluator - implements TransformEvaluator { +class DoFnLifecycleManagerRemovingTransformEvaluator implements TransformEvaluator { + private static final Logger LOG = + LoggerFactory.getLogger(DoFnLifecycleManagerRemovingTransformEvaluator.class); private final TransformEvaluator underlying; - private final ThreadLocal threadLocal; + private final DoFnLifecycleManager lifecycleManager; public static TransformEvaluator wrapping( - TransformEvaluator underlying, - ThreadLocal threadLocal) { - return new ThreadLocalInvalidatingTransformEvaluator<>(underlying, threadLocal); + TransformEvaluator underlying, DoFnLifecycleManager threadLocal) { + return new DoFnLifecycleManagerRemovingTransformEvaluator<>(underlying, threadLocal); } - private ThreadLocalInvalidatingTransformEvaluator( - TransformEvaluator underlying, ThreadLocal threadLocal) { + private DoFnLifecycleManagerRemovingTransformEvaluator( + TransformEvaluator underlying, DoFnLifecycleManager threadLocal) { this.underlying = underlying; - this.threadLocal = threadLocal; + this.lifecycleManager = threadLocal; } @Override @@ -46,7 +49,14 @@ public void processElement(WindowedValue element) throws Exception { try { underlying.processElement(element); } catch (Exception e) { - threadLocal.remove(); + try { + lifecycleManager.remove(); + } catch (Exception removalException) { + LOG.error( + "Exception encountered while cleaning up after processing an element", + removalException); + e.addSuppressed(removalException); + } throw e; } } @@ -56,7 +66,14 @@ public TransformResult finishBundle() throws Exception { try { return underlying.finishBundle(); } catch (Exception e) { - threadLocal.remove(); + try { + lifecycleManager.remove(); + } catch (Exception removalException) { + LOG.error( + "Exception encountered while cleaning up after finishing a bundle", + removalException); + e.addSuppressed(removalException); + } throw e; } } diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoMultiEvaluatorFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoMultiEvaluatorFactory.java index 40533c0072af4..f2455e1f069e8 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoMultiEvaluatorFactory.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoMultiEvaluatorFactory.java @@ -31,6 +31,9 @@ import com.google.common.cache.CacheLoader; import com.google.common.cache.LoadingCache; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + import java.util.Map; /** @@ -38,32 +41,26 @@ * {@link BoundMulti} primitive {@link PTransform}. */ class ParDoMultiEvaluatorFactory implements TransformEvaluatorFactory { - private final LoadingCache>, ThreadLocal>> + private static final Logger LOG = LoggerFactory.getLogger(ParDoMultiEvaluatorFactory.class); + private final LoadingCache>, DoFnLifecycleManager> fnClones; public ParDoMultiEvaluatorFactory() { - fnClones = - CacheBuilder.newBuilder() - .build( - new CacheLoader< - AppliedPTransform>, ThreadLocal>>() { - @Override - public ThreadLocal> load( - AppliedPTransform> key) - throws Exception { - @SuppressWarnings({"unchecked", "rawtypes"}) - ThreadLocal threadLocal = - (ThreadLocal) CloningThreadLocal.of(key.getTransform().getFn()); - return threadLocal; - } - }); + fnClones = CacheBuilder.newBuilder() + .build(new CacheLoader>, DoFnLifecycleManager>() { + @Override + public DoFnLifecycleManager load(AppliedPTransform> key) + throws Exception { + return DoFnLifecycleManager.of(key.getTransform().getFn()); + } + }); } @Override public TransformEvaluator forApplication( AppliedPTransform application, CommittedBundle inputBundle, - EvaluationContext evaluationContext) { + EvaluationContext evaluationContext) throws Exception { @SuppressWarnings({"unchecked", "rawtypes"}) TransformEvaluator evaluator = createMultiEvaluator((AppliedPTransform) application, inputBundle, evaluationContext); @@ -71,38 +68,45 @@ public TransformEvaluator forApplication( } @Override - public void cleanup() { - + public void cleanup() throws Exception { + for (DoFnLifecycleManager lifecycleManager : fnClones.asMap().values()) { + lifecycleManager.removeAll(); + } } private TransformEvaluator createMultiEvaluator( AppliedPTransform, PCollectionTuple, BoundMulti> application, CommittedBundle inputBundle, - EvaluationContext evaluationContext) { + EvaluationContext evaluationContext) throws Exception { Map, PCollection> outputs = application.getOutput().getAll(); - @SuppressWarnings({"unchecked", "rawtypes"}) - ThreadLocal> fnLocal = - (ThreadLocal) fnClones.getUnchecked((AppliedPTransform) application); + DoFnLifecycleManager fnLocal = fnClones.getUnchecked((AppliedPTransform) application); String stepName = evaluationContext.getStepName(application); DirectStepContext stepContext = evaluationContext.getExecutionContext(application, inputBundle.getKey()) .getOrCreateStepContext(stepName, stepName); try { + @SuppressWarnings({"unchecked", "rawtypes"}) TransformEvaluator parDoEvaluator = ParDoEvaluator.create( evaluationContext, stepContext, inputBundle, application, - fnLocal.get(), + (OldDoFn) fnLocal.get(), application.getTransform().getSideInputs(), application.getTransform().getMainOutputTag(), application.getTransform().getSideOutputTags().getAll(), outputs); - return ThreadLocalInvalidatingTransformEvaluator.wrapping(parDoEvaluator, fnLocal); + return DoFnLifecycleManagerRemovingTransformEvaluator.wrapping(parDoEvaluator, fnLocal); } catch (Exception e) { - fnLocal.remove(); + try { + fnLocal.remove(); + } catch (Exception removalException) { + LOG.error("Exception encountered while cleaning up in ParDo evaluator construction", + removalException); + e.addSuppressed(removalException); + } throw e; } } diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoSingleEvaluatorFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoSingleEvaluatorFactory.java index 201fb46b7453d..a0fbd1d71514b 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoSingleEvaluatorFactory.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoSingleEvaluatorFactory.java @@ -31,6 +31,9 @@ import com.google.common.cache.LoadingCache; import com.google.common.collect.ImmutableMap; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + import java.util.Collections; /** @@ -38,22 +41,18 @@ * {@link Bound ParDo.Bound} primitive {@link PTransform}. */ class ParDoSingleEvaluatorFactory implements TransformEvaluatorFactory { - private final LoadingCache>, ThreadLocal>> - fnClones; + private static final Logger LOG = LoggerFactory.getLogger(ParDoSingleEvaluatorFactory.class); + private final LoadingCache>, DoFnLifecycleManager> fnClones; public ParDoSingleEvaluatorFactory() { fnClones = CacheBuilder.newBuilder() .build( - new CacheLoader< - AppliedPTransform>, ThreadLocal>>() { + new CacheLoader>, DoFnLifecycleManager>() { @Override - public ThreadLocal> load(AppliedPTransform> key) + public DoFnLifecycleManager load(AppliedPTransform> key) throws Exception { - @SuppressWarnings({"unchecked", "rawtypes"}) - ThreadLocal threadLocal = - (ThreadLocal) CloningThreadLocal.of(key.getTransform().getFn()); - return threadLocal; + return DoFnLifecycleManager.of(key.getTransform().getFn()); } }); } @@ -62,7 +61,7 @@ public ParDoSingleEvaluatorFactory() { public TransformEvaluator forApplication( final AppliedPTransform application, CommittedBundle inputBundle, - EvaluationContext evaluationContext) { + EvaluationContext evaluationContext) throws Exception { @SuppressWarnings({"unchecked", "rawtypes"}) TransformEvaluator evaluator = createSingleEvaluator((AppliedPTransform) application, inputBundle, evaluationContext); @@ -70,39 +69,45 @@ public TransformEvaluator forApplication( } @Override - public void cleanup() { - + public void cleanup() throws Exception { + for (DoFnLifecycleManager lifecycleManager : fnClones.asMap().values()) { + lifecycleManager.removeAll(); + } } private TransformEvaluator createSingleEvaluator( AppliedPTransform, PCollection, Bound> application, CommittedBundle inputBundle, - EvaluationContext evaluationContext) { + EvaluationContext evaluationContext) throws Exception { TupleTag mainOutputTag = new TupleTag<>("out"); String stepName = evaluationContext.getStepName(application); DirectStepContext stepContext = evaluationContext.getExecutionContext(application, inputBundle.getKey()) .getOrCreateStepContext(stepName, stepName); - @SuppressWarnings({"unchecked", "rawtypes"}) - ThreadLocal> fnLocal = - (ThreadLocal) fnClones.getUnchecked((AppliedPTransform) application); + DoFnLifecycleManager fnLocal = fnClones.getUnchecked((AppliedPTransform) application); try { + @SuppressWarnings({"unchecked", "rawtypes"}) ParDoEvaluator parDoEvaluator = ParDoEvaluator.create( evaluationContext, stepContext, inputBundle, application, - fnLocal.get(), + (OldDoFn) fnLocal.get(), application.getTransform().getSideInputs(), mainOutputTag, Collections.>emptyList(), ImmutableMap., PCollection>of(mainOutputTag, application.getOutput())); - return ThreadLocalInvalidatingTransformEvaluator.wrapping(parDoEvaluator, fnLocal); + return DoFnLifecycleManagerRemovingTransformEvaluator.wrapping(parDoEvaluator, fnLocal); } catch (Exception e) { - fnLocal.remove(); + try { + fnLocal.remove(); + } catch (Exception removalException) { + LOG.error("Exception encountered constructing ParDo evaluator", removalException); + e.addSuppressed(removalException); + } throw e; } } diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/CloningThreadLocalTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/CloningThreadLocalTest.java deleted file mode 100644 index 298db46bec13e..0000000000000 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/CloningThreadLocalTest.java +++ /dev/null @@ -1,92 +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.direct; - -import static org.hamcrest.Matchers.equalTo; -import static org.hamcrest.Matchers.nullValue; -import static org.hamcrest.core.IsNot.not; -import static org.hamcrest.core.IsSame.theInstance; -import static org.junit.Assert.assertThat; - -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.JUnit4; - -import java.io.Serializable; -import java.util.concurrent.Callable; -import java.util.concurrent.Executors; - -/** - * Tests for {@link CloningThreadLocalTest}. - */ -@RunWith(JUnit4.class) -public class CloningThreadLocalTest { - @Test - public void returnsCopiesOfOriginal() throws Exception { - Record original = new Record(); - ThreadLocal loaded = CloningThreadLocal.of(original); - assertThat(loaded.get(), not(nullValue())); - assertThat(loaded.get(), equalTo(original)); - assertThat(loaded.get(), not(theInstance(original))); - } - - @Test - public void returnsDifferentCopiesInDifferentThreads() throws Exception { - Record original = new Record(); - final ThreadLocal loaded = CloningThreadLocal.of(original); - assertThat(loaded.get(), not(nullValue())); - assertThat(loaded.get(), equalTo(original)); - assertThat(loaded.get(), not(theInstance(original))); - - Callable otherThread = - new Callable() { - @Override - public Record call() throws Exception { - return loaded.get(); - } - }; - Record sameThread = loaded.get(); - Record firstOtherThread = Executors.newSingleThreadExecutor().submit(otherThread).get(); - Record secondOtherThread = Executors.newSingleThreadExecutor().submit(otherThread).get(); - - assertThat(sameThread, equalTo(firstOtherThread)); - assertThat(sameThread, equalTo(secondOtherThread)); - assertThat(sameThread, not(theInstance(firstOtherThread))); - assertThat(sameThread, not(theInstance(secondOtherThread))); - assertThat(firstOtherThread, not(theInstance(secondOtherThread))); - } - - private static class Record implements Serializable { - private final double rand = Math.random(); - - @Override - public boolean equals(Object other) { - if (!(other instanceof Record)) { - return false; - } - Record that = (Record) other; - return this.rand == that.rand; - } - - @Override - public int hashCode() { - return 1; - } - } -} diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ThreadLocalInvalidatingTransformEvaluatorTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DoFnLifecycleManagerRemovingTransformEvaluatorTest.java similarity index 73% rename from runners/direct-java/src/test/java/org/apache/beam/runners/direct/ThreadLocalInvalidatingTransformEvaluatorTest.java rename to runners/direct-java/src/test/java/org/apache/beam/runners/direct/DoFnLifecycleManagerRemovingTransformEvaluatorTest.java index 6e477d37c83c4..67f4ff47789f2 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ThreadLocalInvalidatingTransformEvaluatorTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DoFnLifecycleManagerRemovingTransformEvaluatorTest.java @@ -25,8 +25,10 @@ import static org.junit.Assert.assertThat; import static org.junit.Assert.fail; +import org.apache.beam.sdk.transforms.OldDoFn; import org.apache.beam.sdk.util.WindowedValue; +import org.hamcrest.Matchers; import org.junit.Before; import org.junit.Test; import org.junit.runner.RunWith; @@ -36,24 +38,23 @@ import java.util.List; /** - * Tests for {@link ThreadLocalInvalidatingTransformEvaluator}. + * Tests for {@link DoFnLifecycleManagerRemovingTransformEvaluator}. */ @RunWith(JUnit4.class) -public class ThreadLocalInvalidatingTransformEvaluatorTest { - private ThreadLocal threadLocal; +public class DoFnLifecycleManagerRemovingTransformEvaluatorTest { + private DoFnLifecycleManager lifecycleManager; @Before public void setup() { - threadLocal = new ThreadLocal<>(); - threadLocal.set(new Object()); + lifecycleManager = DoFnLifecycleManager.of(new TestFn()); } @Test public void delegatesToUnderlying() throws Exception { RecordingTransformEvaluator underlying = new RecordingTransformEvaluator(); - Object original = threadLocal.get(); + OldDoFn original = lifecycleManager.get(); TransformEvaluator evaluator = - ThreadLocalInvalidatingTransformEvaluator.wrapping(underlying, threadLocal); + DoFnLifecycleManagerRemovingTransformEvaluator.wrapping(underlying, lifecycleManager); WindowedValue first = WindowedValue.valueInGlobalWindow(new Object()); WindowedValue second = WindowedValue.valueInGlobalWindow(new Object()); evaluator.processElement(first); @@ -66,35 +67,36 @@ public void delegatesToUnderlying() throws Exception { } @Test - public void removesOnExceptionInProcessElement() { + public void removesOnExceptionInProcessElement() throws Exception { ThrowingTransformEvaluator underlying = new ThrowingTransformEvaluator(); - Object original = threadLocal.get(); + OldDoFn original = lifecycleManager.get(); assertThat(original, not(nullValue())); TransformEvaluator evaluator = - ThreadLocalInvalidatingTransformEvaluator.wrapping(underlying, threadLocal); + DoFnLifecycleManagerRemovingTransformEvaluator.wrapping(underlying, lifecycleManager); try { evaluator.processElement(WindowedValue.valueInGlobalWindow(new Object())); } catch (Exception e) { - assertThat(threadLocal.get(), nullValue()); + assertThat(lifecycleManager.get(), not(Matchers.>theInstance(original))); return; } fail("Expected ThrowingTransformEvaluator to throw on method call"); } @Test - public void removesOnExceptionInFinishBundle() { + public void removesOnExceptionInFinishBundle() throws Exception { ThrowingTransformEvaluator underlying = new ThrowingTransformEvaluator(); - Object original = threadLocal.get(); - // the ThreadLocal is set when the evaluator starts + OldDoFn original = lifecycleManager.get(); + // the LifecycleManager is set when the evaluator starts assertThat(original, not(nullValue())); TransformEvaluator evaluator = - ThreadLocalInvalidatingTransformEvaluator.wrapping(underlying, threadLocal); + DoFnLifecycleManagerRemovingTransformEvaluator.wrapping(underlying, lifecycleManager); try { evaluator.finishBundle(); } catch (Exception e) { - assertThat(threadLocal.get(), nullValue()); + assertThat(lifecycleManager.get(), + Matchers.not(Matchers.>theInstance(original))); return; } fail("Expected ThrowingTransformEvaluator to throw on method call"); @@ -132,4 +134,11 @@ public TransformResult finishBundle() throws Exception { throw new Exception(); } } + + + private static class TestFn extends OldDoFn { + @Override + public void processElement(ProcessContext c) throws Exception { + } + } } diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DoFnLifecycleManagerTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DoFnLifecycleManagerTest.java new file mode 100644 index 0000000000000..f316e19d48791 --- /dev/null +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DoFnLifecycleManagerTest.java @@ -0,0 +1,119 @@ +/* + * 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.equalTo; +import static org.hamcrest.Matchers.not; +import static org.hamcrest.Matchers.theInstance; +import static org.junit.Assert.assertThat; + +import org.apache.beam.sdk.transforms.OldDoFn; + +import org.hamcrest.Matchers; +import org.junit.Test; + +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.Callable; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.Future; +import java.util.concurrent.TimeUnit; + +/** + * Tests for {@link DoFnLifecycleManager}. + */ +public class DoFnLifecycleManagerTest { + private TestFn fn = new TestFn(); + private DoFnLifecycleManager mgr = DoFnLifecycleManager.of(fn); + + @Test + public void setupOnGet() throws Exception { + TestFn obtained = (TestFn) mgr.get(); + + assertThat(obtained, not(theInstance(fn))); + } + + @Test + public void getMultipleCallsSingleSetupCall() throws Exception { + TestFn obtained = (TestFn) mgr.get(); + TestFn secondObtained = (TestFn) mgr.get(); + + assertThat(obtained, theInstance(secondObtained)); + } + + @Test + public void getMultipleThreadsDifferentInstances() throws Exception { + CountDownLatch startSignal = new CountDownLatch(1); + ExecutorService executor = Executors.newCachedThreadPool(); + List> futures = new ArrayList<>(); + for (int i = 0; i < 10; i++) { + futures.add(executor.submit(new GetFnCallable(mgr, startSignal))); + } + startSignal.countDown(); + List fns = new ArrayList<>(); + for (Future future : futures) { + fns.add(future.get(1L, TimeUnit.SECONDS)); + } + + for (TestFn fn : fns) { + int sameInstances = 0; + for (TestFn otherFn : fns) { + if (otherFn == fn) { + sameInstances++; + } + } + assertThat(sameInstances, equalTo(1)); + } + } + + @Test + public void teardownOnRemove() throws Exception { + TestFn obtained = (TestFn) mgr.get(); + mgr.remove(); + + assertThat(obtained, not(theInstance(fn))); + + assertThat(mgr.get(), not(Matchers.>theInstance(obtained))); + } + + private static class GetFnCallable implements Callable { + private final DoFnLifecycleManager mgr; + private final CountDownLatch startSignal; + + private GetFnCallable(DoFnLifecycleManager mgr, CountDownLatch startSignal) { + this.mgr = mgr; + this.startSignal = startSignal; + } + + @Override + public TestFn call() throws Exception { + startSignal.await(); + return (TestFn) mgr.get(); + } + } + + + private static class TestFn extends OldDoFn { + @Override + public void processElement(ProcessContext c) throws Exception { + } + } +} From d056f4661da2cc399cab44c6604eaa61d1dfd178 Mon Sep 17 00:00:00 2001 From: Thomas Groh Date: Thu, 14 Jul 2016 14:51:02 -0700 Subject: [PATCH 024/346] Add DoFn @Setup and @Teardown Methods annotated with these annotations are used to perform expensive setup work and clean up a DoFn after another method throws an exception or the DoFn is discarded. --- .../runners/direct/DoFnLifecycleManager.java | 38 +- ...ycleManagerRemovingTransformEvaluator.java | 39 +- .../runners/direct/DoFnLifecycleManagers.java | 45 ++ .../direct/ParDoMultiEvaluatorFactory.java | 4 +- .../direct/ParDoSingleEvaluatorFactory.java | 4 +- .../direct/DoFnLifecycleManagerTest.java | 49 ++ .../direct/DoFnLifecycleManagersTest.java | 142 ++++++ .../functions/FlinkDoFnFunction.java | 12 +- .../FlinkMultiOutputDoFnFunction.java | 31 +- .../streaming/FlinkAbstractParDoWrapper.java | 2 + .../FlinkGroupAlsoByWindowWrapper.java | 2 + .../spark/translation/DoFnFunction.java | 23 +- .../spark/translation/MultiDoFnFunction.java | 1 + .../translation/SparkProcessContext.java | 17 + .../org/apache/beam/sdk/transforms/DoFn.java | 31 +- .../beam/sdk/transforms/DoFnReflector.java | 70 ++- .../apache/beam/sdk/transforms/OldDoFn.java | 25 ++ .../org/apache/beam/sdk/transforms/ParDo.java | 15 +- .../sdk/transforms/DoFnReflectorTest.java | 65 +++ .../apache/beam/sdk/transforms/ParDoTest.java | 420 +++++++++++++++++- 20 files changed, 970 insertions(+), 65 deletions(-) create mode 100644 runners/direct-java/src/main/java/org/apache/beam/runners/direct/DoFnLifecycleManagers.java create mode 100644 runners/direct-java/src/test/java/org/apache/beam/runners/direct/DoFnLifecycleManagersTest.java 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 27836577b3a7d..3f4f2c6a880d6 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 @@ -18,6 +18,7 @@ package org.apache.beam.runners.direct; +import org.apache.beam.sdk.runners.PipelineRunner; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.OldDoFn; import org.apache.beam.sdk.util.SerializableUtils; @@ -26,6 +27,13 @@ import com.google.common.cache.CacheLoader; import com.google.common.cache.LoadingCache; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.Iterator; + /** * Manages {@link DoFn} setup, teardown, and serialization. * @@ -35,6 +43,8 @@ * {@link DoFn DoFns}. */ class DoFnLifecycleManager { + private static final Logger LOG = LoggerFactory.getLogger(DoFnLifecycleManager.class); + public static DoFnLifecycleManager of(OldDoFn original) { return new DoFnLifecycleManager(original); } @@ -52,14 +62,30 @@ private DoFnLifecycleManager(OldDoFn original) { public void remove() throws Exception { Thread currentThread = Thread.currentThread(); - outstanding.invalidate(currentThread); + OldDoFn fn = outstanding.asMap().remove(currentThread); + fn.teardown(); } /** - * Remove all {@link DoFn DoFns} from this {@link DoFnLifecycleManager}. + * Remove all {@link DoFn DoFns} from this {@link DoFnLifecycleManager}. Returns all exceptions + * that were thrown while calling the remove methods. + * + *

If the returned Collection is nonempty, an exception was thrown from at least one + * {@link DoFn#teardown()} method, and the {@link PipelineRunner} should throw an exception. */ - public void removeAll() throws Exception { - outstanding.invalidateAll(); + public Collection removeAll() throws Exception { + Iterator> fns = outstanding.asMap().values().iterator(); + Collection thrown = new ArrayList<>(); + while (fns.hasNext()) { + OldDoFn fn = fns.next(); + fns.remove(); + try { + fn.teardown(); + } catch (Exception e) { + thrown.add(e); + } + } + return thrown; } private class DeserializingCacheLoader extends CacheLoader> { @@ -71,8 +97,10 @@ public DeserializingCacheLoader(OldDoFn original) { @Override public OldDoFn load(Thread key) throws Exception { - return (OldDoFn) SerializableUtils.deserializeFromByteArray(original, + OldDoFn fn = (OldDoFn) SerializableUtils.deserializeFromByteArray(original, "DoFn Copy in thread " + key.getName()); + fn.setup(); + return fn; } } } 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 f3d1d4f3d5842..523273cdd1ecd 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 @@ -34,14 +34,14 @@ class DoFnLifecycleManagerRemovingTransformEvaluator implements Transfor private final DoFnLifecycleManager lifecycleManager; public static TransformEvaluator wrapping( - TransformEvaluator underlying, DoFnLifecycleManager threadLocal) { - return new DoFnLifecycleManagerRemovingTransformEvaluator<>(underlying, threadLocal); + TransformEvaluator underlying, DoFnLifecycleManager lifecycleManager) { + return new DoFnLifecycleManagerRemovingTransformEvaluator<>(underlying, lifecycleManager); } private DoFnLifecycleManagerRemovingTransformEvaluator( - TransformEvaluator underlying, DoFnLifecycleManager threadLocal) { + TransformEvaluator underlying, DoFnLifecycleManager lifecycleManager) { this.underlying = underlying; - this.lifecycleManager = threadLocal; + this.lifecycleManager = lifecycleManager; } @Override @@ -49,14 +49,7 @@ public void processElement(WindowedValue element) throws Exception { try { underlying.processElement(element); } catch (Exception e) { - try { - lifecycleManager.remove(); - } catch (Exception removalException) { - LOG.error( - "Exception encountered while cleaning up after processing an element", - removalException); - e.addSuppressed(removalException); - } + onException(e, "Exception encountered while cleaning up after processing an element"); throw e; } } @@ -66,15 +59,21 @@ public TransformResult finishBundle() throws Exception { try { return underlying.finishBundle(); } catch (Exception e) { - try { - lifecycleManager.remove(); - } catch (Exception removalException) { - LOG.error( - "Exception encountered while cleaning up after finishing a bundle", - removalException); - e.addSuppressed(removalException); - } + onException(e, "Exception encountered while cleaning up after finishing a bundle"); throw e; } } + + private void onException(Exception e, String msg) { + try { + lifecycleManager.remove(); + } catch (Exception removalException) { + if (removalException instanceof InterruptedException) { + Thread.currentThread().interrupt(); + } + LOG.error(msg, removalException); + e.addSuppressed(removalException); + } + } + } diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DoFnLifecycleManagers.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DoFnLifecycleManagers.java new file mode 100644 index 0000000000000..6a1dd8f24fa11 --- /dev/null +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DoFnLifecycleManagers.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.runners.direct; + +import java.util.ArrayList; +import java.util.Collection; + +/** + * Utility methods for interacting with {@link DoFnLifecycleManager DoFnLifecycleManagers}. + */ +class DoFnLifecycleManagers { + private DoFnLifecycleManagers() { + /* Do not instantiate */ + } + + static void removeAllFromManagers(Iterable managers) throws Exception { + Collection thrown = new ArrayList<>(); + for (DoFnLifecycleManager manager : managers) { + thrown.addAll(manager.removeAll()); + } + if (!thrown.isEmpty()) { + Exception overallException = new Exception("Exceptions thrown while tearing down DoFns"); + for (Exception e : thrown) { + overallException.addSuppressed(e); + } + throw overallException; + } + } +} diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoMultiEvaluatorFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoMultiEvaluatorFactory.java index f2455e1f069e8..2d05e68b66595 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoMultiEvaluatorFactory.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoMultiEvaluatorFactory.java @@ -69,9 +69,7 @@ public TransformEvaluator forApplication( @Override public void cleanup() throws Exception { - for (DoFnLifecycleManager lifecycleManager : fnClones.asMap().values()) { - lifecycleManager.removeAll(); - } + DoFnLifecycleManagers.removeAllFromManagers(fnClones.asMap().values()); } private TransformEvaluator createMultiEvaluator( diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoSingleEvaluatorFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoSingleEvaluatorFactory.java index a0fbd1d71514b..97cbfa7f7b2ff 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoSingleEvaluatorFactory.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoSingleEvaluatorFactory.java @@ -70,9 +70,7 @@ public TransformEvaluator forApplication( @Override public void cleanup() throws Exception { - for (DoFnLifecycleManager lifecycleManager : fnClones.asMap().values()) { - lifecycleManager.removeAll(); - } + DoFnLifecycleManagers.removeAllFromManagers(fnClones.asMap().values()); } private TransformEvaluator createSingleEvaluator( diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DoFnLifecycleManagerTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DoFnLifecycleManagerTest.java index f316e19d48791..77b32968f8b4f 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DoFnLifecycleManagerTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DoFnLifecycleManagerTest.java @@ -18,7 +18,9 @@ package org.apache.beam.runners.direct; +import static com.google.common.base.Preconditions.checkState; import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.is; import static org.hamcrest.Matchers.not; import static org.hamcrest.Matchers.theInstance; import static org.junit.Assert.assertThat; @@ -49,6 +51,8 @@ public void setupOnGet() throws Exception { TestFn obtained = (TestFn) mgr.get(); assertThat(obtained, not(theInstance(fn))); + assertThat(obtained.setupCalled, is(true)); + assertThat(obtained.teardownCalled, is(false)); } @Test @@ -57,6 +61,8 @@ public void getMultipleCallsSingleSetupCall() throws Exception { TestFn secondObtained = (TestFn) mgr.get(); assertThat(obtained, theInstance(secondObtained)); + assertThat(obtained.setupCalled, is(true)); + assertThat(obtained.teardownCalled, is(false)); } @Test @@ -74,6 +80,7 @@ public void getMultipleThreadsDifferentInstances() throws Exception { } for (TestFn fn : fns) { + assertThat(fn.setupCalled, is(true)); int sameInstances = 0; for (TestFn otherFn : fns) { if (otherFn == fn) { @@ -90,10 +97,33 @@ public void teardownOnRemove() throws Exception { mgr.remove(); assertThat(obtained, not(theInstance(fn))); + assertThat(obtained.setupCalled, is(true)); + assertThat(obtained.teardownCalled, is(true)); assertThat(mgr.get(), not(Matchers.>theInstance(obtained))); } + @Test + public void teardownAllOnRemoveAll() throws Exception { + CountDownLatch startSignal = new CountDownLatch(1); + ExecutorService executor = Executors.newCachedThreadPool(); + List> futures = new ArrayList<>(); + for (int i = 0; i < 10; i++) { + futures.add(executor.submit(new GetFnCallable(mgr, startSignal))); + } + startSignal.countDown(); + List fns = new ArrayList<>(); + for (Future future : futures) { + fns.add(future.get(1L, TimeUnit.SECONDS)); + } + mgr.removeAll(); + + for (TestFn fn : fns) { + assertThat(fn.setupCalled, is(true)); + assertThat(fn.teardownCalled, is(true)); + } + } + private static class GetFnCallable implements Callable { private final DoFnLifecycleManager mgr; private final CountDownLatch startSignal; @@ -112,8 +142,27 @@ public TestFn call() throws Exception { private static class TestFn extends OldDoFn { + boolean setupCalled = false; + boolean teardownCalled = false; + + @Override + public void setup() { + checkState(!setupCalled); + checkState(!teardownCalled); + + setupCalled = true; + } + @Override public void processElement(ProcessContext c) throws Exception { } + + @Override + public void teardown() { + checkState(setupCalled); + checkState(!teardownCalled); + + teardownCalled = true; + } } } diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DoFnLifecycleManagersTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DoFnLifecycleManagersTest.java new file mode 100644 index 0000000000000..8be3d52763be2 --- /dev/null +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DoFnLifecycleManagersTest.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.runners.direct; + +import static org.hamcrest.Matchers.equalTo; + +import org.apache.beam.sdk.transforms.OldDoFn; + +import com.google.common.collect.ImmutableList; + +import org.hamcrest.BaseMatcher; +import org.hamcrest.Description; +import org.hamcrest.Matcher; +import org.hamcrest.Matchers; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.ExpectedException; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +import java.util.ArrayList; +import java.util.Collection; + +/** + * Tests for {@link DoFnLifecycleManagers}. + */ +@RunWith(JUnit4.class) +public class DoFnLifecycleManagersTest { + @Rule public ExpectedException thrown = ExpectedException.none(); + + @Test + public void removeAllWhenManagersThrowSuppressesAndThrows() throws Exception { + DoFnLifecycleManager first = DoFnLifecycleManager.of(new ThrowsInCleanupFn("foo")); + DoFnLifecycleManager second = DoFnLifecycleManager.of(new ThrowsInCleanupFn("bar")); + DoFnLifecycleManager third = DoFnLifecycleManager.of(new ThrowsInCleanupFn("baz")); + first.get(); + second.get(); + third.get(); + + final Collection> suppressions = new ArrayList<>(); + suppressions.add(new ThrowableMessageMatcher("foo")); + suppressions.add(new ThrowableMessageMatcher("bar")); + suppressions.add(new ThrowableMessageMatcher("baz")); + + thrown.expect( + new BaseMatcher() { + @Override + public void describeTo(Description description) { + description + .appendText("Exception suppressing ") + .appendList("[", ", ", "]", suppressions); + } + + @Override + public boolean matches(Object item) { + if (!(item instanceof Exception)) { + return false; + } + Exception that = (Exception) item; + return Matchers.containsInAnyOrder(suppressions) + .matches(ImmutableList.copyOf(that.getSuppressed())); + } + }); + + DoFnLifecycleManagers.removeAllFromManagers(ImmutableList.of(first, second, third)); + } + + @Test + public void whenManagersSucceedSucceeds() throws Exception { + DoFnLifecycleManager first = DoFnLifecycleManager.of(new EmptyFn()); + DoFnLifecycleManager second = DoFnLifecycleManager.of(new EmptyFn()); + DoFnLifecycleManager third = DoFnLifecycleManager.of(new EmptyFn()); + first.get(); + second.get(); + third.get(); + + DoFnLifecycleManagers.removeAllFromManagers(ImmutableList.of(first, second, third)); + } + + private static class ThrowsInCleanupFn extends OldDoFn { + private final String message; + + private ThrowsInCleanupFn(String message) { + this.message = message; + } + + @Override + public void processElement(ProcessContext c) throws Exception { + } + + @Override + public void teardown() throws Exception { + throw new Exception(message); + } + } + + + private static class ThrowableMessageMatcher extends BaseMatcher { + private final Matcher messageMatcher; + + public ThrowableMessageMatcher(String message) { + this.messageMatcher = equalTo(message); + } + + @Override + public boolean matches(Object item) { + if (!(item instanceof Throwable)) { + return false; + } + Throwable that = (Throwable) item; + return messageMatcher.matches(that.getMessage()); + } + + @Override + public void describeTo(Description description) { + description.appendText("a throwable with a message ").appendDescriptionOf(messageMatcher); + } + } + + + private static class EmptyFn extends OldDoFn { + @Override + public void processElement(ProcessContext c) throws Exception { + } + } +} 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 a4af1b025a427..fdf1e59e06d39 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 @@ -25,6 +25,7 @@ import org.apache.beam.sdk.values.PCollectionView; import org.apache.flink.api.common.functions.RichMapPartitionFunction; +import org.apache.flink.configuration.Configuration; import org.apache.flink.util.Collector; import java.util.Map; @@ -86,7 +87,7 @@ public void mapPartition( // side inputs and window access also only works if an element // is in only one window for (WindowedValue value : values) { - for (WindowedValue explodedValue: value.explodeWindows()) { + for (WindowedValue explodedValue : value.explodeWindows()) { context = context.forWindowedValue(value); doFn.processElement(context); } @@ -99,4 +100,13 @@ public void mapPartition( this.doFn.finishBundle(context); } + @Override + public void open(Configuration parameters) throws Exception { + doFn.setup(); + } + + @Override + public void close() throws Exception { + doFn.teardown(); + } } 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 6e673fc1df95a..5013b90d22d5a 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 @@ -27,6 +27,7 @@ import org.apache.beam.sdk.values.TupleTag; import org.apache.flink.api.common.functions.RichMapPartitionFunction; +import org.apache.flink.configuration.Configuration; import org.apache.flink.util.Collector; import java.util.Map; @@ -75,14 +76,15 @@ public void mapPartition( Iterable> values, Collector> out) throws Exception { - FlinkProcessContext context = new FlinkMultiOutputProcessContext<>( - serializedOptions.getPipelineOptions(), - getRuntimeContext(), - doFn, - windowingStrategy, - out, - outputMap, - sideInputs); + FlinkProcessContext context = + new FlinkMultiOutputProcessContext<>( + serializedOptions.getPipelineOptions(), + getRuntimeContext(), + doFn, + windowingStrategy, + out, + outputMap, + sideInputs); this.doFn.startBundle(context); @@ -97,14 +99,23 @@ public void mapPartition( // side inputs and window access also only works if an element // is in only one window for (WindowedValue value : values) { - for (WindowedValue explodedValue: value.explodeWindows()) { + for (WindowedValue explodedValue : value.explodeWindows()) { context = context.forWindowedValue(value); doFn.processElement(context); } } } - this.doFn.finishBundle(context); } + + @Override + public void open(Configuration parameters) throws Exception { + doFn.setup(); + } + + @Override + public void close() throws Exception { + doFn.teardown(); + } } diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/FlinkAbstractParDoWrapper.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/FlinkAbstractParDoWrapper.java index 74ec66a3155bf..a9dd865bf2b3c 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/FlinkAbstractParDoWrapper.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/FlinkAbstractParDoWrapper.java @@ -70,6 +70,7 @@ public FlinkAbstractParDoWrapper(PipelineOptions options, WindowingStrategy(output), this.timerInternals); operator.startBundle(context); } @@ -351,6 +352,7 @@ public void processWatermark(Watermark mark) throws Exception { @Override public void close() throws Exception { operator.finishBundle(context); + operator.teardown(); super.close(); } diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/DoFnFunction.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/DoFnFunction.java index f4ce51633362f..c08d1850d4127 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/DoFnFunction.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/DoFnFunction.java @@ -24,6 +24,8 @@ import org.apache.beam.sdk.values.TupleTag; import org.apache.spark.api.java.function.FlatMapFunction; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import java.util.Iterator; import java.util.LinkedList; @@ -40,6 +42,8 @@ public class DoFnFunction implements FlatMapFunction>, WindowedValue> { private final OldDoFn mFunction; + private static final Logger LOG = LoggerFactory.getLogger(DoFnFunction.class); + private final SparkRuntimeContext mRuntimeContext; private final Map, BroadcastHelper> mSideInputs; @@ -61,8 +65,23 @@ public Iterable> call(Iterator> ite Exception { ProcCtxt ctxt = new ProcCtxt(mFunction, mRuntimeContext, mSideInputs); ctxt.setup(); - mFunction.startBundle(ctxt); - return ctxt.getOutputIterable(iter, mFunction); + try { + mFunction.setup(); + mFunction.startBundle(ctxt); + return ctxt.getOutputIterable(iter, mFunction); + } catch (Exception e) { + try { + // this teardown handles exceptions encountered in setup() and startBundle(). teardown + // after execution or due to exceptions in process element is called in the iterator + // produced by ctxt.getOutputIterable returned from this method. + mFunction.teardown(); + } catch (Exception teardownException) { + LOG.error( + "Suppressing exception while tearing down Function {}", mFunction, teardownException); + e.addSuppressed(teardownException); + } + throw e; + } } private class ProcCtxt extends SparkProcessContext> { diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/MultiDoFnFunction.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/MultiDoFnFunction.java index e33578df83d51..abf0e8379a2ee 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/MultiDoFnFunction.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/MultiDoFnFunction.java @@ -65,6 +65,7 @@ class MultiDoFnFunction public Iterable, WindowedValue>> call(Iterator> iter) throws Exception { ProcCtxt ctxt = new ProcCtxt(mFunction, mRuntimeContext, mSideInputs); + mFunction.setup(); mFunction.startBundle(ctxt); ctxt.setup(); return ctxt.getOutputIterable(iter, mFunction); 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 2f06a1c8508ac..1cdbd92010d8a 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 @@ -238,6 +238,7 @@ protected ValueT computeNext() { try { doFn.processElement(SparkProcessContext.this); } catch (Exception e) { + handleProcessingException(e); throw new SparkProcessException(e); } outputIterator = getOutputIterator(); @@ -249,15 +250,31 @@ protected ValueT computeNext() { calledFinish = true; doFn.finishBundle(SparkProcessContext.this); } catch (Exception e) { + handleProcessingException(e); throw new SparkProcessException(e); } outputIterator = getOutputIterator(); continue; // try to consume outputIterator from start of loop } + try { + doFn.teardown(); + } catch (Exception e) { + LOG.error( + "Suppressing teardown exception that occurred after processing entire input", e); + } return endOfData(); } } } + + private void handleProcessingException(Exception e) { + try { + doFn.teardown(); + } catch (Exception e1) { + LOG.error("Exception while cleaning up DoFn", e1); + e.addSuppressed(e1); + } + } } /** 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 a06467e9b381a..80b67afc8412b 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 @@ -342,6 +342,20 @@ public OutputReceiver outputReceiver() { ///////////////////////////////////////////////////////////////////////////// + + /** + * Annotation for the method to use to prepare an instance for processing bundles of elements. The + * method annotated with this must satisfy the following constraints + *

    + *
  • It must have zero arguments. + *
+ */ + @Documented + @Retention(RetentionPolicy.RUNTIME) + @Target(ElementType.METHOD) + public @interface Setup { + } + /** * Annotation for the method to use to prepare an instance for processing a batch of elements. * The method annotated with this must satisfy the following constraints: @@ -371,7 +385,7 @@ public OutputReceiver outputReceiver() { public @interface ProcessElement {} /** - * Annotation for the method to use to prepare an instance for processing a batch of elements. + * Annotation for the method to use to finish processing a batch of elements. * The method annotated with this must satisfy the following constraints: *
    *
  • It must have at least one argument. @@ -383,6 +397,21 @@ public OutputReceiver outputReceiver() { @Target(ElementType.METHOD) public @interface FinishBundle {} + + /** + * Annotation for the method to use to clean up this instance after processing bundles of + * elements. No other method will be called after a call to the annotated method is made. + * The method annotated with this must satisfy the following constraint: + *
      + *
    • It must have zero arguments. + *
    + */ + @Documented + @Retention(RetentionPolicy.RUNTIME) + @Target(ElementType.METHOD) + public @interface Teardown { + } + /** * Returns an {@link Aggregator} with aggregation logic specified by the * {@link CombineFn} argument. The name provided must be unique across diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnReflector.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnReflector.java index 3dfda55caed40..bf040412b5e0a 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnReflector.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnReflector.java @@ -17,11 +17,15 @@ */ package org.apache.beam.sdk.transforms; +import static com.google.common.base.Preconditions.checkState; + import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.DoFn.ExtraContextFactory; import org.apache.beam.sdk.transforms.DoFn.FinishBundle; import org.apache.beam.sdk.transforms.DoFn.ProcessElement; +import org.apache.beam.sdk.transforms.DoFn.Setup; import org.apache.beam.sdk.transforms.DoFn.StartBundle; +import org.apache.beam.sdk.transforms.DoFn.Teardown; import org.apache.beam.sdk.transforms.display.DisplayData; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; @@ -302,6 +306,15 @@ static List verifyBundleMethodArguments(M new TypeParameter() {}); } + @VisibleForTesting + static void verifyLifecycleMethodArguments(Method m) { + if (m == null) { + return; + } + checkState(void.class.equals(m.getReturnType()), "%s must have void return type", format(m)); + checkState(m.getGenericParameterTypes().length == 0, "%s must take zero arguments", format(m)); + } + /** * Verify the method arguments for a given {@link DoFn} method. * @@ -392,6 +405,8 @@ static List verifyMethodArguments( /** Interface for invoking the {@code OldDoFn} processing methods. */ public interface DoFnInvoker { + /** Invoke {@link OldDoFn#setup} on the bound {@code OldDoFn}. */ + void invokeSetup(); /** Invoke {@link OldDoFn#startBundle} on the bound {@code OldDoFn}. */ void invokeStartBundle( DoFn.Context c, @@ -401,6 +416,9 @@ void invokeFinishBundle( DoFn.Context c, ExtraContextFactory extra); + /** Invoke {@link OldDoFn#teardown()} on the bound {@code DoFn}. */ + void invokeTeardown(); + /** Invoke {@link OldDoFn#processElement} on the bound {@code OldDoFn}. */ public void invokeProcessElement( DoFn.ProcessContext c, @@ -412,9 +430,11 @@ public void invokeProcessElement( */ private static class GenericDoFnReflector extends DoFnReflector { + private final Method setup; private final Method startBundle; private final Method processElement; private final Method finishBundle; + private final Method teardown; private final List processElementArgs; private final List startBundleArgs; private final List finishBundleArgs; @@ -424,13 +444,17 @@ private GenericDoFnReflector( @SuppressWarnings("rawtypes") Class fn) { // Locate the annotated methods this.processElement = findAnnotatedMethod(ProcessElement.class, fn, true); + this.setup = findAnnotatedMethod(Setup.class, fn, false); this.startBundle = findAnnotatedMethod(StartBundle.class, fn, false); this.finishBundle = findAnnotatedMethod(FinishBundle.class, fn, false); + this.teardown = findAnnotatedMethod(Teardown.class, fn, false); // Verify that their method arguments satisfy our conditions. this.processElementArgs = verifyProcessMethodArguments(processElement); this.startBundleArgs = verifyBundleMethodArguments(startBundle); this.finishBundleArgs = verifyBundleMethodArguments(finishBundle); + verifyLifecycleMethodArguments(setup); + verifyLifecycleMethodArguments(teardown); this.constructor = createInvokerConstructor(fn); } @@ -552,8 +576,17 @@ public String subclass(Generic superClass) { .intercept(InvokerDelegation.create( startBundle, BeforeDelegation.INVOKE_PREPARE_FOR_PROCESSING, startBundleArgs)) .method(ElementMatchers.named("invokeFinishBundle")) - .intercept(InvokerDelegation.create( - finishBundle, BeforeDelegation.NOOP, finishBundleArgs)); + .intercept(InvokerDelegation.create(finishBundle, + BeforeDelegation.NOOP, + finishBundleArgs)) + .method(ElementMatchers.named("invokeSetup")) + .intercept(InvokerDelegation.create(setup, + BeforeDelegation.NOOP, + Collections.emptyList())) + .method(ElementMatchers.named("invokeTeardown")) + .intercept(InvokerDelegation.create(teardown, + BeforeDelegation.NOOP, + Collections.emptyList())); @SuppressWarnings("unchecked") Class> dynamicClass = (Class>) builder @@ -735,6 +768,11 @@ private SimpleDoFnAdapter(DoFnReflector reflector, DoFn fn) { this.invoker = reflector.bindInvoker(fn); } + @Override + public void setup() throws Exception { + invoker.invokeSetup(); + } + @Override public void startBundle(OldDoFn.Context c) throws Exception { ContextAdapter adapter = new ContextAdapter<>(fn, c); @@ -747,6 +785,11 @@ public void finishBundle(OldDoFn.Context c) throws Exception { invoker.invokeFinishBundle(adapter, adapter); } + @Override + public void teardown() { + invoker.invokeTeardown(); + } + @Override public void processElement(OldDoFn.ProcessContext c) throws Exception { ProcessContextAdapter adapter = new ProcessContextAdapter<>(fn, c); @@ -940,15 +983,20 @@ private StackManipulation invokeTargetMethod(MethodDescription instrumentedMetho new MethodDescription.ForLoadedMethod(target)).resolve(instrumentedMethod); ParameterList params = targetMethod.getParameters(); - // Instructions to setup the parameters for the call - ArrayList parameters = new ArrayList<>(args.size() + 1); - // 1. The first argument in the delegate method must be the context. This corresponds to - // the first argument in the instrumented method, so copy that. - parameters.add(MethodVariableAccess.of( - params.get(0).getType().getSuperClass()).loadOffset(1)); - // 2. For each of the extra arguments push the appropriate value. - for (AdditionalParameter arg : args) { - parameters.add(pushArgument(arg, instrumentedMethod)); + List parameters; + if (!params.isEmpty()) { + // Instructions to setup the parameters for the call + parameters = new ArrayList<>(args.size() + 1); + // 1. The first argument in the delegate method must be the context. This corresponds to + // the first argument in the instrumented method, so copy that. + parameters.add(MethodVariableAccess.of(params.get(0).getType().getSuperClass()) + .loadOffset(1)); + // 2. For each of the extra arguments push the appropriate value. + for (AdditionalParameter arg : args) { + parameters.add(pushArgument(arg, instrumentedMethod)); + } + } else { + parameters = Collections.emptyList(); } return new StackManipulation.Compound( 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 443599ac282a6..84cd9978c6212 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 @@ -338,6 +338,17 @@ public OldDoFn() { */ private boolean aggregatorsAreFinal; + /** + * Prepares this {@link DoFn} instance for processing bundles. + * + *

    {@link #setup()} will be called at most once per {@link DoFn} instance, and before any other + * {@link DoFn} method is called. + * + *

    By default, does nothing. + */ + public void setup() throws Exception { + } + /** * Prepares this {@code OldDoFn} instance for processing a batch of elements. * @@ -372,6 +383,20 @@ public void startBundle(Context c) throws Exception { public void finishBundle(Context c) throws Exception { } + /** + * Cleans up this {@link DoFn}. + * + *

    {@link #teardown()} will be called before the {@link PipelineRunner} discards a {@link DoFn} + * instance, including due to another {@link DoFn} method throwing an {@link Exception}. No other + * {@link DoFn} methods will be called after a call to {@link #teardown()}. + * + *

    By default, does nothing. + */ + public void teardown() throws Exception { + } + + ///////////////////////////////////////////////////////////////////////////// + /** * {@inheritDoc} * 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 ca6d9b2e0031b..aa5753197b1d0 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 @@ -67,11 +67,11 @@ * For each bundle of input elements processing proceeds as follows: * *

      - *
    1. If required, a fresh instance of the argument {@link OldDoFn} is created - * on a worker. This may be through deserialization or other means. A - * {@link PipelineRunner} may reuse {@link OldDoFn} instances for multiple bundles. - * A {@link OldDoFn} that has terminated abnormally (by throwing an {@link Exception} - * will never be reused.
    2. + *
    3. If required, a fresh instance of the argument {@link DoFn} is created + * on a worker, and {@link DoFn#setup()} is called on this instance. This may be through + * deserialization or other means. A {@link PipelineRunner} may reuse {@link DoFn} instances for + * multiple bundles. A {@link DoFn} that has terminated abnormally (by throwing an + * {@link Exception}) will never be reused.
    4. *
    5. The {@link OldDoFn OldDoFn's} {@link OldDoFn#startBundle} method is called to * initialize it. If this method is not overridden, the call may be optimized * away.
    6. @@ -83,6 +83,11 @@ * {@link OldDoFn#finishBundle} * until a new call to {@link OldDoFn#startBundle} has occurred. * If this method is not overridden, this call may be optimized away. + *
    7. If any of {@link DoFn#setup}, {@link DoFn#startBundle}, {@link DoFn#processElement} or + * {@link DoFn#finishBundle} throw an exception, {@link DoFn#teardown} will be called on the + * {@link DoFn} instance.
    8. + *
    9. If a runner will no longer use a {@link DoFn}, {@link DoFn#teardown()} will be called on + * the discarded instance.
    10. *
    * * Each of the calls to any of the {@link OldDoFn OldDoFn's} processing diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/DoFnReflectorTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/DoFnReflectorTest.java index c47e0cf5a9d84..e05e5e2c667a3 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/DoFnReflectorTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/DoFnReflectorTest.java @@ -25,6 +25,8 @@ import org.apache.beam.sdk.transforms.DoFn.ExtraContextFactory; import org.apache.beam.sdk.transforms.DoFn.ProcessContext; import org.apache.beam.sdk.transforms.DoFn.ProcessElement; +import org.apache.beam.sdk.transforms.DoFn.Setup; +import org.apache.beam.sdk.transforms.DoFn.Teardown; import org.apache.beam.sdk.transforms.dofnreflector.DoFnReflectorTestHelper; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.util.UserCodeException; @@ -53,6 +55,8 @@ public static class Invocations { public boolean wasProcessElementInvoked = false; public boolean wasStartBundleInvoked = false; public boolean wasFinishBundleInvoked = false; + public boolean wasSetupInvoked = false; + public boolean wasTeardownInvoked = false; private final String name; public Invocations(String name) { @@ -144,6 +148,33 @@ private void checkInvokeFinishBundleWorks( } } + private void checkInvokeSetupWorks(DoFnReflector r, Invocations... invocations) throws Exception { + assertTrue("Need at least one invocation to check", invocations.length >= 1); + for (Invocations invocation : invocations) { + assertFalse("Should not yet have called setup on " + invocation.name, + invocation.wasSetupInvoked); + } + r.bindInvoker(fn).invokeSetup(); + for (Invocations invocation : invocations) { + assertTrue("Should have called setup on " + invocation.name, + invocation.wasSetupInvoked); + } + } + + private void checkInvokeTeardownWorks(DoFnReflector r, Invocations... invocations) + throws Exception { + assertTrue("Need at least one invocation to check", invocations.length >= 1); + for (Invocations invocation : invocations) { + assertFalse("Should not yet have called teardown on " + invocation.name, + invocation.wasTeardownInvoked); + } + r.bindInvoker(fn).invokeTeardown(); + for (Invocations invocation : invocations) { + assertTrue("Should have called teardown on " + invocation.name, + invocation.wasTeardownInvoked); + } + } + @Test public void testDoFnWithNoExtraContext() throws Exception { final Invocations invocations = new Invocations("AnonymousClass"); @@ -324,6 +355,40 @@ public void finishBundle(Context c) { checkInvokeFinishBundleWorks(reflector, invocations); } + @Test + public void testDoFnWithSetupTeardown() throws Exception { + final Invocations invocations = new Invocations("AnonymousClass"); + DoFnReflector reflector = underTest(new DoFn() { + @ProcessElement + public void processElement(@SuppressWarnings("unused") ProcessContext c) {} + + @StartBundle + public void startBundle(Context c) { + invocations.wasStartBundleInvoked = true; + assertSame(c, mockContext); + } + + @FinishBundle + public void finishBundle(Context c) { + invocations.wasFinishBundleInvoked = true; + assertSame(c, mockContext); + } + + @Setup + public void before() { + invocations.wasSetupInvoked = true; + } + + @Teardown + public void after() { + invocations.wasTeardownInvoked = true; + } + }); + + checkInvokeSetupWorks(reflector, invocations); + checkInvokeTeardownWorks(reflector, invocations); + } + @Test public void testNoProcessElement() throws Exception { thrown.expect(IllegalStateException.class); 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 7fe053c29eccf..846012486cade 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 @@ -24,17 +24,18 @@ import static org.apache.beam.sdk.util.SerializableUtils.serializeToByteArray; import static org.apache.beam.sdk.util.StringUtils.byteArrayToJsonString; import static org.apache.beam.sdk.util.StringUtils.jsonStringToByteArray; - import static com.google.common.base.Preconditions.checkNotNull; - import static org.hamcrest.Matchers.allOf; import static org.hamcrest.Matchers.anyOf; import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.greaterThan; +import static org.hamcrest.Matchers.is; import static org.hamcrest.collection.IsIterableContainingInAnyOrder.containsInAnyOrder; import static org.hamcrest.collection.IsIterableContainingInOrder.contains; import static org.junit.Assert.assertArrayEquals; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertThat; +import static org.junit.Assert.fail; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.coders.AtomicCoder; @@ -53,6 +54,7 @@ import org.apache.beam.sdk.transforms.windowing.Window; import org.apache.beam.sdk.util.common.ElementByteSizeObserver; import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.PCollectionList; import org.apache.beam.sdk.values.PCollectionTuple; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TimestampedValue; @@ -60,6 +62,7 @@ import org.apache.beam.sdk.values.TupleTagList; import com.fasterxml.jackson.annotation.JsonCreator; + import org.joda.time.Duration; import org.joda.time.Instant; import org.junit.Rule; @@ -77,6 +80,7 @@ import java.util.Arrays; import java.util.Collections; import java.util.List; +import java.util.concurrent.atomic.AtomicBoolean; /** * Tests for ParDo. @@ -169,8 +173,10 @@ public void processElement(OldDoFn.ProcessContext c) throws Exc } static class TestDoFn extends DoFn { - enum State { UNSTARTED, STARTED, PROCESSING, FINISHED } - State state = State.UNSTARTED; + enum State {NOT_SET_UP, UNSTARTED, STARTED, PROCESSING, FINISHED} + + + State state = State.NOT_SET_UP; final List> sideInputViews = new ArrayList<>(); final List> sideOutputTupleTags = new ArrayList<>(); @@ -184,6 +190,12 @@ public TestDoFn(List> sideInputViews, this.sideOutputTupleTags.addAll(sideOutputTupleTags); } + @Setup + public void prepare() { + assertEquals(State.NOT_SET_UP, state); + state = State.UNSTARTED; + } + @StartBundle public void startBundle(Context c) { assertEquals(State.UNSTARTED, state); @@ -1463,4 +1475,404 @@ public void populateDisplayData(Builder builder) { assertThat(displayData, includesDisplayDataFrom(fn)); assertThat(displayData, hasDisplayItem("fn", fn.getClass())); } + + @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()) + .apply(ParDo.of(new CallSequenceEnforcingOldFn())); + + p.run(); + } + + @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()) + .apply(ParDo.of(new CallSequenceEnforcingOldFn()) + .withOutputTags(new TupleTag() {}, TupleTagList.empty())); + + p.run(); + } + + private static class CallSequenceEnforcingOldFn extends OldDoFn { + private boolean setupCalled = false; + private int startBundleCalls = 0; + private int finishBundleCalls = 0; + private boolean teardownCalled = false; + + @Override + public void setup() { + assertThat("setup should not be called twice", setupCalled, is(false)); + assertThat("setup should be called before startBundle", startBundleCalls, equalTo(0)); + assertThat("setup should be called before finishBundle", finishBundleCalls, equalTo(0)); + assertThat("setup should be called before teardown", teardownCalled, is(false)); + setupCalled = true; + } + + @Override + public void startBundle(Context c) { + assertThat("setup should have been called", setupCalled, is(true)); + assertThat( + "Even number of startBundle and finishBundle calls in startBundle", + startBundleCalls, + equalTo(finishBundleCalls)); + assertThat("teardown should not have been called", teardownCalled, is(false)); + startBundleCalls++; + } + + @Override + public void processElement(ProcessContext c) throws Exception { + assertThat("startBundle should have been called", startBundleCalls, greaterThan(0)); + assertThat( + "there should be one startBundle call with no call to finishBundle", + startBundleCalls, + equalTo(finishBundleCalls + 1)); + assertThat("teardown should not have been called", teardownCalled, is(false)); + } + + @Override + public void finishBundle(Context c) { + assertThat("startBundle should have been called", startBundleCalls, greaterThan(0)); + assertThat( + "there should be one bundle that has been started but not finished", + startBundleCalls, + equalTo(finishBundleCalls + 1)); + assertThat("teardown should not have been called", teardownCalled, is(false)); + finishBundleCalls++; + } + + @Override + public void teardown() { + assertThat(setupCalled, is(true)); + assertThat(startBundleCalls, anyOf(equalTo(finishBundleCalls))); + assertThat(teardownCalled, is(false)); + teardownCalled = true; + } + } + + @Test + @Category(RunnableOnService.class) + public void testFnWithContextCallSequence() { + 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()) + .apply(ParDo.of(new CallSequenceEnforcingFn())); + + p.run(); + } + + @Test + @Category(RunnableOnService.class) + public void testFnWithContextCallSequenceMulti() { + 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()) + .apply(ParDo.of(new CallSequenceEnforcingFn()) + .withOutputTags(new TupleTag() { + }, TupleTagList.empty())); + + p.run(); + } + + private static class CallSequenceEnforcingFn extends DoFn { + private boolean setupCalled = false; + private int startBundleCalls = 0; + private int finishBundleCalls = 0; + private boolean teardownCalled = false; + + @Setup + public void before() { + assertThat("setup should not be called twice", setupCalled, is(false)); + assertThat("setup should be called before startBundle", startBundleCalls, equalTo(0)); + assertThat("setup should be called before finishBundle", finishBundleCalls, equalTo(0)); + assertThat("setup should be called before teardown", teardownCalled, is(false)); + setupCalled = true; + } + + @StartBundle + public void begin(Context c) { + assertThat("setup should have been called", setupCalled, is(true)); + assertThat("Even number of startBundle and finishBundle calls in startBundle", + startBundleCalls, + equalTo(finishBundleCalls)); + assertThat("teardown should not have been called", teardownCalled, is(false)); + startBundleCalls++; + } + + @ProcessElement + public void process(ProcessContext c) throws Exception { + assertThat("startBundle should have been called", startBundleCalls, greaterThan(0)); + assertThat("there should be one startBundle call with no call to finishBundle", + startBundleCalls, + equalTo(finishBundleCalls + 1)); + assertThat("teardown should not have been called", teardownCalled, is(false)); + } + + @FinishBundle + public void end(Context c) { + assertThat("startBundle should have been called", startBundleCalls, greaterThan(0)); + assertThat("there should be one bundle that has been started but not finished", + startBundleCalls, + equalTo(finishBundleCalls + 1)); + assertThat("teardown should not have been called", teardownCalled, is(false)); + finishBundleCalls++; + } + + @Teardown + public void after() { + assertThat(setupCalled, is(true)); + assertThat(startBundleCalls, anyOf(equalTo(finishBundleCalls))); + assertThat(teardownCalled, is(false)); + teardownCalled = true; + } + } + + @Test + @Category(NeedsRunner.class) + public void testTeardownCalledAfterExceptionInSetup() { + TestPipeline p = TestPipeline.create(); + ExceptionThrowingOldFn fn = new ExceptionThrowingOldFn(MethodForException.SETUP); + p + .apply(Create.of(1, 2, 3)) + .apply(ParDo.of(fn)); + try { + p.run(); + fail("Pipeline should have failed with an exception"); + } catch (Exception e) { + assertThat( + "Function should have been torn down after exception", + ExceptionThrowingOldFn.teardownCalled.get(), + is(true)); + } + } + + @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)) + .apply(ParDo.of(fn)); + try { + p.run(); + fail("Pipeline should have failed with an exception"); + } catch (Exception e) { + assertThat( + "Function should have been torn down after exception", + ExceptionThrowingOldFn.teardownCalled.get(), + is(true)); + } + } + + @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)) + .apply(ParDo.of(fn)); + try { + p.run(); + fail("Pipeline should have failed with an exception"); + } catch (Exception e) { + assertThat( + "Function should have been torn down after exception", + ExceptionThrowingOldFn.teardownCalled.get(), + is(true)); + } + } + + @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)) + .apply(ParDo.of(fn)); + try { + p.run(); + fail("Pipeline should have failed with an exception"); + } catch (Exception e) { + assertThat( + "Function should have been torn down after exception", + ExceptionThrowingOldFn.teardownCalled.get(), + is(true)); + } + } + + @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 { + p.run(); + fail("Pipeline should have failed with an exception"); + } catch (Exception e) { + assertThat("Function should have been torn down after exception", + ExceptionThrowingOldFn.teardownCalled.get(), + is(true)); + } + } + + @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 { + p.run(); + fail("Pipeline should have failed with an exception"); + } catch (Exception e) { + assertThat("Function should have been torn down after exception", + ExceptionThrowingOldFn.teardownCalled.get(), + is(true)); + } + } + + @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 { + p.run(); + fail("Pipeline should have failed with an exception"); + } catch (Exception e) { + assertThat("Function should have been torn down after exception", + ExceptionThrowingOldFn.teardownCalled.get(), + is(true)); + } + } + + @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 { + p.run(); + fail("Pipeline should have failed with an exception"); + } catch (Exception e) { + assertThat("Function should have been torn down after exception", + ExceptionThrowingOldFn.teardownCalled.get(), + is(true)); + } + } + + private static class ExceptionThrowingOldFn extends OldDoFn { + static AtomicBoolean teardownCalled = new AtomicBoolean(false); + + private final MethodForException toThrow; + private boolean thrown; + + private ExceptionThrowingOldFn(MethodForException toThrow) { + this.toThrow = toThrow; + } + + @Override + public void setup() throws Exception { + throwIfNecessary(MethodForException.SETUP); + } + + @Override + public void startBundle(Context c) throws Exception { + throwIfNecessary(MethodForException.START_BUNDLE); + } + + @Override + public void processElement(ProcessContext c) throws Exception { + throwIfNecessary(MethodForException.PROCESS_ELEMENT); + } + + @Override + public void finishBundle(Context c) throws Exception { + throwIfNecessary(MethodForException.FINISH_BUNDLE); + } + + private void throwIfNecessary(MethodForException method) throws Exception { + if (toThrow == method && !thrown) { + thrown = true; + throw new Exception("Hasn't yet thrown"); + } + } + + @Override + public void teardown() { + if (!thrown) { + fail("Excepted to have a processing method throw an exception"); + } + teardownCalled.set(true); + } + } + + + private static class ExceptionThrowingFn extends DoFn { + static AtomicBoolean teardownCalled = new AtomicBoolean(false); + + private final MethodForException toThrow; + private boolean thrown; + + private ExceptionThrowingFn(MethodForException toThrow) { + this.toThrow = toThrow; + } + + @Setup + public void before() throws Exception { + throwIfNecessary(MethodForException.SETUP); + } + + @StartBundle + public void preBundle(Context c) throws Exception { + throwIfNecessary(MethodForException.START_BUNDLE); + } + + @ProcessElement + public void perElement(ProcessContext c) throws Exception { + throwIfNecessary(MethodForException.PROCESS_ELEMENT); + } + + @FinishBundle + public void postBundle(Context c) throws Exception { + throwIfNecessary(MethodForException.FINISH_BUNDLE); + } + + private void throwIfNecessary(MethodForException method) throws Exception { + if (toThrow == method && !thrown) { + thrown = true; + throw new Exception("Hasn't yet thrown"); + } + } + + @Teardown + public void after() { + if (!thrown) { + fail("Excepted to have a processing method throw an exception"); + } + teardownCalled.set(true); + } + } + + private enum MethodForException { + SETUP, + START_BUNDLE, + PROCESS_ELEMENT, + FINISH_BUNDLE + } } From 6603307062ec99639d3e3e05aebc0d1ea32ad411 Mon Sep 17 00:00:00 2001 From: Thomas Groh Date: Thu, 11 Aug 2016 10:45:43 -0700 Subject: [PATCH 025/346] Move ParDo Lifecycle tests to their own file These tests are not yet functional in all runners, and this makes them easier to ignore. --- runners/google-cloud-dataflow-java/pom.xml | 10 + .../sdk/transforms/ParDoLifecycleTest.java | 448 ++++++++++++++++++ .../apache/beam/sdk/transforms/ParDoTest.java | 405 ---------------- 3 files changed, 458 insertions(+), 405 deletions(-) create mode 100644 sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ParDoLifecycleTest.java diff --git a/runners/google-cloud-dataflow-java/pom.xml b/runners/google-cloud-dataflow-java/pom.xml index 86991b788f509..c32e1845616ca 100644 --- a/runners/google-cloud-dataflow-java/pom.xml +++ b/runners/google-cloud-dataflow-java/pom.xml @@ -60,6 +60,16 @@ true + + + runnable-on-service-tests + + + org/apache/beam/sdk/transforms/ParDoLifecycleTest.java + + + + com.google.guava guava-testlib ${guava.version} diff --git a/runners/direct-java/pom.xml b/runners/direct-java/pom.xml index 0a2b4b9cb4fa0..11481f10f1710 100644 --- a/runners/direct-java/pom.xml +++ b/runners/direct-java/pom.xml @@ -192,6 +192,10 @@ the second relocation. --> com.google.common + + + com.google.common.**.testing + org.apache.beam.runners.direct.repackaged.com.google.common @@ -264,6 +268,8 @@ + com.google.guava guava-testlib test diff --git a/runners/google-cloud-dataflow-java/pom.xml b/runners/google-cloud-dataflow-java/pom.xml index d130281979f89..d5485ef9f1577 100644 --- a/runners/google-cloud-dataflow-java/pom.xml +++ b/runners/google-cloud-dataflow-java/pom.xml @@ -196,6 +196,10 @@ the second relocation. --> com.google.common + + + com.google.common.**.testing + org.apache.beam.sdk.repackaged.com.google.common @@ -310,6 +314,8 @@ + com.google.guava guava-testlib test diff --git a/sdks/java/core/pom.xml b/sdks/java/core/pom.xml index aff4f66d78f3d..fddccea7874b4 100644 --- a/sdks/java/core/pom.xml +++ b/sdks/java/core/pom.xml @@ -193,6 +193,10 @@ exclude 'org.apache.beam.**', and remove the second relocation. --> com.google.common + + + com.google.common.**.testing + org.apache.beam.sdk.repackaged.com.google.common @@ -421,6 +425,8 @@ + com.google.guava guava-testlib test From 09cd1b70eadec0df9ae3b30f8dbc00207afa3dfc Mon Sep 17 00:00:00 2001 From: Daniel Halperin Date: Tue, 16 Aug 2016 13:11:16 -0700 Subject: [PATCH 028/346] ByteKeyRangeTracker: synchronize toString It's the only unsynchronized function, and it could potentially produce bad data. --- .../java/org/apache/beam/sdk/io/range/ByteKeyRangeTracker.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 7dd9a2cc9b208..24bf6e474b4da 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 @@ -176,7 +176,7 @@ public synchronized boolean markDone() { } @Override - public String toString() { + public synchronized String toString() { return toStringHelper(ByteKeyRangeTracker.class) .add("range", range) .add("position", position) From bd53cdc4d17ad6ae6ab58c5455e378b377be268c Mon Sep 17 00:00:00 2001 From: Scott Wegner Date: Wed, 17 Aug 2016 09:28:44 -0700 Subject: [PATCH 029/346] Fix repackaging exclude pattern for guava-testlib --- pom.xml | 2 +- runners/direct-java/pom.xml | 4 ++-- runners/google-cloud-dataflow-java/pom.xml | 4 ++-- sdks/java/core/pom.xml | 4 ++-- 4 files changed, 7 insertions(+), 7 deletions(-) diff --git a/pom.xml b/pom.xml index b5f30c132779f..f39f94aca693c 100644 --- a/pom.xml +++ b/pom.xml @@ -634,7 +634,7 @@ + excluding com.google.common.**.testing.* --> com.google.guava guava-testlib ${guava.version} diff --git a/runners/direct-java/pom.xml b/runners/direct-java/pom.xml index 11481f10f1710..e06883f6e65d7 100644 --- a/runners/direct-java/pom.xml +++ b/runners/direct-java/pom.xml @@ -194,7 +194,7 @@ com.google.common - com.google.common.**.testing + com.google.common.**.testing.* org.apache.beam.runners.direct.repackaged.com.google.common @@ -269,7 +269,7 @@ + excluding com.google.common.**.testing.* --> com.google.guava guava-testlib test diff --git a/runners/google-cloud-dataflow-java/pom.xml b/runners/google-cloud-dataflow-java/pom.xml index d5485ef9f1577..00b5a9ba67c25 100644 --- a/runners/google-cloud-dataflow-java/pom.xml +++ b/runners/google-cloud-dataflow-java/pom.xml @@ -198,7 +198,7 @@ com.google.common - com.google.common.**.testing + com.google.common.**.testing.* org.apache.beam.sdk.repackaged.com.google.common @@ -315,7 +315,7 @@ + excluding com.google.common.**.testing.* --> com.google.guava guava-testlib test diff --git a/sdks/java/core/pom.xml b/sdks/java/core/pom.xml index fddccea7874b4..78aec852bf84e 100644 --- a/sdks/java/core/pom.xml +++ b/sdks/java/core/pom.xml @@ -195,7 +195,7 @@ com.google.common - com.google.common.**.testing + com.google.common.**.testing.* org.apache.beam.sdk.repackaged.com.google.common @@ -426,7 +426,7 @@ + excluding com.google.common.**.testing.* --> com.google.guava guava-testlib test From cc189b42ab6f0aa7e648b8f86b66bd7f11516631 Mon Sep 17 00:00:00 2001 From: Eugene Kirpichov Date: Tue, 9 Aug 2016 17:16:00 -0700 Subject: [PATCH 030/346] Rewrites DoFnReflector to go via DoFnSignature DoFnSignature encapsulates type information about a DoFn, in particular which arguments/features its methods actually use. Before this commit, DoFnReflector would parse/verify/generate code in one go; after this commit, these stages are separated: DoFnSignature encapsulates all information needed to generate the code. Additionally, removes the unnecessary genericity in the implementation of DoFnReflector's code generation for the very different methods processElement and start/finishBundle. The code is simpler if decomposed into utility functions, rather than attempting a uniform representation for different methods. --- .../org/apache/beam/sdk/transforms/DoFn.java | 17 +- .../beam/sdk/transforms/DoFnAdapters.java | 281 ++++ .../beam/sdk/transforms/DoFnReflector.java | 1150 ----------------- .../beam/sdk/transforms/DoFnTester.java | 2 +- .../org/apache/beam/sdk/transforms/ParDo.java | 6 +- .../sdk/transforms/reflect/DoFnInvoker.java | 61 + .../sdk/transforms/reflect/DoFnInvokers.java | 506 ++++++++ .../sdk/transforms/reflect/DoFnSignature.java | 113 ++ .../transforms/reflect/DoFnSignatures.java | 321 +++++ .../sdk/transforms/reflect/package-info.java | 23 + .../sdk/transforms/DoFnReflectorTest.java | 822 ------------ .../beam/sdk/transforms/FlattenTest.java | 4 +- .../transforms/reflect/DoFnInvokersTest.java | 498 +++++++ .../DoFnInvokersTestHelper.java} | 8 +- .../reflect/DoFnSignaturesTest.java | 371 ++++++ ...chmark.java => DoFnInvokersBenchmark.java} | 46 +- 16 files changed, 2212 insertions(+), 2017 deletions(-) create mode 100644 sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnAdapters.java delete mode 100644 sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnReflector.java create mode 100644 sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/DoFnInvoker.java create mode 100644 sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/DoFnInvokers.java create mode 100644 sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/DoFnSignature.java create mode 100644 sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/DoFnSignatures.java create mode 100644 sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/package-info.java delete mode 100644 sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/DoFnReflectorTest.java create mode 100644 sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/reflect/DoFnInvokersTest.java rename sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/{dofnreflector/DoFnReflectorTestHelper.java => reflect/DoFnInvokersTestHelper.java} (93%) create mode 100644 sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/reflect/DoFnSignaturesTest.java rename sdks/java/microbenchmarks/src/main/java/org/apache/beam/sdk/microbenchmarks/transforms/{DoFnReflectorBenchmark.java => DoFnInvokersBenchmark.java} (83%) 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 80b67afc8412b..2348783f9f616 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 @@ -26,6 +26,7 @@ import org.apache.beam.sdk.transforms.OldDoFn.DelegatingAggregator; import org.apache.beam.sdk.transforms.display.DisplayData; import org.apache.beam.sdk.transforms.display.HasDisplayData; +import org.apache.beam.sdk.transforms.reflect.DoFnInvoker; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.values.PCollectionView; @@ -247,7 +248,7 @@ public Duration getAllowedTimestampSkew() { ///////////////////////////////////////////////////////////////////////////// - Map> aggregators = new HashMap<>(); + protected Map> aggregators = new HashMap<>(); /** * Protects aggregators from being created after initialization. @@ -283,7 +284,7 @@ protected TypeDescriptor getOutputTypeDescriptor() { /** * Interface for runner implementors to provide implementations of extra context information. * - *

    The methods on this interface are called by {@link DoFnReflector} before invoking an + *

    The methods on this interface are called by {@link DoFnInvoker} before invoking an * annotated {@link StartBundle}, {@link ProcessElement} or {@link FinishBundle} method that * has indicated it needs the given extra context. * @@ -301,23 +302,23 @@ public interface ExtraContextFactory { BoundedWindow window(); /** - * A placeholder for testing purposes. The return type itself is package-private and not - * implemented. + * A placeholder for testing purposes. */ InputProvider inputProvider(); /** - * A placeholder for testing purposes. The return type itself is package-private and not - * implemented. + * A placeholder for testing purposes. */ OutputReceiver outputReceiver(); } - static interface OutputReceiver { + /** A placeholder for testing handling of output types during {@link DoFn} reflection. */ + public interface OutputReceiver { void output(T output); } - static interface InputProvider { + /** A placeholder for testing handling of input types during {@link DoFn} reflection. */ + public interface InputProvider { T get(); } 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 new file mode 100644 index 0000000000000..71a148ffafe90 --- /dev/null +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnAdapters.java @@ -0,0 +1,281 @@ +/* + * 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.transforms; + +import org.apache.beam.sdk.options.PipelineOptions; +import org.apache.beam.sdk.transforms.display.DisplayData; +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.windowing.BoundedWindow; +import org.apache.beam.sdk.transforms.windowing.PaneInfo; +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; + +import java.io.IOException; + +/** + * Utility class containing adapters for running a {@link DoFn} as an {@link OldDoFn}. + * + * @deprecated This class will go away when we start running {@link DoFn}'s directly (using + * {@link DoFnInvoker}) rather than via {@link OldDoFn}. + */ +@Deprecated +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}. */ + public static OldDoFn toOldDoFn(DoFn fn) { + DoFnSignature signature = DoFnSignatures.INSTANCE.getOrParseSignature(fn.getClass()); + if (signature.processElement().usesSingleWindow()) { + return new WindowDoFnAdapter<>(fn); + } else { + return new SimpleDoFnAdapter<>(fn); + } + } + + /** + * Wraps a {@link DoFn} that doesn't require access to {@link BoundedWindow} as an {@link + * OldDoFn}. + */ + private static class SimpleDoFnAdapter extends OldDoFn { + private final DoFn fn; + private transient DoFnInvoker invoker; + + SimpleDoFnAdapter(DoFn fn) { + super(fn.aggregators); + this.fn = fn; + this.invoker = DoFnInvokers.INSTANCE.newByteBuddyInvoker(fn); + } + + @Override + public void setup() throws Exception { + this.invoker.invokeSetup(); + } + + @Override + public void startBundle(Context c) throws Exception { + this.fn.prepareForProcessing(); + invoker.invokeStartBundle(new ContextAdapter<>(fn, c)); + } + + @Override + public void finishBundle(Context c) throws Exception { + invoker.invokeFinishBundle(new ContextAdapter<>(fn, c)); + } + + @Override + public void teardown() throws Exception { + this.invoker.invokeTeardown(); + } + + @Override + public void processElement(ProcessContext c) throws Exception { + ProcessContextAdapter adapter = new ProcessContextAdapter<>(fn, c); + invoker.invokeProcessElement(adapter, adapter); + } + + @Override + protected TypeDescriptor getInputTypeDescriptor() { + return fn.getInputTypeDescriptor(); + } + + @Override + protected TypeDescriptor getOutputTypeDescriptor() { + return fn.getOutputTypeDescriptor(); + } + + @Override + public Duration getAllowedTimestampSkew() { + return fn.getAllowedTimestampSkew(); + } + + @Override + public void populateDisplayData(DisplayData.Builder builder) { + builder.include(fn); + } + + private void readObject(java.io.ObjectInputStream in) + throws IOException, ClassNotFoundException { + in.defaultReadObject(); + this.invoker = DoFnInvokers.INSTANCE.newByteBuddyInvoker(fn); + } + } + + /** Wraps a {@link DoFn} that requires access to {@link BoundedWindow} as an {@link OldDoFn}. */ + private static class WindowDoFnAdapter extends SimpleDoFnAdapter + implements OldDoFn.RequiresWindowAccess { + + WindowDoFnAdapter(DoFn fn) { + super(fn); + } + } + + /** + * Wraps an {@link OldDoFn.Context} as a {@link DoFn.ExtraContextFactory} inside a {@link + * DoFn.StartBundle} or {@link DoFn.FinishBundle} method, which means the extra context is + * unavailable. + */ + private static class ContextAdapter extends DoFn.Context + implements DoFn.ExtraContextFactory { + + private OldDoFn.Context context; + + private ContextAdapter(DoFn fn, OldDoFn.Context context) { + fn.super(); + this.context = context; + } + + @Override + public PipelineOptions getPipelineOptions() { + return context.getPipelineOptions(); + } + + @Override + public void output(OutputT output) { + context.output(output); + } + + @Override + public void outputWithTimestamp(OutputT output, Instant timestamp) { + context.outputWithTimestamp(output, timestamp); + } + + @Override + public void sideOutput(TupleTag tag, T output) { + context.sideOutput(tag, output); + } + + @Override + public void sideOutputWithTimestamp(TupleTag tag, T output, Instant timestamp) { + context.sideOutputWithTimestamp(tag, output, timestamp); + } + + @Override + public BoundedWindow window() { + // The DoFn doesn't allow us to ask for these outside ProcessElements, so this + // should be unreachable. + throw new UnsupportedOperationException("Can only get the window in ProcessElements"); + } + + @Override + public DoFn.InputProvider inputProvider() { + throw new UnsupportedOperationException("inputProvider() exists only for testing"); + } + + @Override + public DoFn.OutputReceiver outputReceiver() { + throw new UnsupportedOperationException("outputReceiver() exists only for testing"); + } + } + + /** + * Wraps an {@link OldDoFn.ProcessContext} as a {@link DoFn.ExtraContextFactory} inside a {@link + * DoFn.ProcessElement} method. + */ + private static class ProcessContextAdapter + extends DoFn.ProcessContext + implements DoFn.ExtraContextFactory { + + private OldDoFn.ProcessContext context; + + private ProcessContextAdapter( + DoFn fn, OldDoFn.ProcessContext context) { + fn.super(); + this.context = context; + } + + @Override + public PipelineOptions getPipelineOptions() { + return context.getPipelineOptions(); + } + + @Override + public T sideInput(PCollectionView view) { + return context.sideInput(view); + } + + @Override + public void output(OutputT output) { + context.output(output); + } + + @Override + public void outputWithTimestamp(OutputT output, Instant timestamp) { + context.outputWithTimestamp(output, timestamp); + } + + @Override + public void sideOutput(TupleTag tag, T output) { + context.sideOutput(tag, output); + } + + @Override + public void sideOutputWithTimestamp(TupleTag tag, T output, Instant timestamp) { + context.sideOutputWithTimestamp(tag, output, timestamp); + } + + @Override + public InputT element() { + return context.element(); + } + + @Override + public Instant timestamp() { + return context.timestamp(); + } + + @Override + public PaneInfo pane() { + return context.pane(); + } + + @Override + public BoundedWindow window() { + return context.window(); + } + + @Override + public DoFn.InputProvider inputProvider() { + throw new UnsupportedOperationException("inputProvider() exists only for testing"); + } + + @Override + public DoFn.OutputReceiver outputReceiver() { + throw new UnsupportedOperationException("outputReceiver() exists only for testing"); + } + } +} diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnReflector.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnReflector.java deleted file mode 100644 index bf040412b5e0a..0000000000000 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFnReflector.java +++ /dev/null @@ -1,1150 +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.transforms; - -import static com.google.common.base.Preconditions.checkState; - -import org.apache.beam.sdk.options.PipelineOptions; -import org.apache.beam.sdk.transforms.DoFn.ExtraContextFactory; -import org.apache.beam.sdk.transforms.DoFn.FinishBundle; -import org.apache.beam.sdk.transforms.DoFn.ProcessElement; -import org.apache.beam.sdk.transforms.DoFn.Setup; -import org.apache.beam.sdk.transforms.DoFn.StartBundle; -import org.apache.beam.sdk.transforms.DoFn.Teardown; -import org.apache.beam.sdk.transforms.display.DisplayData; -import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.transforms.windowing.PaneInfo; -import org.apache.beam.sdk.util.UserCodeException; -import org.apache.beam.sdk.util.common.ReflectHelpers; -import org.apache.beam.sdk.values.PCollectionView; -import org.apache.beam.sdk.values.TupleTag; -import org.apache.beam.sdk.values.TypeDescriptor; - -import com.google.common.annotations.VisibleForTesting; -import com.google.common.base.Function; -import com.google.common.base.Preconditions; -import com.google.common.base.Predicate; -import com.google.common.collect.FluentIterable; -import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableMap; -import com.google.common.reflect.TypeParameter; -import com.google.common.reflect.TypeToken; - -import net.bytebuddy.ByteBuddy; -import net.bytebuddy.NamingStrategy.SuffixingRandom; -import net.bytebuddy.description.field.FieldDescription; -import net.bytebuddy.description.method.MethodDescription; -import net.bytebuddy.description.method.ParameterList; -import net.bytebuddy.description.modifier.FieldManifestation; -import net.bytebuddy.description.modifier.Visibility; -import net.bytebuddy.description.type.TypeDescription; -import net.bytebuddy.description.type.TypeDescription.Generic; -import net.bytebuddy.dynamic.DynamicType; -import net.bytebuddy.dynamic.loading.ClassLoadingStrategy; -import net.bytebuddy.dynamic.scaffold.InstrumentedType; -import net.bytebuddy.dynamic.scaffold.subclass.ConstructorStrategy.Default; -import net.bytebuddy.implementation.Implementation; -import net.bytebuddy.implementation.MethodCall.MethodLocator; -import net.bytebuddy.implementation.StubMethod; -import net.bytebuddy.implementation.bind.MethodDelegationBinder.MethodInvoker; -import net.bytebuddy.implementation.bind.annotation.TargetMethodAnnotationDrivenBinder.TerminationHandler; -import net.bytebuddy.implementation.bytecode.ByteCodeAppender; -import net.bytebuddy.implementation.bytecode.Duplication; -import net.bytebuddy.implementation.bytecode.StackManipulation; -import net.bytebuddy.implementation.bytecode.Throw; -import net.bytebuddy.implementation.bytecode.assign.Assigner; -import net.bytebuddy.implementation.bytecode.member.FieldAccess; -import net.bytebuddy.implementation.bytecode.member.MethodInvocation; -import net.bytebuddy.implementation.bytecode.member.MethodReturn; -import net.bytebuddy.implementation.bytecode.member.MethodVariableAccess; -import net.bytebuddy.jar.asm.Label; -import net.bytebuddy.jar.asm.MethodVisitor; -import net.bytebuddy.jar.asm.Opcodes; -import net.bytebuddy.matcher.ElementMatchers; -import org.joda.time.Duration; -import org.joda.time.Instant; - -import java.io.IOException; -import java.lang.annotation.Annotation; -import java.lang.reflect.Constructor; -import java.lang.reflect.InvocationTargetException; -import java.lang.reflect.Method; -import java.lang.reflect.Modifier; -import java.lang.reflect.ParameterizedType; -import java.lang.reflect.Type; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collection; -import java.util.Collections; -import java.util.LinkedHashMap; -import java.util.LinkedHashSet; -import java.util.List; -import java.util.Map; -import javax.annotation.Nonnull; -import javax.annotation.Nullable; - - -/** - * Utility implementing the necessary reflection for working with {@link DoFn}s. - */ -public abstract class DoFnReflector { - - private static final String FN_DELEGATE_FIELD_NAME = "delegate"; - - private enum Availability { - /** Indicates parameters only available in {@code @ProcessElement} methods. */ - PROCESS_ELEMENT_ONLY, - /** Indicates parameters available in all methods. */ - EVERYWHERE - } - - /** - * Enumeration of the parameters available from the {@link ExtraContextFactory} to use as - * additional parameters for {@link DoFn} methods. - *

    - * We don't rely on looking for properly annotated methods within {@link ExtraContextFactory} - * because erasure would make it impossible to completely fill in the type token for context - * parameters that depend on the input/output type. - */ - private enum AdditionalParameter { - - /** Any {@link BoundedWindow} parameter is populated by the window of the current element. */ - WINDOW_OF_ELEMENT(Availability.PROCESS_ELEMENT_ONLY, BoundedWindow.class, "window") { - @Override - public TypeToken tokenFor(TypeToken in, TypeToken out) { - return TypeToken.of(BoundedWindow.class); - } - }, - - INPUT_PROVIDER(Availability.PROCESS_ELEMENT_ONLY, DoFn.InputProvider.class, "inputProvider") { - @Override - public TypeToken tokenFor(TypeToken in, TypeToken out) { - return new TypeToken>() {}.where( - new TypeParameter() {}, in); - } - - @Override - public boolean isHidden() { - return true; - } - }, - - OUTPUT_RECEIVER( - Availability.PROCESS_ELEMENT_ONLY, DoFn.OutputReceiver.class, "outputReceiver") { - @Override - public TypeToken tokenFor(TypeToken in, TypeToken out) { - return new TypeToken>() {}.where( - new TypeParameter() {}, out); - } - - @Override - public boolean isHidden() { - return true; - } - }; - - /** - * Create a type token representing the given parameter. May use the type token associated - * with the input and output types of the {@link DoFn}, depending on the extra - * context. - */ - abstract TypeToken tokenFor( - TypeToken in, TypeToken out); - - /** - * Indicates whether this enum is for testing only, hence should not appear in error messages, - * etc. Defaults to {@code false}. - */ - boolean isHidden() { - return false; - } - - private final Class rawType; - private final Availability availability; - private final transient MethodDescription method; - - private AdditionalParameter(Availability availability, Class rawType, String method) { - this.availability = availability; - this.rawType = rawType; - try { - this.method = new MethodDescription.ForLoadedMethod( - ExtraContextFactory.class.getMethod(method)); - } catch (NoSuchMethodException | SecurityException e) { - throw new RuntimeException( - "Unable to access method " + method + " on " + ExtraContextFactory.class, e); - } - } - } - - private static final Map, AdditionalParameter> EXTRA_CONTEXTS; - private static final Map, AdditionalParameter> EXTRA_PROCESS_CONTEXTS; - - static { - ImmutableMap.Builder, AdditionalParameter> everywhereBuilder = - ImmutableMap., AdditionalParameter>builder(); - ImmutableMap.Builder, AdditionalParameter> processElementBuilder = - ImmutableMap., AdditionalParameter>builder(); - - for (AdditionalParameter value : AdditionalParameter.values()) { - switch (value.availability) { - case EVERYWHERE: - everywhereBuilder.put(value.rawType, value); - break; - case PROCESS_ELEMENT_ONLY: - processElementBuilder.put(value.rawType, value); - break; - } - } - - EXTRA_CONTEXTS = everywhereBuilder.build(); - EXTRA_PROCESS_CONTEXTS = processElementBuilder - // Process Element contexts include everything available everywhere - .putAll(EXTRA_CONTEXTS) - .build(); - } - - /** - * @return true if the reflected {@link DoFn} uses a Single Window. - */ - public abstract boolean usesSingleWindow(); - - /** Create an {@link DoFnInvoker} bound to the given {@link OldDoFn}. */ - public abstract DoFnInvoker bindInvoker( - DoFn fn); - - private static final Map, DoFnReflector> REFLECTOR_CACHE = - new LinkedHashMap, DoFnReflector>(); - - /** - * @return the {@link DoFnReflector} for the given {@link DoFn}. - */ - public static DoFnReflector of( - @SuppressWarnings("rawtypes") Class fn) { - DoFnReflector reflector = REFLECTOR_CACHE.get(fn); - if (reflector != null) { - return reflector; - } - - reflector = new GenericDoFnReflector(fn); - REFLECTOR_CACHE.put(fn, reflector); - return reflector; - } - - /** - * Create a {@link OldDoFn} that the {@link DoFn}. - */ - public OldDoFn toDoFn(DoFn fn) { - if (usesSingleWindow()) { - return new WindowDoFnAdapter(this, fn); - } else { - return new SimpleDoFnAdapter(this, fn); - } - } - - private static String formatType(TypeToken t) { - return ReflectHelpers.TYPE_SIMPLE_DESCRIPTION.apply(t.getType()); - } - - private static String format(Method m) { - return ReflectHelpers.CLASS_AND_METHOD_FORMATTER.apply(m); - } - - private static Collection describeSupportedTypes( - Map, AdditionalParameter> extraProcessContexts, - final TypeToken in, final TypeToken out) { - return FluentIterable - .from(extraProcessContexts.values()) - .filter(new Predicate() { - @Override - public boolean apply(@Nonnull AdditionalParameter additionalParameter) { - return !additionalParameter.isHidden(); - } - }) - .transform(new Function() { - @Override - @Nonnull - public String apply(@Nonnull AdditionalParameter input) { - return formatType(input.tokenFor(in, out)); - } - }) - .toSortedSet(String.CASE_INSENSITIVE_ORDER); - } - - @VisibleForTesting - static List verifyProcessMethodArguments(Method m) { - return verifyMethodArguments(m, - EXTRA_PROCESS_CONTEXTS, - new TypeToken.ProcessContext>() {}, - new TypeParameter() {}, - new TypeParameter() {}); - } - - @VisibleForTesting - static List verifyBundleMethodArguments(Method m) { - if (m == null) { - return null; - } - return verifyMethodArguments(m, - EXTRA_CONTEXTS, - new TypeToken.Context>() {}, - new TypeParameter() {}, - new TypeParameter() {}); - } - - @VisibleForTesting - static void verifyLifecycleMethodArguments(Method m) { - if (m == null) { - return; - } - checkState(void.class.equals(m.getReturnType()), "%s must have void return type", format(m)); - checkState(m.getGenericParameterTypes().length == 0, "%s must take zero arguments", format(m)); - } - - /** - * Verify the method arguments for a given {@link DoFn} method. - * - *

    The requirements for a method to be valid, are: - *

      - *
    1. The method has at least one argument. - *
    2. The first argument is of type firstContextArg. - *
    3. The remaining arguments have raw types that appear in {@code contexts} - *
    4. Any generics on the extra context arguments match what is expected. Currently, this - * is exercised only by placeholders. For example, {@code InputReceiver must either match - * the {@code InputT} {@code OldDoFn.ProcessContext} or use a wildcard, etc. - *
    - * - * @param m the method to verify - * @param contexts mapping from raw classes to the {@link AdditionalParameter} used - * to create new instances. - * @param firstContextArg the expected type of the first context argument - * @param iParam TypeParameter representing the input type - * @param oParam TypeParameter representing the output type - */ - @VisibleForTesting - static List verifyMethodArguments( - Method m, - Map, AdditionalParameter> contexts, - TypeToken firstContextArg, - TypeParameter iParam, - TypeParameter oParam) { - - if (!void.class.equals(m.getReturnType())) { - throw new IllegalStateException(String.format( - "%s must have a void return type", format(m))); - } - if (m.isVarArgs()) { - throw new IllegalStateException(String.format( - "%s must not have var args", format(m))); - } - - // The first parameter must be present, and must be the specified type - Type[] params = m.getGenericParameterTypes(); - TypeToken contextToken = null; - if (params.length > 0) { - contextToken = TypeToken.of(params[0]); - } - if (contextToken == null - || !contextToken.getRawType().equals(firstContextArg.getRawType())) { - throw new IllegalStateException(String.format( - "%s must take a %s as its first argument", - format(m), firstContextArg.getRawType().getSimpleName())); - } - AdditionalParameter[] contextInfos = new AdditionalParameter[params.length - 1]; - - // Fill in the generics in the allExtraContextArgs interface from the types in the - // Context or ProcessContext OldDoFn. - ParameterizedType pt = (ParameterizedType) contextToken.getType(); - // We actually want the owner, since ProcessContext and Context are owned by DoFn. - pt = (ParameterizedType) pt.getOwnerType(); - @SuppressWarnings("unchecked") - TypeToken iActual = (TypeToken) TypeToken.of(pt.getActualTypeArguments()[0]); - @SuppressWarnings("unchecked") - TypeToken oActual = (TypeToken) TypeToken.of(pt.getActualTypeArguments()[1]); - - // All of the remaining parameters must be a super-interface of allExtraContextArgs - // that is not listed in the EXCLUDED_INTERFACES set. - for (int i = 1; i < params.length; i++) { - TypeToken param = TypeToken.of(params[i]); - - AdditionalParameter info = contexts.get(param.getRawType()); - if (info == null) { - throw new IllegalStateException(String.format( - "%s is not a valid context parameter for method %s. Should be one of %s", - formatType(param), format(m), - describeSupportedTypes(contexts, iActual, oActual))); - } - - // If we get here, the class matches, but maybe the generics don't: - TypeToken expected = info.tokenFor(iActual, oActual); - if (!expected.isSubtypeOf(param)) { - throw new IllegalStateException(String.format( - "Incompatible generics in context parameter %s for method %s. Should be %s", - formatType(param), format(m), formatType(info.tokenFor(iActual, oActual)))); - } - - // Register the (now validated) context info - contextInfos[i - 1] = info; - } - return ImmutableList.copyOf(contextInfos); - } - - /** Interface for invoking the {@code OldDoFn} processing methods. */ - public interface DoFnInvoker { - /** Invoke {@link OldDoFn#setup} on the bound {@code OldDoFn}. */ - void invokeSetup(); - /** Invoke {@link OldDoFn#startBundle} on the bound {@code OldDoFn}. */ - void invokeStartBundle( - DoFn.Context c, - ExtraContextFactory extra); - /** Invoke {@link OldDoFn#finishBundle} on the bound {@code OldDoFn}. */ - void invokeFinishBundle( - DoFn.Context c, - ExtraContextFactory extra); - - /** Invoke {@link OldDoFn#teardown()} on the bound {@code DoFn}. */ - void invokeTeardown(); - - /** Invoke {@link OldDoFn#processElement} on the bound {@code OldDoFn}. */ - public void invokeProcessElement( - DoFn.ProcessContext c, - ExtraContextFactory extra); - } - - /** - * Implementation of {@link DoFnReflector} for the arbitrary {@link DoFn}. - */ - private static class GenericDoFnReflector extends DoFnReflector { - - private final Method setup; - private final Method startBundle; - private final Method processElement; - private final Method finishBundle; - private final Method teardown; - private final List processElementArgs; - private final List startBundleArgs; - private final List finishBundleArgs; - private final Constructor constructor; - - private GenericDoFnReflector( - @SuppressWarnings("rawtypes") Class fn) { - // Locate the annotated methods - this.processElement = findAnnotatedMethod(ProcessElement.class, fn, true); - this.setup = findAnnotatedMethod(Setup.class, fn, false); - this.startBundle = findAnnotatedMethod(StartBundle.class, fn, false); - this.finishBundle = findAnnotatedMethod(FinishBundle.class, fn, false); - this.teardown = findAnnotatedMethod(Teardown.class, fn, false); - - // Verify that their method arguments satisfy our conditions. - this.processElementArgs = verifyProcessMethodArguments(processElement); - this.startBundleArgs = verifyBundleMethodArguments(startBundle); - this.finishBundleArgs = verifyBundleMethodArguments(finishBundle); - verifyLifecycleMethodArguments(setup); - verifyLifecycleMethodArguments(teardown); - - this.constructor = createInvokerConstructor(fn); - } - - private static Collection declaredMethodsWithAnnotation( - Class anno, - Class startClass, Class stopClass) { - Collection matches = new ArrayList<>(); - - Class clazz = startClass; - LinkedHashSet> interfaces = new LinkedHashSet<>(); - - // First, find all declared methods on the startClass and parents (up to stopClass) - while (clazz != null && !clazz.equals(stopClass)) { - for (Method method : clazz.getDeclaredMethods()) { - if (method.isAnnotationPresent(anno)) { - matches.add(method); - } - } - - Collections.addAll(interfaces, clazz.getInterfaces()); - - clazz = clazz.getSuperclass(); - } - - // Now, iterate over all the discovered interfaces - for (Method method : ReflectHelpers.getClosureOfMethodsOnInterfaces(interfaces)) { - if (method.isAnnotationPresent(anno)) { - matches.add(method); - } - } - return matches; - } - - private static Method findAnnotatedMethod( - Class anno, Class fnClazz, boolean required) { - Collection matches = declaredMethodsWithAnnotation( - anno, fnClazz, DoFn.class); - - if (matches.size() == 0) { - if (required) { - throw new IllegalStateException(String.format( - "No method annotated with @%s found in %s", - anno.getSimpleName(), fnClazz.getName())); - } else { - return null; - } - } - - // If we have at least one match, then either it should be the only match - // or it should be an extension of the other matches (which came from parent - // classes). - Method first = matches.iterator().next(); - for (Method other : matches) { - if (!first.getName().equals(other.getName()) - || !Arrays.equals(first.getParameterTypes(), other.getParameterTypes())) { - throw new IllegalStateException(String.format( - "Found multiple methods annotated with @%s. [%s] and [%s]", - anno.getSimpleName(), format(first), format(other))); - } - } - - // We need to be able to call it. We require it is public. - if ((first.getModifiers() & Modifier.PUBLIC) == 0) { - throw new IllegalStateException(format(first) + " must be public"); - } - - // And make sure its not static. - if ((first.getModifiers() & Modifier.STATIC) != 0) { - throw new IllegalStateException(format(first) + " must not be static"); - } - - return first; - } - - @Override - public boolean usesSingleWindow() { - return usesContext(AdditionalParameter.WINDOW_OF_ELEMENT); - } - - private boolean usesContext(AdditionalParameter param) { - return processElementArgs.contains(param) - || (startBundleArgs != null && startBundleArgs.contains(param)) - || (finishBundleArgs != null && finishBundleArgs.contains(param)); - } - - /** - * Use ByteBuddy to generate the code for a {@link DoFnInvoker} that invokes the given - * {@link DoFn}. - * @param clazz - * @return - */ - private Constructor> createInvokerConstructor( - @SuppressWarnings("rawtypes") Class clazz) { - - final TypeDescription clazzDescription = new TypeDescription.ForLoadedType(clazz); - - DynamicType.Builder builder = new ByteBuddy() - // Create subclasses inside the target class, to have access to - // private and package-private bits - .with(new SuffixingRandom("auxiliary") { - @Override - public String subclass(Generic superClass) { - return super.name(clazzDescription); - } - }) - // Create a subclass of DoFnInvoker - .subclass(DoFnInvoker.class, Default.NO_CONSTRUCTORS) - .defineField(FN_DELEGATE_FIELD_NAME, clazz, Visibility.PRIVATE, FieldManifestation.FINAL) - // Define a constructor to populate fields appropriately. - .defineConstructor(Visibility.PUBLIC) - .withParameter(clazz) - .intercept(new InvokerConstructor()) - // Implement the three methods by calling into the appropriate functions on the fn. - .method(ElementMatchers.named("invokeProcessElement")) - .intercept(InvokerDelegation.create( - processElement, BeforeDelegation.NOOP, processElementArgs)) - .method(ElementMatchers.named("invokeStartBundle")) - .intercept(InvokerDelegation.create( - startBundle, BeforeDelegation.INVOKE_PREPARE_FOR_PROCESSING, startBundleArgs)) - .method(ElementMatchers.named("invokeFinishBundle")) - .intercept(InvokerDelegation.create(finishBundle, - BeforeDelegation.NOOP, - finishBundleArgs)) - .method(ElementMatchers.named("invokeSetup")) - .intercept(InvokerDelegation.create(setup, - BeforeDelegation.NOOP, - Collections.emptyList())) - .method(ElementMatchers.named("invokeTeardown")) - .intercept(InvokerDelegation.create(teardown, - BeforeDelegation.NOOP, - Collections.emptyList())); - - @SuppressWarnings("unchecked") - Class> dynamicClass = (Class>) builder - .make() - .load(getClass().getClassLoader(), ClassLoadingStrategy.Default.INJECTION) - .getLoaded(); - try { - return dynamicClass.getConstructor(clazz); - } catch (IllegalArgumentException - | NoSuchMethodException - | SecurityException e) { - throw new RuntimeException(e); - } - } - - @Override - public DoFnInvoker bindInvoker( - DoFn fn) { - try { - @SuppressWarnings("unchecked") - DoFnInvoker invoker = - (DoFnInvoker) constructor.newInstance(fn); - return invoker; - } catch (InstantiationException - | IllegalAccessException - | IllegalArgumentException - | InvocationTargetException - | SecurityException e) { - throw new RuntimeException("Unable to bind invoker for " + fn.getClass(), e); - } - } - } - - private static class ContextAdapter - extends DoFn.Context - implements DoFn.ExtraContextFactory { - - private OldDoFn.Context context; - - private ContextAdapter( - DoFn fn, OldDoFn.Context context) { - fn.super(); - this.context = context; - } - - @Override - public PipelineOptions getPipelineOptions() { - return context.getPipelineOptions(); - } - - @Override - public void output(OutputT output) { - context.output(output); - } - - @Override - public void outputWithTimestamp(OutputT output, Instant timestamp) { - context.outputWithTimestamp(output, timestamp); - } - - @Override - public void sideOutput(TupleTag tag, T output) { - context.sideOutput(tag, output); - } - - @Override - public void sideOutputWithTimestamp(TupleTag tag, T output, Instant timestamp) { - context.sideOutputWithTimestamp(tag, output, timestamp); - } - - @Override - public BoundedWindow window() { - // The DoFn doesn't allow us to ask for these outside ProcessElements, so this - // should be unreachable. - throw new UnsupportedOperationException("Can only get the window in ProcessElements"); - } - - @Override - public DoFn.InputProvider inputProvider() { - throw new UnsupportedOperationException("inputProvider() exists only for testing"); - } - - @Override - public DoFn.OutputReceiver outputReceiver() { - throw new UnsupportedOperationException("outputReceiver() exists only for testing"); - } - } - - private static class ProcessContextAdapter - extends DoFn.ProcessContext - implements DoFn.ExtraContextFactory { - - private OldDoFn.ProcessContext context; - - private ProcessContextAdapter( - DoFn fn, - OldDoFn.ProcessContext context) { - fn.super(); - this.context = context; - } - - @Override - public PipelineOptions getPipelineOptions() { - return context.getPipelineOptions(); - } - - @Override - public T sideInput(PCollectionView view) { - return context.sideInput(view); - } - - @Override - public void output(OutputT output) { - context.output(output); - } - - @Override - public void outputWithTimestamp(OutputT output, Instant timestamp) { - context.outputWithTimestamp(output, timestamp); - } - - @Override - public void sideOutput(TupleTag tag, T output) { - context.sideOutput(tag, output); - } - - @Override - public void sideOutputWithTimestamp(TupleTag tag, T output, Instant timestamp) { - context.sideOutputWithTimestamp(tag, output, timestamp); - } - - @Override - public InputT element() { - return context.element(); - } - - @Override - public Instant timestamp() { - return context.timestamp(); - } - - @Override - public PaneInfo pane() { - return context.pane(); - } - - @Override - public BoundedWindow window() { - return context.window(); - } - - @Override - public DoFn.InputProvider inputProvider() { - throw new UnsupportedOperationException("inputProvider() exists only for testing"); - } - - @Override - public DoFn.OutputReceiver outputReceiver() { - throw new UnsupportedOperationException("outputReceiver() exists only for testing"); - } - } - - public static Class getDoFnClass(OldDoFn fn) { - if (fn instanceof SimpleDoFnAdapter) { - return ((SimpleDoFnAdapter) fn).fn.getClass(); - } else { - return fn.getClass(); - } - } - - private static class SimpleDoFnAdapter extends OldDoFn { - - private final DoFn fn; - private transient DoFnInvoker invoker; - - private SimpleDoFnAdapter(DoFnReflector reflector, DoFn fn) { - super(fn.aggregators); - this.fn = fn; - this.invoker = reflector.bindInvoker(fn); - } - - @Override - public void setup() throws Exception { - invoker.invokeSetup(); - } - - @Override - public void startBundle(OldDoFn.Context c) throws Exception { - ContextAdapter adapter = new ContextAdapter<>(fn, c); - invoker.invokeStartBundle(adapter, adapter); - } - - @Override - public void finishBundle(OldDoFn.Context c) throws Exception { - ContextAdapter adapter = new ContextAdapter<>(fn, c); - invoker.invokeFinishBundle(adapter, adapter); - } - - @Override - public void teardown() { - invoker.invokeTeardown(); - } - - @Override - public void processElement(OldDoFn.ProcessContext c) throws Exception { - ProcessContextAdapter adapter = new ProcessContextAdapter<>(fn, c); - invoker.invokeProcessElement(adapter, adapter); - } - - @Override - protected TypeDescriptor getInputTypeDescriptor() { - return fn.getInputTypeDescriptor(); - } - - @Override - protected TypeDescriptor getOutputTypeDescriptor() { - return fn.getOutputTypeDescriptor(); - } - - @Override - public Duration getAllowedTimestampSkew() { - return fn.getAllowedTimestampSkew(); - } - - @Override - public void populateDisplayData(DisplayData.Builder builder) { - builder.include(fn); - } - - private void readObject(java.io.ObjectInputStream in) - throws IOException, ClassNotFoundException { - in.defaultReadObject(); - invoker = DoFnReflector.of(fn.getClass()).bindInvoker(fn); - } - } - - private static class WindowDoFnAdapter - extends SimpleDoFnAdapter implements OldDoFn.RequiresWindowAccess { - - private WindowDoFnAdapter(DoFnReflector reflector, DoFn fn) { - super(reflector, fn); - } - } - - private static enum BeforeDelegation { - NOOP { - @Override - StackManipulation manipulation( - TypeDescription delegateType, MethodDescription instrumentedMethod, boolean finalStep) { - Preconditions.checkArgument(!finalStep, - "Shouldn't use NOOP delegation if there is nothing to do afterwards."); - return StackManipulation.Trivial.INSTANCE; - } - }, - INVOKE_PREPARE_FOR_PROCESSING { - private final Assigner assigner = Assigner.DEFAULT; - - @Override - StackManipulation manipulation( - TypeDescription delegateType, MethodDescription instrumentedMethod, boolean finalStep) { - MethodDescription prepareMethod; - try { - prepareMethod = new MethodLocator.ForExplicitMethod( - new MethodDescription.ForLoadedMethod( - DoFn.class.getDeclaredMethod("prepareForProcessing"))) - .resolve(instrumentedMethod); - } catch (NoSuchMethodException | SecurityException e) { - throw new RuntimeException("Unable to locate prepareForProcessing method", e); - } - - if (finalStep) { - return new StackManipulation.Compound( - // Invoke the prepare method - MethodInvoker.Simple.INSTANCE.invoke(prepareMethod), - // Return from the invokeStartBundle when we're done. - TerminationHandler.Returning.INSTANCE.resolve( - assigner, instrumentedMethod, prepareMethod)); - } else { - return new StackManipulation.Compound( - // Duplicate the delegation target so that it remains after we invoke prepare - Duplication.duplicate(delegateType), - // Invoke the prepare method - MethodInvoker.Simple.INSTANCE.invoke(prepareMethod), - // Drop the return value from prepareForProcessing - TerminationHandler.Dropping.INSTANCE.resolve( - assigner, instrumentedMethod, prepareMethod)); - } - } - }; - - /** - * Stack manipulation to perform prior to the delegate call. - * - *
      - *
    • Precondition: Stack has the delegate target on top of the stack - *
    • Postcondition: If finalStep is true, then we've returned from the method. Otherwise, the - * stack still has the delegate target on top of the stack. - *
    - * - * @param delegateType The type of the delegate target, in case it needs to be duplicated. - * @param instrumentedMethod The method bing instrumented. Necessary for resolving types and - * other information. - * @param finalStep If true, return from the {@code invokeStartBundle} method after invoking - * {@code prepareForProcessing} on the delegate. - */ - abstract StackManipulation manipulation( - TypeDescription delegateType, MethodDescription instrumentedMethod, boolean finalStep); - } - - /** - * A byte-buddy {@link Implementation} that delegates a call that receives - * {@link AdditionalParameter} to the given {@link DoFn} method. - */ - private static final class InvokerDelegation implements Implementation { - @Nullable - private final Method target; - private final BeforeDelegation before; - private final List args; - private final Assigner assigner = Assigner.DEFAULT; - private FieldDescription field; - - /** - * Create the {@link InvokerDelegation} for the specified method. - * - * @param target the method to delegate to - * @param isStartBundle whether or not this is the {@code startBundle} call - * @param args the {@link AdditionalParameter} to be passed to the {@code target} - */ - private InvokerDelegation( - @Nullable Method target, - BeforeDelegation before, - List args) { - this.target = target; - this.before = before; - this.args = args; - } - - /** - * Generate the {@link Implementation} of one of the life-cycle methods of a - * {@link DoFn}. - */ - private static Implementation create( - @Nullable final Method target, BeforeDelegation before, List args) { - if (target == null && before == BeforeDelegation.NOOP) { - // There is no target to call and nothing needs to happen before. Just produce a stub. - return StubMethod.INSTANCE; - } else { - // We need to generate a non-empty method implementation. - return new InvokerDelegation(target, before, args); - } - } - - @Override - public InstrumentedType prepare(InstrumentedType instrumentedType) { - // Remember the field description of the instrumented type. - field = instrumentedType.getDeclaredFields() - .filter(ElementMatchers.named(FN_DELEGATE_FIELD_NAME)).getOnly(); - - // Delegating the method call doesn't require any changes to the instrumented type. - return instrumentedType; - } - - /** - * Stack manipulation to push the {@link DoFn} reference stored in the - * delegate field of the invoker on to the top of the stack. - * - *

    This implementation is derived from the code for - * {@code MethodCall.invoke(m).onInstanceField(clazz, delegateField)} with two key differences. - * First, it doesn't add a synthetic field each time, which is critical to avoid duplicate field - * definitions. Second, it uses the {@link AdditionalParameter} to populate the arguments to the - * method. - */ - private StackManipulation pushDelegateField() { - return new StackManipulation.Compound( - // Push "this" reference to the stack - MethodVariableAccess.REFERENCE.loadOffset(0), - // Access the delegate field of the the invoker - FieldAccess.forField(field).getter()); - } - - private StackManipulation pushArgument( - AdditionalParameter arg, MethodDescription instrumentedMethod) { - MethodDescription transform = arg.method; - - return new StackManipulation.Compound( - // Push the ExtraContextFactory which must have been argument 2 of the instrumented method - MethodVariableAccess.REFERENCE.loadOffset(2), - // Invoke the appropriate method to produce the context argument - MethodInvocation.invoke(transform)); - } - - private StackManipulation invokeTargetMethod(MethodDescription instrumentedMethod) { - MethodDescription targetMethod = new MethodLocator.ForExplicitMethod( - new MethodDescription.ForLoadedMethod(target)).resolve(instrumentedMethod); - ParameterList params = targetMethod.getParameters(); - - List parameters; - if (!params.isEmpty()) { - // Instructions to setup the parameters for the call - parameters = new ArrayList<>(args.size() + 1); - // 1. The first argument in the delegate method must be the context. This corresponds to - // the first argument in the instrumented method, so copy that. - parameters.add(MethodVariableAccess.of(params.get(0).getType().getSuperClass()) - .loadOffset(1)); - // 2. For each of the extra arguments push the appropriate value. - for (AdditionalParameter arg : args) { - parameters.add(pushArgument(arg, instrumentedMethod)); - } - } else { - parameters = Collections.emptyList(); - } - - return new StackManipulation.Compound( - // Push the parameters - new StackManipulation.Compound(parameters), - // Invoke the target method - wrapWithUserCodeException(MethodInvoker.Simple.INSTANCE.invoke(targetMethod)), - // Return from the instrumented method - TerminationHandler.Returning.INSTANCE.resolve( - assigner, instrumentedMethod, targetMethod)); - } - - /** - * Wrap a given stack manipulation in a try catch block. Any exceptions thrown within the - * try are wrapped with a {@link UserCodeException}. - */ - private StackManipulation wrapWithUserCodeException( - final StackManipulation tryBody) { - final MethodDescription createUserCodeException; - try { - createUserCodeException = new MethodDescription.ForLoadedMethod( - UserCodeException.class.getDeclaredMethod("wrap", Throwable.class)); - } catch (NoSuchMethodException | SecurityException e) { - throw new RuntimeException("Unable to find UserCodeException.wrap", e); - } - - return new StackManipulation() { - @Override - public boolean isValid() { - return tryBody.isValid(); - } - - @Override - public Size apply(MethodVisitor mv, Context implementationContext) { - Label tryBlockStart = new Label(); - Label tryBlockEnd = new Label(); - Label catchBlockStart = new Label(); - Label catchBlockEnd = new Label(); - - String throwableName = - new TypeDescription.ForLoadedType(Throwable.class).getInternalName(); - mv.visitTryCatchBlock(tryBlockStart, tryBlockEnd, catchBlockStart, throwableName); - - // The try block attempts to perform the expected operations, then jumps to success - mv.visitLabel(tryBlockStart); - Size trySize = tryBody.apply(mv, implementationContext); - mv.visitJumpInsn(Opcodes.GOTO, catchBlockEnd); - mv.visitLabel(tryBlockEnd); - - // The handler wraps the exception, and then throws. - mv.visitLabel(catchBlockStart); - // Add the exception to the frame - mv.visitFrame(Opcodes.F_SAME1, - // No local variables - 0, new Object[] {}, - // 1 stack element (the throwable) - 1, new Object[] { throwableName }); - - Size catchSize = new StackManipulation.Compound( - MethodInvocation.invoke(createUserCodeException), - Throw.INSTANCE) - .apply(mv, implementationContext); - - mv.visitLabel(catchBlockEnd); - // The frame contents after the try/catch block is the same - // as it was before. - mv.visitFrame(Opcodes.F_SAME, - // No local variables - 0, new Object[] {}, - // No new stack variables - 0, new Object[] {}); - - return new Size( - trySize.getSizeImpact(), - Math.max(trySize.getMaximalSize(), catchSize.getMaximalSize())); - } - }; - } - - @Override - public ByteCodeAppender appender(final Target implementationTarget) { - return new ByteCodeAppender() { - @Override - public Size apply( - MethodVisitor methodVisitor, - Context implementationContext, - MethodDescription instrumentedMethod) { - StackManipulation.Size size = new StackManipulation.Compound( - // Put the target on top of the stack - pushDelegateField(), - // Do any necessary pre-delegation work - before.manipulation(field.getType().asErasure(), instrumentedMethod, target == null), - // Invoke the target method, if there is one. If there wasn't, then isStartBundle was - // true, and we've already emitted the appropriate return instructions. - target != null - ? invokeTargetMethod(instrumentedMethod) - : StackManipulation.Trivial.INSTANCE) - .apply(methodVisitor, implementationContext); - return new Size(size.getMaximalSize(), instrumentedMethod.getStackSize()); - } - }; - } - } - - /** - * A constructor {@link Implementation} for a {@link DoFnInvoker class}. Produces the byte code - * for a constructor that takes a single argument and assigns it to the delegate field. - * {@link AdditionalParameter} to the given {@link DoFn} method. - */ - private static final class InvokerConstructor implements Implementation { - @Override - public InstrumentedType prepare(InstrumentedType instrumentedType) { - return instrumentedType; - } - - @Override - public ByteCodeAppender appender(final Target implementationTarget) { - return new ByteCodeAppender() { - @Override - public Size apply( - MethodVisitor methodVisitor, - Context implementationContext, - MethodDescription instrumentedMethod) { - StackManipulation.Size size = new StackManipulation.Compound( - // Load the this reference - MethodVariableAccess.REFERENCE.loadOffset(0), - // Invoke the super constructor (default constructor of Object) - MethodInvocation - .invoke(new TypeDescription.ForLoadedType(Object.class) - .getDeclaredMethods() - .filter(ElementMatchers.isConstructor() - .and(ElementMatchers.takesArguments(0))) - .getOnly()), - // Load the this reference - MethodVariableAccess.REFERENCE.loadOffset(0), - // Load the delegate argument - MethodVariableAccess.REFERENCE.loadOffset(1), - // Assign the delegate argument to the delegate field - FieldAccess.forField(implementationTarget.getInstrumentedType() - .getDeclaredFields() - .filter(ElementMatchers.named(FN_DELEGATE_FIELD_NAME)) - .getOnly()).putter(), - // Return void. - MethodReturn.VOID - ).apply(methodVisitor, implementationContext); - return new Size(size.getMaximalSize(), instrumentedMethod.getStackSize()); - } - }; - } - } -} 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 dd1baab4a2658..4cd410a195973 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 @@ -92,7 +92,7 @@ public static DoFnTester of(OldDoFn DoFnTester of(DoFn fn) { - return new DoFnTester(DoFnReflector.of(fn.getClass()).toDoFn(fn)); + return new DoFnTester(DoFnAdapters.toOldDoFn(fn)); } /** 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 aa5753197b1d0..af500ba0c9c39 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 @@ -536,7 +536,7 @@ private static Bound of( private static OldDoFn adapt(DoFn fn) { - return DoFnReflector.of(fn.getClass()).toDoFn(fn); + return DoFnAdapters.toOldDoFn(fn); } /** @@ -747,7 +747,7 @@ protected Coder getDefaultOutputCoder(PCollection inp @Override protected String getKindString() { - Class clazz = DoFnReflector.getDoFnClass(fn); + Class clazz = DoFnAdapters.getDoFnClass(fn); if (clazz.isAnonymousClass()) { return "AnonymousParDo"; } else { @@ -968,7 +968,7 @@ public Coder getDefaultOutputCoder( @Override protected String getKindString() { - Class clazz = DoFnReflector.getDoFnClass(fn); + Class clazz = DoFnAdapters.getDoFnClass(fn); if (clazz.isAnonymousClass()) { return "AnonymousParMultiDo"; } else { 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 new file mode 100644 index 0000000000000..5818a5933f2d2 --- /dev/null +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/DoFnInvoker.java @@ -0,0 +1,61 @@ +/* + * 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.transforms.reflect; + +import org.apache.beam.sdk.transforms.DoFn; + +/** + * Interface for invoking the {@code DoFn} processing methods. + * + * Instantiating a {@link DoFnInvoker} associates it with a specific {@link DoFn} instance, + * referred to as the bound {@link DoFn}. + */ +public interface DoFnInvoker { + /** + * Invoke the {@link DoFn.Setup} method on the bound {@link DoFn}. + */ + void invokeSetup(); + + /** + * Invoke the {@link DoFn.StartBundle} method on the bound {@link DoFn}. + * + * @param c The {@link DoFn.Context} to invoke the fn with. + */ + void invokeStartBundle(DoFn.Context c); + + /** + * Invoke the {@link DoFn.FinishBundle} method on the bound {@link DoFn}. + * + * @param c The {@link DoFn.Context} to invoke the fn with. + */ + void invokeFinishBundle(DoFn.Context c); + + /** + * Invoke the {@link DoFn.Teardown} method on the bound {@link DoFn}. + */ + void invokeTeardown(); + + /** + * Invoke the {@link DoFn.ProcessElement} method on the bound {@link DoFn}. + * + * @param c The {@link DoFn.ProcessContext} to invoke the fn with. + * @param extra Factory for producing extra parameter objects (such as window), if necessary. + */ + void invokeProcessElement( + DoFn.ProcessContext c, DoFn.ExtraContextFactory extra); +} 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 new file mode 100644 index 0000000000000..73874d767d3c1 --- /dev/null +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/DoFnInvokers.java @@ -0,0 +1,506 @@ +/* + * 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.transforms.reflect; + +import static com.google.common.base.Preconditions.checkArgument; +import static com.google.common.base.Preconditions.checkNotNull; + +import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.transforms.DoFn.FinishBundle; +import org.apache.beam.sdk.transforms.DoFn.ProcessElement; +import org.apache.beam.sdk.transforms.DoFn.Setup; +import org.apache.beam.sdk.transforms.DoFn.StartBundle; +import org.apache.beam.sdk.transforms.DoFn.Teardown; +import org.apache.beam.sdk.util.UserCodeException; + +import net.bytebuddy.ByteBuddy; +import net.bytebuddy.NamingStrategy; +import net.bytebuddy.description.field.FieldDescription; +import net.bytebuddy.description.method.MethodDescription; +import net.bytebuddy.description.modifier.FieldManifestation; +import net.bytebuddy.description.modifier.Visibility; +import net.bytebuddy.description.type.TypeDescription; +import net.bytebuddy.dynamic.DynamicType; +import net.bytebuddy.dynamic.loading.ClassLoadingStrategy; +import net.bytebuddy.dynamic.scaffold.InstrumentedType; +import net.bytebuddy.dynamic.scaffold.subclass.ConstructorStrategy; +import net.bytebuddy.implementation.Implementation; +import net.bytebuddy.implementation.MethodCall; +import net.bytebuddy.implementation.bind.MethodDelegationBinder; +import net.bytebuddy.implementation.bytecode.ByteCodeAppender; +import net.bytebuddy.implementation.bytecode.StackManipulation; +import net.bytebuddy.implementation.bytecode.Throw; +import net.bytebuddy.implementation.bytecode.member.FieldAccess; +import net.bytebuddy.implementation.bytecode.member.MethodInvocation; +import net.bytebuddy.implementation.bytecode.member.MethodReturn; +import net.bytebuddy.implementation.bytecode.member.MethodVariableAccess; +import net.bytebuddy.jar.asm.Label; +import net.bytebuddy.jar.asm.MethodVisitor; +import net.bytebuddy.jar.asm.Opcodes; +import net.bytebuddy.matcher.ElementMatchers; + +import java.lang.reflect.Constructor; +import java.lang.reflect.InvocationTargetException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.EnumMap; +import java.util.LinkedHashMap; +import java.util.Map; + +import javax.annotation.Nullable; + +/** Dynamically generates {@link DoFnInvoker} instances for invoking a {@link DoFn}. */ +public class DoFnInvokers { + public static final DoFnInvokers INSTANCE = new DoFnInvokers(); + + private static final String FN_DELEGATE_FIELD_NAME = "delegate"; + + /** + * 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() {} + + /** @return the {@link DoFnInvoker} for the given {@link DoFn}. */ + public DoFnInvoker newByteBuddyInvoker( + DoFn fn) { + return newByteBuddyInvoker(DoFnSignatures.INSTANCE.getOrParseSignature(fn.getClass()), fn); + } + + /** @return the {@link DoFnInvoker} for the given {@link DoFn}. */ + public DoFnInvoker newByteBuddyInvoker( + DoFnSignature signature, DoFn fn) { + checkArgument( + signature.fnClass().equals(fn.getClass()), + "Signature is for class %s, but fn is of class %s", + signature.fnClass(), + fn.getClass()); + try { + @SuppressWarnings("unchecked") + DoFnInvoker invoker = + (DoFnInvoker) + getOrGenerateByteBuddyInvokerConstructor(signature).newInstance(fn); + return invoker; + } catch (InstantiationException + | IllegalAccessException + | IllegalArgumentException + | InvocationTargetException + | SecurityException e) { + throw new RuntimeException("Unable to bind invoker for " + fn.getClass(), e); + } + } + + /** + * Returns a generated constructor for a {@link DoFnInvoker} for the given {@link DoFn} class and + * caches it. + */ + private synchronized Constructor getOrGenerateByteBuddyInvokerConstructor( + DoFnSignature signature) { + Class fnClass = signature.fnClass(); + Constructor constructor = byteBuddyInvokerConstructorCache.get(fnClass); + if (constructor == null) { + Class> invokerClass = generateInvokerClass(signature); + try { + constructor = invokerClass.getConstructor(fnClass); + } catch (IllegalArgumentException | NoSuchMethodException | SecurityException e) { + throw new RuntimeException(e); + } + byteBuddyInvokerConstructorCache.put(fnClass, constructor); + } + return constructor; + } + + /** Generates a {@link DoFnInvoker} class for the given {@link DoFnSignature}. */ + private static Class> generateInvokerClass(DoFnSignature signature) { + Class fnClass = signature.fnClass(); + + final TypeDescription clazzDescription = new TypeDescription.ForLoadedType(fnClass); + + DynamicType.Builder builder = + new ByteBuddy() + // Create subclasses inside the target class, to have access to + // private and package-private bits + .with( + new NamingStrategy.SuffixingRandom("auxiliary") { + @Override + public String subclass(TypeDescription.Generic superClass) { + return super.name(clazzDescription); + } + }) + // Create a subclass of DoFnInvoker + .subclass(DoFnInvoker.class, ConstructorStrategy.Default.NO_CONSTRUCTORS) + .defineField( + FN_DELEGATE_FIELD_NAME, fnClass, Visibility.PRIVATE, FieldManifestation.FINAL) + .defineConstructor(Visibility.PUBLIC) + .withParameter(fnClass) + .intercept(new InvokerConstructor()) + .method(ElementMatchers.named("invokeProcessElement")) + .intercept(new ProcessElementDelegation(signature.processElement())) + .method(ElementMatchers.named("invokeStartBundle")) + .intercept( + signature.startBundle() == null + ? new NoopMethodImplementation() + : new BundleMethodDelegation(signature.startBundle())) + .method(ElementMatchers.named("invokeFinishBundle")) + .intercept( + signature.finishBundle() == null + ? new NoopMethodImplementation() + : new BundleMethodDelegation(signature.finishBundle())) + .method(ElementMatchers.named("invokeSetup")) + .intercept( + signature.setup() == null + ? new NoopMethodImplementation() + : new LifecycleMethodDelegation(signature.setup())) + .method(ElementMatchers.named("invokeTeardown")) + .intercept( + signature.teardown() == null + ? new NoopMethodImplementation() + : new LifecycleMethodDelegation(signature.teardown())); + + DynamicType.Unloaded unloaded = builder.make(); + + @SuppressWarnings("unchecked") + Class> res = + (Class>) + unloaded + .load(DoFnInvokers.class.getClassLoader(), ClassLoadingStrategy.Default.INJECTION) + .getLoaded(); + return res; + } + + /** Implements an invoker method by doing nothing and immediately returning void. */ + private static class NoopMethodImplementation implements Implementation { + @Override + public InstrumentedType prepare(InstrumentedType instrumentedType) { + return instrumentedType; + } + + @Override + public ByteCodeAppender appender(final Target implementationTarget) { + return new ByteCodeAppender() { + @Override + public Size apply( + MethodVisitor methodVisitor, + Context implementationContext, + MethodDescription instrumentedMethod) { + StackManipulation manipulation = MethodReturn.VOID; + StackManipulation.Size size = manipulation.apply(methodVisitor, implementationContext); + return new Size(size.getMaximalSize(), instrumentedMethod.getStackSize()); + } + }; + } + } + + /** + * Base class for implementing an invoker method by delegating to a method of the target {@link + * DoFn}. + */ + private abstract static class MethodDelegation implements Implementation { + FieldDescription delegateField; + + @Override + public InstrumentedType prepare(InstrumentedType instrumentedType) { + // Remember the field description of the instrumented type. + delegateField = + instrumentedType + .getDeclaredFields() + .filter(ElementMatchers.named(FN_DELEGATE_FIELD_NAME)) + .getOnly(); + + // Delegating the method call doesn't require any changes to the instrumented type. + return instrumentedType; + } + + @Override + public ByteCodeAppender appender(final Target implementationTarget) { + return new ByteCodeAppender() { + @Override + public Size apply( + MethodVisitor methodVisitor, + Context implementationContext, + MethodDescription instrumentedMethod) { + StackManipulation manipulation = + new StackManipulation.Compound( + // Push "this" reference to the stack + MethodVariableAccess.REFERENCE.loadOffset(0), + // Access the delegate field of the the invoker + FieldAccess.forField(delegateField).getter(), + invokeTargetMethod(instrumentedMethod)); + StackManipulation.Size size = manipulation.apply(methodVisitor, implementationContext); + return new Size(size.getMaximalSize(), instrumentedMethod.getStackSize()); + } + }; + } + + /** + * Generates code to invoke the target method. When this is called the delegate field will be on + * top of the stack. This should add any necessary arguments to the stack and then perform the + * method invocation. + */ + protected abstract StackManipulation invokeTargetMethod(MethodDescription instrumentedMethod); + } + + /** + * Implements the invoker's {@link DoFnInvoker#invokeProcessElement} method by delegating to the + * {@link DoFn.ProcessElement} method. + */ + private static final class ProcessElementDelegation extends MethodDelegation { + private static final Map + EXTRA_CONTEXT_FACTORY_METHODS; + + static { + try { + Map methods = + new EnumMap<>(DoFnSignature.ProcessElementMethod.Parameter.class); + methods.put( + DoFnSignature.ProcessElementMethod.Parameter.BOUNDED_WINDOW, + new MethodDescription.ForLoadedMethod( + DoFn.ExtraContextFactory.class.getMethod("window"))); + methods.put( + DoFnSignature.ProcessElementMethod.Parameter.INPUT_PROVIDER, + new MethodDescription.ForLoadedMethod( + DoFn.ExtraContextFactory.class.getMethod("inputProvider"))); + methods.put( + DoFnSignature.ProcessElementMethod.Parameter.OUTPUT_RECEIVER, + new MethodDescription.ForLoadedMethod( + DoFn.ExtraContextFactory.class.getMethod("outputReceiver"))); + EXTRA_CONTEXT_FACTORY_METHODS = Collections.unmodifiableMap(methods); + } catch (Exception e) { + throw new RuntimeException( + "Failed to locate an ExtraContextFactory method that was expected to exist", e); + } + } + + private final DoFnSignature.ProcessElementMethod signature; + + /** Implementation of {@link MethodDelegation} for the {@link ProcessElement} method. */ + private ProcessElementDelegation(DoFnSignature.ProcessElementMethod signature) { + this.signature = signature; + } + + @Override + protected StackManipulation invokeTargetMethod(MethodDescription instrumentedMethod) { + MethodDescription targetMethod = + new MethodCall.MethodLocator.ForExplicitMethod( + new MethodDescription.ForLoadedMethod(signature.targetMethod())) + .resolve(instrumentedMethod); + + // Parameters of the wrapper invoker method: + // DoFn.ProcessContext, ExtraContextFactory. + // Parameters of the wrapped DoFn method: + // DoFn.ProcessContext, [BoundedWindow, InputProvider, OutputReceiver] in any order + ArrayList parameters = new ArrayList<>(); + // Push the ProcessContext argument. + parameters.add(MethodVariableAccess.REFERENCE.loadOffset(1)); + // Push the extra arguments in their actual order. + StackManipulation pushExtraContextFactory = MethodVariableAccess.REFERENCE.loadOffset(2); + for (DoFnSignature.ProcessElementMethod.Parameter param : signature.extraParameters()) { + parameters.add( + new StackManipulation.Compound( + pushExtraContextFactory, + MethodInvocation.invoke(EXTRA_CONTEXT_FACTORY_METHODS.get(param)))); + } + + return new StackManipulation.Compound( + // Push the parameters + new StackManipulation.Compound(parameters), + // Invoke the target method + wrapWithUserCodeException( + MethodDelegationBinder.MethodInvoker.Simple.INSTANCE.invoke(targetMethod)), + // Return from the instrumented method + MethodReturn.VOID); + } + } + + /** + * Implements {@link DoFnInvoker#invokeStartBundle} or {@link DoFnInvoker#invokeFinishBundle} by + * delegating respectively to the {@link StartBundle} and {@link FinishBundle} methods. + */ + private static final class BundleMethodDelegation extends MethodDelegation { + private final DoFnSignature.BundleMethod signature; + + private BundleMethodDelegation(@Nullable DoFnSignature.BundleMethod signature) { + this.signature = signature; + } + + @Override + protected StackManipulation invokeTargetMethod(MethodDescription instrumentedMethod) { + MethodDescription targetMethod = + new MethodCall.MethodLocator.ForExplicitMethod( + new MethodDescription.ForLoadedMethod(checkNotNull(signature).targetMethod())) + .resolve(instrumentedMethod); + return new StackManipulation.Compound( + // Push the parameters + MethodVariableAccess.REFERENCE.loadOffset(1), + // Invoke the target method + wrapWithUserCodeException( + MethodDelegationBinder.MethodInvoker.Simple.INSTANCE.invoke(targetMethod)), + MethodReturn.VOID); + } + } + + /** + * Implements {@link DoFnInvoker#invokeSetup} or {@link DoFnInvoker#invokeTeardown} by delegating + * respectively to the {@link Setup} and {@link Teardown} methods. + */ + private static final class LifecycleMethodDelegation extends MethodDelegation { + private final DoFnSignature.LifecycleMethod signature; + + private LifecycleMethodDelegation(@Nullable DoFnSignature.LifecycleMethod signature) { + this.signature = signature; + } + + @Override + protected StackManipulation invokeTargetMethod(MethodDescription instrumentedMethod) { + MethodDescription targetMethod = + new MethodCall.MethodLocator.ForExplicitMethod( + new MethodDescription.ForLoadedMethod(checkNotNull(signature).targetMethod())) + .resolve(instrumentedMethod); + return new StackManipulation.Compound( + wrapWithUserCodeException( + MethodDelegationBinder.MethodInvoker.Simple.INSTANCE.invoke(targetMethod)), + MethodReturn.VOID); + } + } + + /** + * Wraps a given stack manipulation in a try catch block. Any exceptions thrown within the try are + * wrapped with a {@link UserCodeException}. + */ + private static StackManipulation wrapWithUserCodeException(final StackManipulation tryBody) { + final MethodDescription createUserCodeException; + try { + createUserCodeException = + new MethodDescription.ForLoadedMethod( + UserCodeException.class.getDeclaredMethod("wrap", Throwable.class)); + } catch (NoSuchMethodException | SecurityException e) { + throw new RuntimeException("Unable to find UserCodeException.wrap", e); + } + + return new StackManipulation() { + @Override + public boolean isValid() { + return tryBody.isValid(); + } + + @Override + public Size apply(MethodVisitor mv, Implementation.Context implementationContext) { + Label tryBlockStart = new Label(); + Label tryBlockEnd = new Label(); + Label catchBlockStart = new Label(); + Label catchBlockEnd = new Label(); + + String throwableName = new TypeDescription.ForLoadedType(Throwable.class).getInternalName(); + mv.visitTryCatchBlock(tryBlockStart, tryBlockEnd, catchBlockStart, throwableName); + + // The try block attempts to perform the expected operations, then jumps to success + mv.visitLabel(tryBlockStart); + Size trySize = tryBody.apply(mv, implementationContext); + mv.visitJumpInsn(Opcodes.GOTO, catchBlockEnd); + mv.visitLabel(tryBlockEnd); + + // The handler wraps the exception, and then throws. + mv.visitLabel(catchBlockStart); + // Add the exception to the frame + mv.visitFrame( + Opcodes.F_SAME1, + // No local variables + 0, + new Object[] {}, + // 1 stack element (the throwable) + 1, + new Object[] {throwableName}); + + Size catchSize = + new Compound(MethodInvocation.invoke(createUserCodeException), Throw.INSTANCE) + .apply(mv, implementationContext); + + mv.visitLabel(catchBlockEnd); + // The frame contents after the try/catch block is the same + // as it was before. + mv.visitFrame( + Opcodes.F_SAME, + // No local variables + 0, + new Object[] {}, + // No new stack variables + 0, + new Object[] {}); + + return new Size( + trySize.getSizeImpact(), + Math.max(trySize.getMaximalSize(), catchSize.getMaximalSize())); + } + }; + } + + /** + * A constructor {@link Implementation} for a {@link DoFnInvoker class}. Produces the byte code + * for a constructor that takes a single argument and assigns it to the delegate field. + */ + private static final class InvokerConstructor implements Implementation { + @Override + public InstrumentedType prepare(InstrumentedType instrumentedType) { + return instrumentedType; + } + + @Override + public ByteCodeAppender appender(final Target implementationTarget) { + return new ByteCodeAppender() { + @Override + public Size apply( + MethodVisitor methodVisitor, + Context implementationContext, + MethodDescription instrumentedMethod) { + StackManipulation.Size size = + new StackManipulation.Compound( + // Load the this reference + MethodVariableAccess.REFERENCE.loadOffset(0), + // Invoke the super constructor (default constructor of Object) + MethodInvocation.invoke( + new TypeDescription.ForLoadedType(Object.class) + .getDeclaredMethods() + .filter( + ElementMatchers.isConstructor() + .and(ElementMatchers.takesArguments(0))) + .getOnly()), + // Load the this reference + MethodVariableAccess.REFERENCE.loadOffset(0), + // Load the delegate argument + MethodVariableAccess.REFERENCE.loadOffset(1), + // Assign the delegate argument to the delegate field + FieldAccess.forField( + implementationTarget + .getInstrumentedType() + .getDeclaredFields() + .filter(ElementMatchers.named(FN_DELEGATE_FIELD_NAME)) + .getOnly()) + .putter(), + // Return void. + MethodReturn.VOID) + .apply(methodVisitor, implementationContext); + return new Size(size.getMaximalSize(), instrumentedMethod.getStackSize()); + } + }; + } + } +} 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 new file mode 100644 index 0000000000000..6730140ac309d --- /dev/null +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/DoFnSignature.java @@ -0,0 +1,113 @@ +/* + * 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.transforms.reflect; + +import org.apache.beam.sdk.transforms.DoFn; + +import com.google.auto.value.AutoValue; + +import java.lang.reflect.Method; +import java.util.Collections; +import java.util.List; + +import javax.annotation.Nullable; + +/** + * Describes the signature of a {@link DoFn}, in particular, which features it uses, which extra + * context it requires, types of the input and output elements, etc. + * + *

    See A new DoFn. + */ +@AutoValue +public abstract class DoFnSignature { + public abstract Class fnClass(); + + public abstract ProcessElementMethod processElement(); + + @Nullable + public abstract BundleMethod startBundle(); + + @Nullable + public abstract BundleMethod finishBundle(); + + @Nullable + public abstract LifecycleMethod setup(); + + @Nullable + public abstract LifecycleMethod teardown(); + + static DoFnSignature create( + Class fnClass, + ProcessElementMethod processElement, + @Nullable BundleMethod startBundle, + @Nullable BundleMethod finishBundle, + @Nullable LifecycleMethod setup, + @Nullable LifecycleMethod teardown) { + return new AutoValue_DoFnSignature( + fnClass, + processElement, + startBundle, + finishBundle, + setup, + teardown); + } + + /** Describes a {@link DoFn.ProcessElement} method. */ + @AutoValue + public abstract static class ProcessElementMethod { + enum Parameter { + BOUNDED_WINDOW, + INPUT_PROVIDER, + OUTPUT_RECEIVER + } + + public abstract Method targetMethod(); + + public abstract List extraParameters(); + + static ProcessElementMethod create(Method targetMethod, List extraParameters) { + return new AutoValue_DoFnSignature_ProcessElementMethod( + targetMethod, Collections.unmodifiableList(extraParameters)); + } + + /** @return true if the reflected {@link DoFn} uses a Single Window. */ + public boolean usesSingleWindow() { + return extraParameters().contains(Parameter.BOUNDED_WINDOW); + } + } + + /** Describes a {@link DoFn.StartBundle} or {@link DoFn.FinishBundle} method. */ + @AutoValue + public abstract static class BundleMethod { + public abstract Method targetMethod(); + + static BundleMethod create(Method targetMethod) { + return new AutoValue_DoFnSignature_BundleMethod(targetMethod); + } + } + + /** Describes a {@link DoFn.Setup} or {@link DoFn.Teardown} method. */ + @AutoValue + public abstract static class LifecycleMethod { + public abstract Method targetMethod(); + + static LifecycleMethod create(Method targetMethod) { + return new AutoValue_DoFnSignature_LifecycleMethod(targetMethod); + } + } +} 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 new file mode 100644 index 0000000000000..80b3b4f5b825a --- /dev/null +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/DoFnSignatures.java @@ -0,0 +1,321 @@ +/* + * 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.transforms.reflect; + +import static com.google.common.base.Preconditions.checkArgument; +import static com.google.common.base.Preconditions.checkNotNull; + +import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.transforms.windowing.BoundedWindow; +import org.apache.beam.sdk.util.common.ReflectHelpers; + +import com.google.common.annotations.VisibleForTesting; +import com.google.common.reflect.TypeParameter; +import com.google.common.reflect.TypeToken; + +import java.lang.annotation.Annotation; +import java.lang.reflect.Method; +import java.lang.reflect.Modifier; +import java.lang.reflect.ParameterizedType; +import java.lang.reflect.Type; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.LinkedHashMap; +import java.util.LinkedHashSet; +import java.util.List; +import java.util.Map; + +/** + * Parses a {@link DoFn} and computes its {@link DoFnSignature}. See {@link #getOrParseSignature}. + */ +public class DoFnSignatures { + public static final DoFnSignatures INSTANCE = new DoFnSignatures(); + + private DoFnSignatures() {} + + private final Map, DoFnSignature> signatureCache = new LinkedHashMap<>(); + + /** @return the {@link DoFnSignature} for the given {@link DoFn}. */ + public synchronized DoFnSignature getOrParseSignature( + @SuppressWarnings("rawtypes") Class fn) { + DoFnSignature signature = signatureCache.get(fn); + if (signature == null) { + signatureCache.put(fn, signature = parseSignature(fn)); + } + return signature; + } + + /** Analyzes a given {@link DoFn} class and extracts its {@link DoFnSignature}. */ + private static DoFnSignature parseSignature(Class fnClass) { + TypeToken inputT = null; + TypeToken outputT = null; + + // Extract the input and output type. + checkArgument( + DoFn.class.isAssignableFrom(fnClass), + "%s must be subtype of DoFn", + fnClass.getSimpleName()); + TypeToken fnToken = TypeToken.of(fnClass); + for (TypeToken supertype : fnToken.getTypes()) { + if (!supertype.getRawType().equals(DoFn.class)) { + continue; + } + Type[] args = ((ParameterizedType) supertype.getType()).getActualTypeArguments(); + inputT = TypeToken.of(args[0]); + outputT = TypeToken.of(args[1]); + } + checkNotNull(inputT, "Unable to determine input type from %s", fnClass); + + Method processElementMethod = findAnnotatedMethod(DoFn.ProcessElement.class, fnClass, true); + Method startBundleMethod = findAnnotatedMethod(DoFn.StartBundle.class, fnClass, false); + Method finishBundleMethod = findAnnotatedMethod(DoFn.FinishBundle.class, fnClass, false); + Method setupMethod = findAnnotatedMethod(DoFn.Setup.class, fnClass, false); + Method teardownMethod = findAnnotatedMethod(DoFn.Teardown.class, fnClass, false); + + return DoFnSignature.create( + fnClass, + analyzeProcessElementMethod(fnToken, processElementMethod, inputT, outputT), + (startBundleMethod == null) + ? null + : analyzeBundleMethod(fnToken, startBundleMethod, inputT, outputT), + (finishBundleMethod == null) + ? null + : analyzeBundleMethod(fnToken, finishBundleMethod, inputT, outputT), + (setupMethod == null) ? null : analyzeLifecycleMethod(setupMethod), + (teardownMethod == null) ? null : analyzeLifecycleMethod(teardownMethod)); + } + + /** + * Generates a type token for {@code DoFn.ProcessContext} given {@code InputT} + * and {@code OutputT}. + */ + private static + TypeToken.ProcessContext> doFnProcessContextTypeOf( + TypeToken inputT, TypeToken outputT) { + return new TypeToken.ProcessContext>() {}.where( + new TypeParameter() {}, inputT) + .where(new TypeParameter() {}, outputT); + } + + /** + * Generates a type token for {@code DoFn.Context} given {@code InputT} and + * {@code OutputT}. + */ + private static TypeToken.Context> doFnContextTypeOf( + TypeToken inputT, TypeToken outputT) { + return new TypeToken.Context>() {}.where( + new TypeParameter() {}, inputT) + .where(new TypeParameter() {}, outputT); + } + + /** Generates a type token for {@code DoFn.InputProvider} given {@code InputT}. */ + private static TypeToken> inputProviderTypeOf( + TypeToken inputT) { + return new TypeToken>() {}.where( + new TypeParameter() {}, inputT); + } + + /** Generates a type token for {@code DoFn.OutputReceiver} given {@code OutputT}. */ + private static TypeToken> outputReceiverTypeOf( + TypeToken inputT) { + return new TypeToken>() {}.where( + new TypeParameter() {}, inputT); + } + + @VisibleForTesting + static DoFnSignature.ProcessElementMethod analyzeProcessElementMethod( + TypeToken fnClass, Method m, TypeToken inputT, TypeToken outputT) { + checkArgument( + void.class.equals(m.getReturnType()), "%s must have a void return type", format(m)); + checkArgument(!m.isVarArgs(), "%s must not have var args", format(m)); + + TypeToken processContextToken = doFnProcessContextTypeOf(inputT, outputT); + + Type[] params = m.getGenericParameterTypes(); + TypeToken contextToken = null; + if (params.length > 0) { + contextToken = fnClass.resolveType(params[0]); + } + checkArgument( + contextToken != null && contextToken.equals(processContextToken), + "%s must take a %s as its first argument", + format(m), + formatType(processContextToken)); + + List extraParameters = new ArrayList<>(); + TypeToken expectedInputProviderT = inputProviderTypeOf(inputT); + TypeToken expectedOutputReceiverT = outputReceiverTypeOf(outputT); + for (int i = 1; i < params.length; ++i) { + TypeToken param = fnClass.resolveType(params[i]); + Class rawType = param.getRawType(); + if (rawType.equals(BoundedWindow.class)) { + checkArgument( + !extraParameters.contains(DoFnSignature.ProcessElementMethod.Parameter.BOUNDED_WINDOW), + "Multiple BoundedWindow parameters in %s", + format(m)); + extraParameters.add(DoFnSignature.ProcessElementMethod.Parameter.BOUNDED_WINDOW); + } else if (rawType.equals(DoFn.InputProvider.class)) { + checkArgument( + !extraParameters.contains(DoFnSignature.ProcessElementMethod.Parameter.INPUT_PROVIDER), + "Multiple InputProvider parameters in %s", + format(m)); + checkArgument( + param.equals(expectedInputProviderT), + "Wrong type of InputProvider parameter for method %s: %s, should be %s", + format(m), + formatType(param), + formatType(expectedInputProviderT)); + extraParameters.add(DoFnSignature.ProcessElementMethod.Parameter.INPUT_PROVIDER); + } else if (rawType.equals(DoFn.OutputReceiver.class)) { + checkArgument( + !extraParameters.contains(DoFnSignature.ProcessElementMethod.Parameter.OUTPUT_RECEIVER), + "Multiple OutputReceiver parameters in %s", + format(m)); + checkArgument( + param.equals(expectedOutputReceiverT), + "Wrong type of OutputReceiver parameter for method %s: %s, should be %s", + format(m), + formatType(param), + formatType(expectedOutputReceiverT)); + extraParameters.add(DoFnSignature.ProcessElementMethod.Parameter.OUTPUT_RECEIVER); + } else { + List allowedParamTypes = + Arrays.asList(formatType(new TypeToken() {})); + checkArgument( + false, + "%s is not a valid context parameter for method %s. Should be one of %s", + formatType(param), + format(m), + allowedParamTypes); + } + } + + return DoFnSignature.ProcessElementMethod.create(m, extraParameters); + } + + @VisibleForTesting + static DoFnSignature.BundleMethod analyzeBundleMethod( + TypeToken fnToken, Method m, TypeToken inputT, TypeToken outputT) { + checkArgument( + void.class.equals(m.getReturnType()), "%s must have a void return type", format(m)); + checkArgument(!m.isVarArgs(), "%s must not have var args", format(m)); + + TypeToken expectedContextToken = doFnContextTypeOf(inputT, outputT); + + Type[] params = m.getGenericParameterTypes(); + checkArgument( + params.length == 1, + "%s must have a single argument of type %s", + format(m), + formatType(expectedContextToken)); + TypeToken contextToken = fnToken.resolveType(params[0]); + checkArgument( + contextToken.equals(expectedContextToken), + "Wrong type of context argument to %s: %s, must be %s", + format(m), + formatType(contextToken), + formatType(expectedContextToken)); + + return DoFnSignature.BundleMethod.create(m); + } + + private static DoFnSignature.LifecycleMethod analyzeLifecycleMethod(Method m) { + checkArgument( + void.class.equals(m.getReturnType()), "%s must have a void return type", format(m)); + checkArgument( + m.getGenericParameterTypes().length == 0, "%s must take zero arguments", format(m)); + return DoFnSignature.LifecycleMethod.create(m); + } + + private static Collection declaredMethodsWithAnnotation( + Class anno, Class startClass, Class stopClass) { + Collection matches = new ArrayList<>(); + + Class clazz = startClass; + LinkedHashSet> interfaces = new LinkedHashSet<>(); + + // First, find all declared methods on the startClass and parents (up to stopClass) + while (clazz != null && !clazz.equals(stopClass)) { + for (Method method : clazz.getDeclaredMethods()) { + if (method.isAnnotationPresent(anno)) { + matches.add(method); + } + } + + Collections.addAll(interfaces, clazz.getInterfaces()); + + clazz = clazz.getSuperclass(); + } + + // Now, iterate over all the discovered interfaces + for (Method method : ReflectHelpers.getClosureOfMethodsOnInterfaces(interfaces)) { + if (method.isAnnotationPresent(anno)) { + matches.add(method); + } + } + return matches; + } + + private static Method findAnnotatedMethod( + Class anno, Class fnClazz, boolean required) { + Collection matches = declaredMethodsWithAnnotation(anno, fnClazz, DoFn.class); + + if (matches.size() == 0) { + checkArgument( + !required, + "No method annotated with @%s found in %s", + anno.getSimpleName(), + fnClazz.getName()); + return null; + } + + // If we have at least one match, then either it should be the only match + // or it should be an extension of the other matches (which came from parent + // classes). + Method first = matches.iterator().next(); + for (Method other : matches) { + checkArgument( + first.getName().equals(other.getName()) + && Arrays.equals(first.getParameterTypes(), other.getParameterTypes()), + "Found multiple methods annotated with @%s. [%s] and [%s]", + anno.getSimpleName(), + format(first), + format(other)); + } + + // We need to be able to call it. We require it is public. + checkArgument( + (first.getModifiers() & Modifier.PUBLIC) != 0, "%s must be public", format(first)); + + // And make sure its not static. + checkArgument( + (first.getModifiers() & Modifier.STATIC) == 0, "%s must not be static", format(first)); + + return first; + } + + private static String format(Method m) { + return ReflectHelpers.CLASS_AND_METHOD_FORMATTER.apply(m); + } + + private static String formatType(TypeToken t) { + return ReflectHelpers.TYPE_SIMPLE_DESCRIPTION.apply(t.getType()); + } +} diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/package-info.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/package-info.java new file mode 100644 index 0000000000000..4df5209e6b392 --- /dev/null +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/reflect/package-info.java @@ -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. + */ +/** + * Defines reflection-based utilities for analyzing {@link org.apache.beam.sdk.transforms.DoFn}'s + * and creating {@link org.apache.beam.sdk.transforms.reflect.DoFnSignature}'s and + * {@link org.apache.beam.sdk.transforms.reflect.DoFnInvoker}'s from them. + */ +package org.apache.beam.sdk.transforms.reflect; diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/DoFnReflectorTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/DoFnReflectorTest.java deleted file mode 100644 index e05e5e2c667a3..0000000000000 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/DoFnReflectorTest.java +++ /dev/null @@ -1,822 +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.transforms; - -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertSame; -import static org.junit.Assert.assertTrue; - -import org.apache.beam.sdk.transforms.DoFn.Context; -import org.apache.beam.sdk.transforms.DoFn.ExtraContextFactory; -import org.apache.beam.sdk.transforms.DoFn.ProcessContext; -import org.apache.beam.sdk.transforms.DoFn.ProcessElement; -import org.apache.beam.sdk.transforms.DoFn.Setup; -import org.apache.beam.sdk.transforms.DoFn.Teardown; -import org.apache.beam.sdk.transforms.dofnreflector.DoFnReflectorTestHelper; -import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.UserCodeException; - -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; - -import java.lang.reflect.Method; - -/** - * Tests for {@link DoFnReflector}. - */ -@RunWith(JUnit4.class) -public class DoFnReflectorTest { - - /** - * A convenience struct holding flags that indicate whether a particular method was invoked. - */ - public static class Invocations { - public boolean wasProcessElementInvoked = false; - public boolean wasStartBundleInvoked = false; - public boolean wasFinishBundleInvoked = false; - public boolean wasSetupInvoked = false; - public boolean wasTeardownInvoked = false; - private final String name; - - public Invocations(String name) { - this.name = name; - } - } - - private DoFn fn; - - @Rule - public ExpectedException thrown = ExpectedException.none(); - - @Mock - private DoFn.ProcessContext mockContext; - @Mock - private BoundedWindow mockWindow; - @Mock - private DoFn.InputProvider mockInputProvider; - @Mock - private DoFn.OutputReceiver mockOutputReceiver; - - private ExtraContextFactory extraContextFactory; - - @Before - public void setUp() { - MockitoAnnotations.initMocks(this); - this.extraContextFactory = new ExtraContextFactory() { - @Override - public BoundedWindow window() { - return mockWindow; - } - - @Override - public DoFn.InputProvider inputProvider() { - return mockInputProvider; - } - - @Override - public DoFn.OutputReceiver outputReceiver() { - return mockOutputReceiver; - } - }; - } - - private DoFnReflector underTest(DoFn fn) { - this.fn = fn; - return DoFnReflector.of(fn.getClass()); - } - - private void checkInvokeProcessElementWorks( - DoFnReflector r, Invocations... invocations) throws Exception { - assertTrue("Need at least one invocation to check", invocations.length >= 1); - for (Invocations invocation : invocations) { - assertFalse("Should not yet have called processElement on " + invocation.name, - invocation.wasProcessElementInvoked); - } - r.bindInvoker(fn).invokeProcessElement(mockContext, extraContextFactory); - for (Invocations invocation : invocations) { - assertTrue("Should have called processElement on " + invocation.name, - invocation.wasProcessElementInvoked); - } - } - - private void checkInvokeStartBundleWorks( - DoFnReflector r, Invocations... invocations) throws Exception { - assertTrue("Need at least one invocation to check", invocations.length >= 1); - for (Invocations invocation : invocations) { - assertFalse("Should not yet have called startBundle on " + invocation.name, - invocation.wasStartBundleInvoked); - } - r.bindInvoker(fn).invokeStartBundle(mockContext, extraContextFactory); - for (Invocations invocation : invocations) { - assertTrue("Should have called startBundle on " + invocation.name, - invocation.wasStartBundleInvoked); - } - } - - private void checkInvokeFinishBundleWorks( - DoFnReflector r, Invocations... invocations) throws Exception { - assertTrue("Need at least one invocation to check", invocations.length >= 1); - for (Invocations invocation : invocations) { - assertFalse("Should not yet have called finishBundle on " + invocation.name, - invocation.wasFinishBundleInvoked); - } - r.bindInvoker(fn).invokeFinishBundle(mockContext, extraContextFactory); - for (Invocations invocation : invocations) { - assertTrue("Should have called finishBundle on " + invocation.name, - invocation.wasFinishBundleInvoked); - } - } - - private void checkInvokeSetupWorks(DoFnReflector r, Invocations... invocations) throws Exception { - assertTrue("Need at least one invocation to check", invocations.length >= 1); - for (Invocations invocation : invocations) { - assertFalse("Should not yet have called setup on " + invocation.name, - invocation.wasSetupInvoked); - } - r.bindInvoker(fn).invokeSetup(); - for (Invocations invocation : invocations) { - assertTrue("Should have called setup on " + invocation.name, - invocation.wasSetupInvoked); - } - } - - private void checkInvokeTeardownWorks(DoFnReflector r, Invocations... invocations) - throws Exception { - assertTrue("Need at least one invocation to check", invocations.length >= 1); - for (Invocations invocation : invocations) { - assertFalse("Should not yet have called teardown on " + invocation.name, - invocation.wasTeardownInvoked); - } - r.bindInvoker(fn).invokeTeardown(); - for (Invocations invocation : invocations) { - assertTrue("Should have called teardown on " + invocation.name, - invocation.wasTeardownInvoked); - } - } - - @Test - public void testDoFnWithNoExtraContext() throws Exception { - final Invocations invocations = new Invocations("AnonymousClass"); - DoFnReflector reflector = underTest(new DoFn() { - - @ProcessElement - public void processElement(ProcessContext c) - throws Exception { - invocations.wasProcessElementInvoked = true; - assertSame(c, mockContext); - } - }); - - assertFalse(reflector.usesSingleWindow()); - - checkInvokeProcessElementWorks(reflector, invocations); - } - - @Test - public void testDoFnInvokersReused() throws Exception { - // Ensures that we don't create a new Invoker class for every instance of the OldDoFn. - IdentityParent fn1 = new IdentityParent(); - IdentityParent fn2 = new IdentityParent(); - DoFnReflector reflector1 = underTest(fn1); - DoFnReflector reflector2 = underTest(fn2); - assertSame("DoFnReflector instances should be cached and reused for identical types", - reflector1, reflector2); - assertSame("Invoker classes should only be generated once for each type", - reflector1.bindInvoker(fn1).getClass(), - reflector2.bindInvoker(fn2).getClass()); - } - - interface InterfaceWithProcessElement { - @ProcessElement - void processElement(DoFn.ProcessContext c); - } - - interface LayersOfInterfaces extends InterfaceWithProcessElement {} - - private class IdentityUsingInterfaceWithProcessElement - extends DoFn - implements LayersOfInterfaces { - - private Invocations invocations = new Invocations("Named Class"); - - @Override - public void processElement(DoFn.ProcessContext c) { - invocations.wasProcessElementInvoked = true; - assertSame(c, mockContext); - } - } - - @Test - public void testDoFnWithProcessElementInterface() throws Exception { - IdentityUsingInterfaceWithProcessElement fn = new IdentityUsingInterfaceWithProcessElement(); - DoFnReflector reflector = underTest(fn); - assertFalse(reflector.usesSingleWindow()); - checkInvokeProcessElementWorks(reflector, fn.invocations); - } - - private class IdentityParent extends DoFn { - protected Invocations parentInvocations = new Invocations("IdentityParent"); - - @ProcessElement - public void process(ProcessContext c) { - parentInvocations.wasProcessElementInvoked = true; - assertSame(c, mockContext); - } - } - - private class IdentityChildWithoutOverride extends IdentityParent { - } - - private class IdentityChildWithOverride extends IdentityParent { - protected Invocations childInvocations = new Invocations("IdentityChildWithOverride"); - - @Override - public void process(DoFn.ProcessContext c) { - super.process(c); - childInvocations.wasProcessElementInvoked = true; - } - } - - @Test - public void testDoFnWithMethodInSuperclass() throws Exception { - IdentityChildWithoutOverride fn = new IdentityChildWithoutOverride(); - DoFnReflector reflector = underTest(fn); - assertFalse(reflector.usesSingleWindow()); - checkInvokeProcessElementWorks(reflector, fn.parentInvocations); - } - - @Test - public void testDoFnWithMethodInSubclass() throws Exception { - IdentityChildWithOverride fn = new IdentityChildWithOverride(); - DoFnReflector reflector = underTest(fn); - assertFalse(reflector.usesSingleWindow()); - checkInvokeProcessElementWorks(reflector, fn.parentInvocations, fn.childInvocations); - } - - @Test - public void testDoFnWithWindow() throws Exception { - final Invocations invocations = new Invocations("AnonymousClass"); - DoFnReflector reflector = underTest(new DoFn() { - - @ProcessElement - public void processElement(ProcessContext c, BoundedWindow w) - throws Exception { - invocations.wasProcessElementInvoked = true; - assertSame(c, mockContext); - assertSame(w, mockWindow); - } - }); - - assertTrue(reflector.usesSingleWindow()); - - checkInvokeProcessElementWorks(reflector, invocations); - } - - @Test - public void testDoFnWithOutputReceiver() throws Exception { - final Invocations invocations = new Invocations("AnonymousClass"); - DoFnReflector reflector = underTest(new DoFn() { - - @ProcessElement - public void processElement(ProcessContext c, DoFn.OutputReceiver o) - throws Exception { - invocations.wasProcessElementInvoked = true; - assertSame(c, mockContext); - assertSame(o, mockOutputReceiver); - } - }); - - assertFalse(reflector.usesSingleWindow()); - - checkInvokeProcessElementWorks(reflector, invocations); - } - - @Test - public void testDoFnWithInputProvider() throws Exception { - final Invocations invocations = new Invocations("AnonymousClass"); - DoFnReflector reflector = underTest(new DoFn() { - - @ProcessElement - public void processElement(ProcessContext c, DoFn.InputProvider i) - throws Exception { - invocations.wasProcessElementInvoked = true; - assertSame(c, mockContext); - assertSame(i, mockInputProvider); - } - }); - - assertFalse(reflector.usesSingleWindow()); - - checkInvokeProcessElementWorks(reflector, invocations); - } - - @Test - public void testDoFnWithStartBundle() throws Exception { - final Invocations invocations = new Invocations("AnonymousClass"); - DoFnReflector reflector = underTest(new DoFn() { - @ProcessElement - public void processElement(@SuppressWarnings("unused") ProcessContext c) {} - - @StartBundle - public void startBundle(Context c) { - invocations.wasStartBundleInvoked = true; - assertSame(c, mockContext); - } - - @FinishBundle - public void finishBundle(Context c) { - invocations.wasFinishBundleInvoked = true; - assertSame(c, mockContext); - } - }); - - checkInvokeStartBundleWorks(reflector, invocations); - checkInvokeFinishBundleWorks(reflector, invocations); - } - - @Test - public void testDoFnWithSetupTeardown() throws Exception { - final Invocations invocations = new Invocations("AnonymousClass"); - DoFnReflector reflector = underTest(new DoFn() { - @ProcessElement - public void processElement(@SuppressWarnings("unused") ProcessContext c) {} - - @StartBundle - public void startBundle(Context c) { - invocations.wasStartBundleInvoked = true; - assertSame(c, mockContext); - } - - @FinishBundle - public void finishBundle(Context c) { - invocations.wasFinishBundleInvoked = true; - assertSame(c, mockContext); - } - - @Setup - public void before() { - invocations.wasSetupInvoked = true; - } - - @Teardown - public void after() { - invocations.wasTeardownInvoked = true; - } - }); - - checkInvokeSetupWorks(reflector, invocations); - checkInvokeTeardownWorks(reflector, invocations); - } - - @Test - public void testNoProcessElement() throws Exception { - thrown.expect(IllegalStateException.class); - thrown.expectMessage("No method annotated with @ProcessElement found"); - thrown.expectMessage(getClass().getName() + "$"); - underTest(new DoFn() {}); - } - - @Test - public void testMultipleProcessElement() throws Exception { - thrown.expect(IllegalStateException.class); - thrown.expectMessage("Found multiple methods annotated with @ProcessElement"); - thrown.expectMessage("foo()"); - thrown.expectMessage("bar()"); - thrown.expectMessage(getClass().getName() + "$"); - underTest(new DoFn() { - @ProcessElement - public void foo() {} - - @ProcessElement - public void bar() {} - }); - } - - @Test - public void testMultipleStartBundleElement() throws Exception { - thrown.expect(IllegalStateException.class); - thrown.expectMessage("Found multiple methods annotated with @StartBundle"); - thrown.expectMessage("bar()"); - thrown.expectMessage("baz()"); - thrown.expectMessage(getClass().getName() + "$"); - underTest(new DoFn() { - @ProcessElement - public void foo() {} - - @StartBundle - public void bar() {} - - @StartBundle - public void baz() {} - }); - } - - @Test - public void testMultipleFinishBundleElement() throws Exception { - thrown.expect(IllegalStateException.class); - thrown.expectMessage("Found multiple methods annotated with @FinishBundle"); - thrown.expectMessage("bar()"); - thrown.expectMessage("baz()"); - thrown.expectMessage(getClass().getName() + "$"); - underTest(new DoFn() { - @ProcessElement - public void foo() {} - - @FinishBundle - public void bar() {} - - @FinishBundle - public void baz() {} - }); - } - - private static class PrivateDoFnClass extends DoFn { - final Invocations invocations = new Invocations(getClass().getName()); - - @ProcessElement - public void processThis(ProcessContext c) { - invocations.wasProcessElementInvoked = true; - } - } - - @Test - public void testLocalPrivateDoFnClass() throws Exception { - PrivateDoFnClass fn = new PrivateDoFnClass(); - DoFnReflector reflector = underTest(fn); - checkInvokeProcessElementWorks(reflector, fn.invocations); - } - - @Test - public void testStaticPackagePrivateDoFnClass() throws Exception { - Invocations invocations = new Invocations("StaticPackagePrivateDoFn"); - DoFnReflector reflector = - underTest(DoFnReflectorTestHelper.newStaticPackagePrivateDoFn(invocations)); - checkInvokeProcessElementWorks(reflector, invocations); - } - - @Test - public void testInnerPackagePrivateDoFnClass() throws Exception { - Invocations invocations = new Invocations("InnerPackagePrivateDoFn"); - DoFnReflector reflector = - underTest(new DoFnReflectorTestHelper().newInnerPackagePrivateDoFn(invocations)); - checkInvokeProcessElementWorks(reflector, invocations); - } - - @Test - public void testStaticPrivateDoFnClass() throws Exception { - Invocations invocations = new Invocations("StaticPrivateDoFn"); - DoFnReflector reflector = underTest(DoFnReflectorTestHelper.newStaticPrivateDoFn(invocations)); - checkInvokeProcessElementWorks(reflector, invocations); - } - - @Test - public void testInnerPrivateDoFnClass() throws Exception { - Invocations invocations = new Invocations("StaticInnerDoFn"); - DoFnReflector reflector = - underTest(new DoFnReflectorTestHelper().newInnerPrivateDoFn(invocations)); - checkInvokeProcessElementWorks(reflector, invocations); - } - - @Test - public void testAnonymousInnerDoFnInOtherPackage() throws Exception { - Invocations invocations = new Invocations("AnonymousInnerDoFnInOtherPackage"); - DoFnReflector reflector = - underTest(new DoFnReflectorTestHelper().newInnerAnonymousDoFn(invocations)); - checkInvokeProcessElementWorks(reflector, invocations); - } - - @Test - public void testStaticAnonymousDoFnInOtherPackage() throws Exception { - Invocations invocations = new Invocations("AnonymousStaticDoFnInOtherPackage"); - DoFnReflector reflector = - underTest(DoFnReflectorTestHelper.newStaticAnonymousDoFn(invocations)); - checkInvokeProcessElementWorks(reflector, invocations); - } - - @Test - public void testPrivateProcessElement() throws Exception { - thrown.expect(IllegalStateException.class); - thrown.expectMessage("process() must be public"); - thrown.expectMessage(getClass().getName() + "$"); - underTest(new DoFn() { - @ProcessElement - private void process() {} - }); - } - - @Test - public void testPrivateStartBundle() throws Exception { - thrown.expect(IllegalStateException.class); - thrown.expectMessage("startBundle() must be public"); - thrown.expectMessage(getClass().getName() + "$"); - underTest(new DoFn() { - @ProcessElement - public void processElement() {} - - @StartBundle - void startBundle() {} - }); - } - - @Test - public void testPrivateFinishBundle() throws Exception { - thrown.expect(IllegalStateException.class); - thrown.expectMessage("finishBundle() must be public"); - thrown.expectMessage(getClass().getName() + "$"); - underTest(new DoFn() { - @ProcessElement - public void processElement() {} - - @FinishBundle - void finishBundle() {} - }); - } - - @SuppressWarnings({"unused"}) - private void missingProcessContext() {} - - @Test - public void testMissingProcessContext() throws Exception { - thrown.expect(IllegalStateException.class); - thrown.expectMessage(getClass().getName() - + "#missingProcessContext() must take a ProcessContext as its first argument"); - - DoFnReflector.verifyProcessMethodArguments( - getClass().getDeclaredMethod("missingProcessContext")); - } - - @SuppressWarnings({"unused"}) - private void badProcessContext(String s) {} - - @Test - public void testBadProcessContextType() throws Exception { - thrown.expect(IllegalStateException.class); - thrown.expectMessage(getClass().getName() - + "#badProcessContext(String) must take a ProcessContext as its first argument"); - - DoFnReflector.verifyProcessMethodArguments( - getClass().getDeclaredMethod("badProcessContext", String.class)); - } - - @SuppressWarnings({"unused"}) - private void badExtraContext(DoFn.Context c, int n) {} - - @Test - public void testBadExtraContext() throws Exception { - thrown.expect(IllegalStateException.class); - thrown.expectMessage( - "int is not a valid context parameter for method " - + getClass().getName() + "#badExtraContext(Context, int). Should be one of ["); - - DoFnReflector.verifyBundleMethodArguments( - getClass().getDeclaredMethod("badExtraContext", Context.class, int.class)); - } - - @SuppressWarnings({"unused"}) - private void badExtraProcessContext( - DoFn.ProcessContext c, Integer n) {} - - @Test - public void testBadExtraProcessContextType() throws Exception { - thrown.expect(IllegalStateException.class); - thrown.expectMessage( - "Integer is not a valid context parameter for method " - + getClass().getName() + "#badExtraProcessContext(ProcessContext, Integer)" - + ". Should be one of [BoundedWindow]"); - - DoFnReflector.verifyProcessMethodArguments( - getClass().getDeclaredMethod("badExtraProcessContext", - ProcessContext.class, Integer.class)); - } - - @SuppressWarnings("unused") - private int badReturnType() { - return 0; - } - - @Test - public void testBadReturnType() throws Exception { - thrown.expect(IllegalStateException.class); - thrown.expectMessage(getClass().getName() + "#badReturnType() must have a void return type"); - - DoFnReflector.verifyProcessMethodArguments(getClass().getDeclaredMethod("badReturnType")); - } - - @SuppressWarnings("unused") - private void goodGenerics( - DoFn.ProcessContext c, - DoFn.InputProvider input, - DoFn.OutputReceiver output) {} - - @Test - public void testValidGenerics() throws Exception { - Method method = - getClass() - .getDeclaredMethod( - "goodGenerics", - DoFn.ProcessContext.class, - DoFn.InputProvider.class, - DoFn.OutputReceiver.class); - DoFnReflector.verifyProcessMethodArguments(method); - } - - @SuppressWarnings("unused") - private void goodWildcards( - DoFn.ProcessContext c, - DoFn.InputProvider input, - DoFn.OutputReceiver output) {} - - @Test - public void testGoodWildcards() throws Exception { - Method method = - getClass() - .getDeclaredMethod( - "goodWildcards", - DoFn.ProcessContext.class, - DoFn.InputProvider.class, - DoFn.OutputReceiver.class); - DoFnReflector.verifyProcessMethodArguments(method); - } - - @SuppressWarnings("unused") - private void goodBoundedWildcards( - DoFn.ProcessContext c, - DoFn.InputProvider input, - DoFn.OutputReceiver output) {} - - @Test - public void testGoodBoundedWildcards() throws Exception { - Method method = - getClass() - .getDeclaredMethod( - "goodBoundedWildcards", - DoFn.ProcessContext.class, - DoFn.InputProvider.class, - DoFn.OutputReceiver.class); - DoFnReflector.verifyProcessMethodArguments(method); - } - - @SuppressWarnings("unused") - private void goodTypeVariables( - DoFn.ProcessContext c, - DoFn.InputProvider input, - DoFn.OutputReceiver output) {} - - @Test - public void testGoodTypeVariables() throws Exception { - Method method = - getClass() - .getDeclaredMethod( - "goodTypeVariables", - DoFn.ProcessContext.class, - DoFn.InputProvider.class, - DoFn.OutputReceiver.class); - DoFnReflector.verifyProcessMethodArguments(method); - } - - @SuppressWarnings("unused") - private void badGenericTwoArgs( - DoFn.ProcessContext c, - DoFn.InputProvider input, - DoFn.OutputReceiver output) {} - - @Test - public void testBadGenericsTwoArgs() throws Exception { - Method method = - getClass() - .getDeclaredMethod( - "badGenericTwoArgs", - DoFn.ProcessContext.class, - DoFn.InputProvider.class, - DoFn.OutputReceiver.class); - - thrown.expect(IllegalStateException.class); - thrown.expectMessage("Incompatible generics in context parameter " - + "OutputReceiver " - + "for method " + getClass().getName() - + "#badGenericTwoArgs(ProcessContext, InputProvider, OutputReceiver). Should be " - + "OutputReceiver"); - - DoFnReflector.verifyProcessMethodArguments(method); - } - - @SuppressWarnings("unused") - private void badGenericWildCards( - DoFn.ProcessContext c, - DoFn.InputProvider input, - DoFn.OutputReceiver output) {} - - @Test - public void testBadGenericWildCards() throws Exception { - Method method = - getClass() - .getDeclaredMethod( - "badGenericWildCards", - DoFn.ProcessContext.class, - DoFn.InputProvider.class, - DoFn.OutputReceiver.class); - - thrown.expect(IllegalStateException.class); - thrown.expectMessage("Incompatible generics in context parameter " - + "OutputReceiver for method " - + getClass().getName() - + "#badGenericWildCards(ProcessContext, InputProvider, OutputReceiver). Should be " - + "OutputReceiver"); - - DoFnReflector.verifyProcessMethodArguments(method); - } - - @SuppressWarnings("unused") - private void badTypeVariables(DoFn.ProcessContext c, - DoFn.InputProvider input, DoFn.OutputReceiver output) {} - - @Test - public void testBadTypeVariables() throws Exception { - Method method = - getClass() - .getDeclaredMethod( - "badTypeVariables", - DoFn.ProcessContext.class, - DoFn.InputProvider.class, - DoFn.OutputReceiver.class); - - thrown.expect(IllegalStateException.class); - thrown.expectMessage("Incompatible generics in context parameter " - + "OutputReceiver for method " + getClass().getName() - + "#badTypeVariables(ProcessContext, InputProvider, OutputReceiver). Should be " - + "OutputReceiver"); - - DoFnReflector.verifyProcessMethodArguments(method); - } - - @Test - public void testProcessElementException() throws Exception { - DoFn fn = new DoFn() { - @ProcessElement - public void processElement(@SuppressWarnings("unused") ProcessContext c) { - throw new IllegalArgumentException("bogus"); - } - }; - - thrown.expect(UserCodeException.class); - thrown.expectMessage("bogus"); - DoFnReflector.of(fn.getClass()).bindInvoker(fn).invokeProcessElement(null, null); - } - - @Test - public void testStartBundleException() throws Exception { - DoFn fn = new DoFn() { - @StartBundle - public void startBundle(@SuppressWarnings("unused") Context c) { - throw new IllegalArgumentException("bogus"); - } - - @ProcessElement - public void processElement(@SuppressWarnings("unused") ProcessContext c) { - } - }; - - thrown.expect(UserCodeException.class); - thrown.expectMessage("bogus"); - DoFnReflector.of(fn.getClass()).bindInvoker(fn).invokeStartBundle(null, null); - } - - @Test - public void testFinishBundleException() throws Exception { - DoFn fn = new DoFn() { - @FinishBundle - public void finishBundle(@SuppressWarnings("unused") Context c) { - throw new IllegalArgumentException("bogus"); - } - - @ProcessElement - public void processElement(@SuppressWarnings("unused") ProcessContext c) { - } - }; - - thrown.expect(UserCodeException.class); - thrown.expectMessage("bogus"); - DoFnReflector.of(fn.getClass()).bindInvoker(fn).invokeFinishBundle(null, null); - } -} 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 604536bca80e1..3469223c38fe2 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 @@ -99,7 +99,7 @@ public void testFlattenPCollectionListThenParDo() { PCollection output = makePCollectionListOfStrings(p, inputs) .apply(Flatten.pCollections()) - .apply(ParDo.of(new IdentityFn(){})); + .apply(ParDo.of(new IdentityFn())); PAssert.that(output).containsInAnyOrder(flattenLists(inputs)); p.run(); @@ -152,7 +152,7 @@ public void testFlattenPCollectionListEmptyThenParDo() { PCollection output = PCollectionList.empty(p) .apply(Flatten.pCollections()).setCoder(StringUtf8Coder.of()) - .apply(ParDo.of(new IdentityFn(){})); + .apply(ParDo.of(new IdentityFn())); PAssert.that(output).empty(); p.run(); 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 new file mode 100644 index 0000000000000..7e756e245f917 --- /dev/null +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/reflect/DoFnInvokersTest.java @@ -0,0 +1,498 @@ +/* + * 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.transforms.reflect; + +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertSame; +import static org.junit.Assert.assertTrue; + +import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.transforms.windowing.BoundedWindow; +import org.apache.beam.sdk.util.UserCodeException; + +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.ExpectedException; +import org.mockito.Mock; +import org.mockito.MockitoAnnotations; + +/** Tests for {@link DoFnInvokers}. */ +public class DoFnInvokersTest { + /** A convenience struct holding flags that indicate whether a particular method was invoked. */ + public static class Invocations { + public boolean wasProcessElementInvoked = false; + public boolean wasStartBundleInvoked = false; + public boolean wasFinishBundleInvoked = false; + public boolean wasSetupInvoked = false; + public boolean wasTeardownInvoked = false; + private final String name; + + public Invocations(String name) { + this.name = name; + } + } + + @Rule public ExpectedException thrown = ExpectedException.none(); + + @Mock private DoFn.ProcessContext mockContext; + @Mock private BoundedWindow mockWindow; + @Mock private DoFn.InputProvider mockInputProvider; + @Mock private DoFn.OutputReceiver mockOutputReceiver; + + private DoFn.ExtraContextFactory extraContextFactory; + + @Before + public void setUp() { + MockitoAnnotations.initMocks(this); + this.extraContextFactory = + new DoFn.ExtraContextFactory() { + @Override + public BoundedWindow window() { + return mockWindow; + } + + @Override + public DoFn.InputProvider inputProvider() { + return mockInputProvider; + } + + @Override + public DoFn.OutputReceiver outputReceiver() { + return mockOutputReceiver; + } + }; + } + + private void checkInvokeProcessElementWorks(DoFn fn, Invocations... invocations) + throws Exception { + assertTrue("Need at least one invocation to check", invocations.length >= 1); + for (Invocations invocation : invocations) { + assertFalse( + "Should not yet have called processElement on " + invocation.name, + invocation.wasProcessElementInvoked); + } + DoFnInvokers.INSTANCE + .newByteBuddyInvoker(fn) + .invokeProcessElement(mockContext, extraContextFactory); + for (Invocations invocation : invocations) { + assertTrue( + "Should have called processElement on " + invocation.name, + invocation.wasProcessElementInvoked); + } + } + + private void checkInvokeStartBundleWorks(DoFn fn, Invocations... invocations) + throws Exception { + assertTrue("Need at least one invocation to check", invocations.length >= 1); + for (Invocations invocation : invocations) { + assertFalse( + "Should not yet have called startBundle on " + invocation.name, + invocation.wasStartBundleInvoked); + } + DoFnInvokers.INSTANCE.newByteBuddyInvoker(fn).invokeStartBundle(mockContext); + for (Invocations invocation : invocations) { + assertTrue( + "Should have called startBundle on " + invocation.name, invocation.wasStartBundleInvoked); + } + } + + private void checkInvokeFinishBundleWorks(DoFn fn, Invocations... invocations) + throws Exception { + assertTrue("Need at least one invocation to check", invocations.length >= 1); + for (Invocations invocation : invocations) { + assertFalse( + "Should not yet have called finishBundle on " + invocation.name, + invocation.wasFinishBundleInvoked); + } + DoFnInvokers.INSTANCE.newByteBuddyInvoker(fn).invokeFinishBundle(mockContext); + for (Invocations invocation : invocations) { + assertTrue( + "Should have called finishBundle on " + invocation.name, + invocation.wasFinishBundleInvoked); + } + } + + private void checkInvokeSetupWorks(DoFn fn, Invocations... invocations) + throws Exception { + assertTrue("Need at least one invocation to check", invocations.length >= 1); + for (Invocations invocation : invocations) { + assertFalse( + "Should not yet have called setup on " + invocation.name, invocation.wasSetupInvoked); + } + DoFnInvokers.INSTANCE.newByteBuddyInvoker(fn).invokeSetup(); + for (Invocations invocation : invocations) { + assertTrue("Should have called setup on " + invocation.name, invocation.wasSetupInvoked); + } + } + + private void checkInvokeTeardownWorks(DoFn fn, Invocations... invocations) + throws Exception { + assertTrue("Need at least one invocation to check", invocations.length >= 1); + for (Invocations invocation : invocations) { + assertFalse( + "Should not yet have called teardown on " + invocation.name, + invocation.wasTeardownInvoked); + } + DoFnInvokers.INSTANCE.newByteBuddyInvoker(fn).invokeTeardown(); + for (Invocations invocation : invocations) { + assertTrue( + "Should have called teardown on " + invocation.name, invocation.wasTeardownInvoked); + } + } + + @Test + public void testDoFnWithNoExtraContext() throws Exception { + final Invocations invocations = new Invocations("AnonymousClass"); + DoFn fn = + new DoFn() { + @ProcessElement + public void processElement(ProcessContext c) throws Exception { + invocations.wasProcessElementInvoked = true; + assertSame(c, mockContext); + } + }; + + assertFalse( + DoFnSignatures.INSTANCE + .getOrParseSignature(fn.getClass()) + .processElement() + .usesSingleWindow()); + + checkInvokeProcessElementWorks(fn, invocations); + } + + @Test + public void testDoFnInvokersReused() throws Exception { + // Ensures that we don't create a new Invoker class for every instance of the DoFn. + IdentityParent fn1 = new IdentityParent(); + IdentityParent fn2 = new IdentityParent(); + assertSame( + "Invoker classes should only be generated once for each type", + DoFnInvokers.INSTANCE.newByteBuddyInvoker(fn1).getClass(), + DoFnInvokers.INSTANCE.newByteBuddyInvoker(fn2).getClass()); + } + + interface InterfaceWithProcessElement { + @DoFn.ProcessElement + void processElement(DoFn.ProcessContext c); + } + + interface LayersOfInterfaces extends InterfaceWithProcessElement {} + + private class IdentityUsingInterfaceWithProcessElement extends DoFn + implements LayersOfInterfaces { + + private Invocations invocations = new Invocations("Named Class"); + + @Override + public void processElement(DoFn.ProcessContext c) { + invocations.wasProcessElementInvoked = true; + assertSame(c, mockContext); + } + } + + @Test + public void testDoFnWithProcessElementInterface() throws Exception { + IdentityUsingInterfaceWithProcessElement fn = new IdentityUsingInterfaceWithProcessElement(); + assertFalse( + DoFnSignatures.INSTANCE + .getOrParseSignature(fn.getClass()) + .processElement() + .usesSingleWindow()); + checkInvokeProcessElementWorks(fn, fn.invocations); + } + + private class IdentityParent extends DoFn { + protected Invocations parentInvocations = new Invocations("IdentityParent"); + + @ProcessElement + public void process(ProcessContext c) { + parentInvocations.wasProcessElementInvoked = true; + assertSame(c, mockContext); + } + } + + private class IdentityChildWithoutOverride extends IdentityParent {} + + private class IdentityChildWithOverride extends IdentityParent { + protected Invocations childInvocations = new Invocations("IdentityChildWithOverride"); + + @Override + public void process(DoFn.ProcessContext c) { + super.process(c); + childInvocations.wasProcessElementInvoked = true; + } + } + + @Test + public void testDoFnWithMethodInSuperclass() throws Exception { + IdentityChildWithoutOverride fn = new IdentityChildWithoutOverride(); + assertFalse( + DoFnSignatures.INSTANCE + .getOrParseSignature(fn.getClass()) + .processElement() + .usesSingleWindow()); + checkInvokeProcessElementWorks(fn, fn.parentInvocations); + } + + @Test + public void testDoFnWithMethodInSubclass() throws Exception { + IdentityChildWithOverride fn = new IdentityChildWithOverride(); + assertFalse( + DoFnSignatures.INSTANCE + .getOrParseSignature(fn.getClass()) + .processElement() + .usesSingleWindow()); + checkInvokeProcessElementWorks(fn, fn.parentInvocations, fn.childInvocations); + } + + @Test + public void testDoFnWithWindow() throws Exception { + final Invocations invocations = new Invocations("AnonymousClass"); + DoFn fn = + new DoFn() { + @ProcessElement + public void processElement(ProcessContext c, BoundedWindow w) throws Exception { + invocations.wasProcessElementInvoked = true; + assertSame(c, mockContext); + assertSame(w, mockWindow); + } + }; + + assertTrue( + DoFnSignatures.INSTANCE + .getOrParseSignature(fn.getClass()) + .processElement() + .usesSingleWindow()); + + checkInvokeProcessElementWorks(fn, invocations); + } + + @Test + public void testDoFnWithOutputReceiver() throws Exception { + final Invocations invocations = new Invocations("AnonymousClass"); + DoFn fn = + new DoFn() { + @ProcessElement + public void processElement(ProcessContext c, OutputReceiver o) throws Exception { + invocations.wasProcessElementInvoked = true; + assertSame(c, mockContext); + assertSame(o, mockOutputReceiver); + } + }; + + assertFalse( + DoFnSignatures.INSTANCE + .getOrParseSignature(fn.getClass()) + .processElement() + .usesSingleWindow()); + + checkInvokeProcessElementWorks(fn, invocations); + } + + @Test + public void testDoFnWithInputProvider() throws Exception { + final Invocations invocations = new Invocations("AnonymousClass"); + DoFn fn = + new DoFn() { + @ProcessElement + public void processElement(ProcessContext c, InputProvider i) throws Exception { + invocations.wasProcessElementInvoked = true; + assertSame(c, mockContext); + assertSame(i, mockInputProvider); + } + }; + + assertFalse( + DoFnSignatures.INSTANCE + .getOrParseSignature(fn.getClass()) + .processElement() + .usesSingleWindow()); + + checkInvokeProcessElementWorks(fn, invocations); + } + + @Test + public void testDoFnWithStartBundle() throws Exception { + final Invocations invocations = new Invocations("AnonymousClass"); + DoFn fn = + new DoFn() { + @ProcessElement + public void processElement(@SuppressWarnings("unused") ProcessContext c) {} + + @StartBundle + public void startBundle(Context c) { + invocations.wasStartBundleInvoked = true; + assertSame(c, mockContext); + } + + @FinishBundle + public void finishBundle(Context c) { + invocations.wasFinishBundleInvoked = true; + assertSame(c, mockContext); + } + }; + + checkInvokeStartBundleWorks(fn, invocations); + checkInvokeFinishBundleWorks(fn, invocations); + } + + @Test + public void testDoFnWithSetupTeardown() throws Exception { + final Invocations invocations = new Invocations("AnonymousClass"); + DoFn fn = + new DoFn() { + @ProcessElement + public void processElement(@SuppressWarnings("unused") ProcessContext c) {} + + @StartBundle + public void startBundle(Context c) { + invocations.wasStartBundleInvoked = true; + assertSame(c, mockContext); + } + + @FinishBundle + public void finishBundle(Context c) { + invocations.wasFinishBundleInvoked = true; + assertSame(c, mockContext); + } + + @Setup + public void before() { + invocations.wasSetupInvoked = true; + } + + @Teardown + public void after() { + invocations.wasTeardownInvoked = true; + } + }; + + checkInvokeSetupWorks(fn, invocations); + checkInvokeTeardownWorks(fn, invocations); + } + + private static class PrivateDoFnClass extends DoFn { + final Invocations invocations = new Invocations(getClass().getName()); + + @ProcessElement + public void processThis(ProcessContext c) { + invocations.wasProcessElementInvoked = true; + } + } + + @Test + public void testLocalPrivateDoFnClass() throws Exception { + PrivateDoFnClass fn = new PrivateDoFnClass(); + checkInvokeProcessElementWorks(fn, fn.invocations); + } + + @Test + public void testStaticPackagePrivateDoFnClass() throws Exception { + Invocations invocations = new Invocations("StaticPackagePrivateDoFn"); + checkInvokeProcessElementWorks( + DoFnInvokersTestHelper.newStaticPackagePrivateDoFn(invocations), invocations); + } + + @Test + public void testInnerPackagePrivateDoFnClass() throws Exception { + Invocations invocations = new Invocations("InnerPackagePrivateDoFn"); + checkInvokeProcessElementWorks( + new DoFnInvokersTestHelper().newInnerPackagePrivateDoFn(invocations), invocations); + } + + @Test + public void testStaticPrivateDoFnClass() throws Exception { + Invocations invocations = new Invocations("StaticPrivateDoFn"); + checkInvokeProcessElementWorks( + DoFnInvokersTestHelper.newStaticPrivateDoFn(invocations), invocations); + } + + @Test + public void testInnerPrivateDoFnClass() throws Exception { + Invocations invocations = new Invocations("StaticInnerDoFn"); + checkInvokeProcessElementWorks( + new DoFnInvokersTestHelper().newInnerPrivateDoFn(invocations), invocations); + } + + @Test + public void testAnonymousInnerDoFnInOtherPackage() throws Exception { + Invocations invocations = new Invocations("AnonymousInnerDoFnInOtherPackage"); + checkInvokeProcessElementWorks( + new DoFnInvokersTestHelper().newInnerAnonymousDoFn(invocations), invocations); + } + + @Test + public void testStaticAnonymousDoFnInOtherPackage() throws Exception { + Invocations invocations = new Invocations("AnonymousStaticDoFnInOtherPackage"); + checkInvokeProcessElementWorks( + DoFnInvokersTestHelper.newStaticAnonymousDoFn(invocations), invocations); + } + + @Test + public void testProcessElementException() throws Exception { + DoFn fn = + new DoFn() { + @ProcessElement + public void processElement(@SuppressWarnings("unused") ProcessContext c) { + throw new IllegalArgumentException("bogus"); + } + }; + + thrown.expect(UserCodeException.class); + thrown.expectMessage("bogus"); + DoFnInvokers.INSTANCE.newByteBuddyInvoker(fn).invokeProcessElement(null, null); + } + + @Test + public void testStartBundleException() throws Exception { + DoFn fn = + new DoFn() { + @StartBundle + public void startBundle(@SuppressWarnings("unused") Context c) { + throw new IllegalArgumentException("bogus"); + } + + @ProcessElement + public void processElement(@SuppressWarnings("unused") ProcessContext c) {} + }; + + thrown.expect(UserCodeException.class); + thrown.expectMessage("bogus"); + DoFnInvokers.INSTANCE.newByteBuddyInvoker(fn).invokeStartBundle(null); + } + + @Test + public void testFinishBundleException() throws Exception { + DoFn fn = + new DoFn() { + @FinishBundle + public void finishBundle(@SuppressWarnings("unused") Context c) { + throw new IllegalArgumentException("bogus"); + } + + @ProcessElement + public void processElement(@SuppressWarnings("unused") ProcessContext c) {} + }; + + thrown.expect(UserCodeException.class); + thrown.expectMessage("bogus"); + DoFnInvokers.INSTANCE.newByteBuddyInvoker(fn).invokeFinishBundle(null); + } +} diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/dofnreflector/DoFnReflectorTestHelper.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/reflect/DoFnInvokersTestHelper.java similarity index 93% rename from sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/dofnreflector/DoFnReflectorTestHelper.java rename to sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/reflect/DoFnInvokersTestHelper.java index 90fba12b9599d..7bfdddc052262 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/dofnreflector/DoFnReflectorTestHelper.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/reflect/DoFnInvokersTestHelper.java @@ -15,16 +15,16 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.sdk.transforms.dofnreflector; +package org.apache.beam.sdk.transforms.reflect; import org.apache.beam.sdk.transforms.DoFn; -import org.apache.beam.sdk.transforms.DoFnReflectorTest.Invocations; +import org.apache.beam.sdk.transforms.reflect.DoFnInvokersTest.Invocations; /** - * Test helper for DoFnReflectorTest, which needs to test package-private access + * Test helper for {@link DoFnInvokersTest}, which needs to test package-private access * to DoFns in other packages. */ -public class DoFnReflectorTestHelper { +public class DoFnInvokersTestHelper { private static class StaticPrivateDoFn extends DoFn { final Invocations invocations; 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 new file mode 100644 index 0000000000000..1a26df22cc34e --- /dev/null +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/reflect/DoFnSignaturesTest.java @@ -0,0 +1,371 @@ +/* + * 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.transforms.reflect; + +import org.apache.beam.sdk.transforms.DoFn; + +import com.google.common.reflect.TypeToken; + +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.ExpectedException; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +import java.lang.reflect.Method; +import java.util.List; + +/** Tests for {@link DoFnSignatures}. */ +@RunWith(JUnit4.class) +public class DoFnSignaturesTest { + + @Rule public ExpectedException thrown = ExpectedException.none(); + + private static class FakeDoFn extends DoFn {} + + @SuppressWarnings({"unused"}) + private void missingProcessContext() {} + + @Test + public void testMissingProcessContext() throws Exception { + thrown.expect(IllegalArgumentException.class); + thrown.expectMessage( + getClass().getName() + + "#missingProcessContext() must take a ProcessContext<> as its first argument"); + + DoFnSignatures.analyzeProcessElementMethod( + TypeToken.of(FakeDoFn.class), + getClass().getDeclaredMethod("missingProcessContext"), + TypeToken.of(Integer.class), + TypeToken.of(String.class)); + } + + @SuppressWarnings({"unused"}) + private void badProcessContext(String s) {} + + @Test + public void testBadProcessContextType() throws Exception { + thrown.expect(IllegalArgumentException.class); + thrown.expectMessage( + getClass().getName() + + "#badProcessContext(String) must take a ProcessContext<> as its first argument"); + + DoFnSignatures.analyzeProcessElementMethod( + TypeToken.of(FakeDoFn.class), + getClass().getDeclaredMethod("badProcessContext", String.class), + TypeToken.of(Integer.class), + TypeToken.of(String.class)); + } + + @SuppressWarnings({"unused"}) + private void badExtraContext(DoFn.Context c, int n) {} + + @Test + public void testBadExtraContext() throws Exception { + thrown.expect(IllegalArgumentException.class); + thrown.expectMessage( + getClass().getName() + + "#badExtraContext(Context, int) must have a single argument of type Context"); + + DoFnSignatures.analyzeBundleMethod( + TypeToken.of(FakeDoFn.class), + getClass().getDeclaredMethod("badExtraContext", DoFn.Context.class, int.class), + TypeToken.of(Integer.class), + TypeToken.of(String.class)); + } + + @SuppressWarnings({"unused"}) + private void badExtraProcessContext(DoFn.ProcessContext c, Integer n) {} + + @Test + public void testBadExtraProcessContextType() throws Exception { + thrown.expect(IllegalArgumentException.class); + thrown.expectMessage( + "Integer is not a valid context parameter for method " + + getClass().getName() + + "#badExtraProcessContext(ProcessContext, Integer)" + + ". Should be one of [BoundedWindow]"); + + DoFnSignatures.analyzeProcessElementMethod( + TypeToken.of(FakeDoFn.class), + getClass() + .getDeclaredMethod("badExtraProcessContext", DoFn.ProcessContext.class, Integer.class), + TypeToken.of(Integer.class), + TypeToken.of(String.class)); + } + + @SuppressWarnings("unused") + private int badReturnType() { + return 0; + } + + @Test + public void testBadReturnType() throws Exception { + thrown.expect(IllegalArgumentException.class); + thrown.expectMessage(getClass().getName() + "#badReturnType() must have a void return type"); + + DoFnSignatures.analyzeProcessElementMethod( + TypeToken.of(FakeDoFn.class), + getClass().getDeclaredMethod("badReturnType"), + TypeToken.of(Integer.class), + TypeToken.of(String.class)); + } + + @SuppressWarnings("unused") + private void goodConcreteTypes( + DoFn.ProcessContext c, + DoFn.InputProvider input, + DoFn.OutputReceiver output) {} + + @Test + public void testGoodConcreteTypes() throws Exception { + Method method = + getClass() + .getDeclaredMethod( + "goodConcreteTypes", + DoFn.ProcessContext.class, + DoFn.InputProvider.class, + DoFn.OutputReceiver.class); + DoFnSignatures.analyzeProcessElementMethod( + TypeToken.of(FakeDoFn.class), + method, + TypeToken.of(Integer.class), + TypeToken.of(String.class)); + } + + private static class GoodTypeVariables extends DoFn { + @ProcessElement + @SuppressWarnings("unused") + public void goodTypeVariables( + DoFn.ProcessContext c, + DoFn.InputProvider input, + DoFn.OutputReceiver output) {} + } + + @Test + public void testGoodTypeVariables() throws Exception { + DoFnSignatures.INSTANCE.getOrParseSignature(GoodTypeVariables.class); + } + + private static class IdentityFn extends DoFn { + @ProcessElement + @SuppressWarnings("unused") + public void processElement(ProcessContext c, InputProvider input, OutputReceiver output) { + c.output(c.element()); + } + } + + private static class IdentityListFn extends IdentityFn> {} + + @Test + public void testIdentityFnApplied() throws Exception { + DoFnSignatures.INSTANCE.getOrParseSignature(new IdentityFn() {}.getClass()); + } + + @SuppressWarnings("unused") + private void badGenericTwoArgs( + DoFn.ProcessContext c, + DoFn.InputProvider input, + DoFn.OutputReceiver output) {} + + @Test + public void testBadGenericsTwoArgs() throws Exception { + Method method = + getClass() + .getDeclaredMethod( + "badGenericTwoArgs", + DoFn.ProcessContext.class, + DoFn.InputProvider.class, + DoFn.OutputReceiver.class); + + thrown.expect(IllegalArgumentException.class); + thrown.expectMessage( + "Wrong type of OutputReceiver parameter " + + "for method " + + getClass().getName() + + "#badGenericTwoArgs(ProcessContext, InputProvider, OutputReceiver): " + + "OutputReceiver, should be " + + "OutputReceiver"); + + DoFnSignatures.analyzeProcessElementMethod( + TypeToken.of(FakeDoFn.class), + method, + TypeToken.of(Integer.class), + TypeToken.of(String.class)); + } + + @SuppressWarnings("unused") + private void badGenericWildCards( + DoFn.ProcessContext c, + DoFn.InputProvider input, + DoFn.OutputReceiver output) {} + + @Test + public void testBadGenericWildCards() throws Exception { + Method method = + getClass() + .getDeclaredMethod( + "badGenericWildCards", + DoFn.ProcessContext.class, + DoFn.InputProvider.class, + DoFn.OutputReceiver.class); + + thrown.expect(IllegalArgumentException.class); + thrown.expectMessage( + "Wrong type of OutputReceiver parameter for method " + + getClass().getName() + + "#badGenericWildCards(ProcessContext, InputProvider, OutputReceiver): " + + "OutputReceiver, should be " + + "OutputReceiver"); + + DoFnSignatures.analyzeProcessElementMethod( + TypeToken.of(FakeDoFn.class), + method, + TypeToken.of(Integer.class), + TypeToken.of(String.class)); + } + + static class BadTypeVariables extends DoFn { + @ProcessElement + @SuppressWarnings("unused") + public void badTypeVariables( + DoFn.ProcessContext c, + DoFn.InputProvider input, + DoFn.OutputReceiver output) {} + } + + @Test + public void testBadTypeVariables() throws Exception { + thrown.expect(IllegalArgumentException.class); + thrown.expectMessage( + "Wrong type of OutputReceiver parameter for method " + + BadTypeVariables.class.getName() + + "#badTypeVariables(ProcessContext, InputProvider, OutputReceiver): " + + "OutputReceiver, should be " + + "OutputReceiver"); + + DoFnSignatures.INSTANCE.getOrParseSignature(BadTypeVariables.class); + } + + @Test + public void testNoProcessElement() throws Exception { + thrown.expect(IllegalArgumentException.class); + thrown.expectMessage("No method annotated with @ProcessElement found"); + thrown.expectMessage(getClass().getName() + "$"); + DoFnSignatures.INSTANCE.getOrParseSignature(new DoFn() {}.getClass()); + } + + @Test + public void testMultipleProcessElement() throws Exception { + thrown.expect(IllegalArgumentException.class); + thrown.expectMessage("Found multiple methods annotated with @ProcessElement"); + thrown.expectMessage("foo()"); + thrown.expectMessage("bar()"); + thrown.expectMessage(getClass().getName() + "$"); + DoFnSignatures.INSTANCE.getOrParseSignature( + new DoFn() { + @ProcessElement + public void foo() {} + + @ProcessElement + public void bar() {} + }.getClass()); + } + + @Test + public void testMultipleStartBundleElement() throws Exception { + thrown.expect(IllegalArgumentException.class); + thrown.expectMessage("Found multiple methods annotated with @StartBundle"); + thrown.expectMessage("bar()"); + thrown.expectMessage("baz()"); + thrown.expectMessage(getClass().getName() + "$"); + DoFnSignatures.INSTANCE.getOrParseSignature( + new DoFn() { + @ProcessElement + public void foo() {} + + @StartBundle + public void bar() {} + + @StartBundle + public void baz() {} + }.getClass()); + } + + @Test + public void testMultipleFinishBundleMethods() throws Exception { + thrown.expect(IllegalArgumentException.class); + thrown.expectMessage("Found multiple methods annotated with @FinishBundle"); + thrown.expectMessage("bar(Context)"); + thrown.expectMessage("baz(Context)"); + thrown.expectMessage(getClass().getName() + "$"); + DoFnSignatures.INSTANCE.getOrParseSignature( + new DoFn() { + @ProcessElement + public void foo(ProcessContext context) {} + + @FinishBundle + public void bar(Context context) {} + + @FinishBundle + public void baz(Context context) {} + }.getClass()); + } + + @Test + public void testPrivateProcessElement() throws Exception { + thrown.expect(IllegalArgumentException.class); + thrown.expectMessage("process() must be public"); + thrown.expectMessage(getClass().getName() + "$"); + DoFnSignatures.INSTANCE.getOrParseSignature( + new DoFn() { + @ProcessElement + private void process() {} + }.getClass()); + } + + @Test + public void testPrivateStartBundle() throws Exception { + thrown.expect(IllegalArgumentException.class); + thrown.expectMessage("startBundle() must be public"); + thrown.expectMessage(getClass().getName() + "$"); + DoFnSignatures.INSTANCE.getOrParseSignature( + new DoFn() { + @ProcessElement + public void processElement() {} + + @StartBundle + void startBundle() {} + }.getClass()); + } + + @Test + public void testPrivateFinishBundle() throws Exception { + thrown.expect(IllegalArgumentException.class); + thrown.expectMessage("finishBundle() must be public"); + thrown.expectMessage(getClass().getName() + "$"); + DoFnSignatures.INSTANCE.getOrParseSignature( + new DoFn() { + @ProcessElement + public void processElement() {} + + @FinishBundle + void finishBundle() {} + }.getClass()); + } +} diff --git a/sdks/java/microbenchmarks/src/main/java/org/apache/beam/sdk/microbenchmarks/transforms/DoFnReflectorBenchmark.java b/sdks/java/microbenchmarks/src/main/java/org/apache/beam/sdk/microbenchmarks/transforms/DoFnInvokersBenchmark.java similarity index 83% rename from sdks/java/microbenchmarks/src/main/java/org/apache/beam/sdk/microbenchmarks/transforms/DoFnReflectorBenchmark.java rename to sdks/java/microbenchmarks/src/main/java/org/apache/beam/sdk/microbenchmarks/transforms/DoFnInvokersBenchmark.java index 91ecd162d8751..a574ed83dc427 100644 --- a/sdks/java/microbenchmarks/src/main/java/org/apache/beam/sdk/microbenchmarks/transforms/DoFnReflectorBenchmark.java +++ b/sdks/java/microbenchmarks/src/main/java/org/apache/beam/sdk/microbenchmarks/transforms/DoFnInvokersBenchmark.java @@ -22,15 +22,15 @@ import org.apache.beam.sdk.transforms.Combine.CombineFn; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.DoFn.ExtraContextFactory; -import org.apache.beam.sdk.transforms.DoFnReflector; -import org.apache.beam.sdk.transforms.DoFnReflector.DoFnInvoker; +import org.apache.beam.sdk.transforms.DoFnAdapters; import org.apache.beam.sdk.transforms.OldDoFn; +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.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; import org.openjdk.jmh.annotations.Benchmark; import org.openjdk.jmh.annotations.Fork; @@ -40,36 +40,33 @@ import org.openjdk.jmh.annotations.Warmup; /** - * Benchmarks for {@link OldDoFn} and {@link DoFn} invocations, specifically - * for measuring the overhead of {@link DoFnReflector}. + * Benchmarks for {@link OldDoFn} and {@link DoFn} invocations, specifically for measuring the + * overhead of {@link DoFnInvokers}. */ @State(Scope.Benchmark) @Fork(1) @Warmup(iterations = 5) -public class DoFnReflectorBenchmark { +public class DoFnInvokersBenchmark { private static final String ELEMENT = "some string to use for testing"; private OldDoFn oldDoFn = new UpperCaseOldDoFn(); private DoFn doFn = new UpperCaseDoFn(); - private StubOldDoFnProcessContext stubOldDoFnContext = new StubOldDoFnProcessContext(oldDoFn, - ELEMENT); - private StubDoFnProcessContext stubDoFnContext = - new StubDoFnProcessContext(doFn, ELEMENT); + private StubOldDoFnProcessContext stubOldDoFnContext = + new StubOldDoFnProcessContext(oldDoFn, ELEMENT); + private StubDoFnProcessContext stubDoFnContext = new StubDoFnProcessContext(doFn, ELEMENT); private ExtraContextFactory extraContextFactory = new DoFn.FakeExtraContextFactory<>(); - private DoFnReflector doFnReflector; private OldDoFn adaptedDoFnWithContext; private DoFnInvoker invoker; @Setup public void setUp() { - doFnReflector = DoFnReflector.of(doFn.getClass()); - adaptedDoFnWithContext = doFnReflector.toDoFn(doFn); - invoker = doFnReflector.bindInvoker(doFn); + adaptedDoFnWithContext = DoFnAdapters.toOldDoFn(doFn); + invoker = DoFnInvokers.INSTANCE.newByteBuddyInvoker(doFn); } @Benchmark @@ -162,24 +159,21 @@ public void outputWithTimestamp(String output, Instant timestamp) { } @Override - public void sideOutput(TupleTag tag, T output) { - } + public void sideOutput(TupleTag tag, T output) {} @Override - public void sideOutputWithTimestamp(TupleTag tag, T output, Instant timestamp) { - } + public void sideOutputWithTimestamp(TupleTag tag, T output, Instant timestamp) {} @Override - protected Aggregator - createAggregatorInternal(String name, CombineFn combiner) { + protected Aggregator createAggregatorInternal( + String name, CombineFn combiner) { return null; } } - private static class StubDoFnProcessContext - extends DoFn.ProcessContext { + private static class StubDoFnProcessContext extends DoFn.ProcessContext { private final String element; - private String output; + private String output; public StubDoFnProcessContext(DoFn fn, String element) { fn.super(); @@ -222,11 +216,9 @@ public void outputWithTimestamp(String output, Instant timestamp) { } @Override - public void sideOutput(TupleTag tag, T output) { - } + public void sideOutput(TupleTag tag, T output) {} @Override - public void sideOutputWithTimestamp(TupleTag tag, T output, Instant timestamp) { - } + public void sideOutputWithTimestamp(TupleTag tag, T output, Instant timestamp) {} } } From da638b6ed31a74a97cbdb9a44f26df7bc60e2f78 Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Wed, 27 Jul 2016 14:23:15 -0700 Subject: [PATCH 031/346] Replace ParDo with simpler transforms where possible There are a number of places in the Java SDK where we use ParDo.of(DoFn) when MapElements or other higher-level composites are applicable and readable. This change alters a number of those. --- .../apache/beam/sdk/transforms/Combine.java | 28 +++++++++---------- .../org/apache/beam/sdk/transforms/Count.java | 8 +++--- .../beam/sdk/transforms/FlatMapElements.java | 4 +-- .../apache/beam/sdk/transforms/Flatten.java | 12 ++++---- .../org/apache/beam/sdk/transforms/Keys.java | 8 +++--- .../apache/beam/sdk/transforms/KvSwap.java | 9 +++--- .../beam/sdk/transforms/MapElements.java | 16 ++++++++--- .../beam/sdk/transforms/RemoveDuplicates.java | 8 +++--- .../apache/beam/sdk/transforms/Values.java | 8 +++--- .../apache/beam/sdk/transforms/WithKeys.java | 9 +++--- .../beam/sdk/transforms/windowing/Window.java | 11 ++++---- .../org/apache/beam/sdk/PipelineTest.java | 12 ++++---- .../org/apache/beam/sdk/io/WriteTest.java | 4 ++- .../beam/sdk/transforms/MapElementsTest.java | 8 +++--- .../org/apache/beam/sdk/io/kafka/KafkaIO.java | 10 ++++--- 15 files changed, 81 insertions(+), 74 deletions(-) diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Combine.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Combine.java index 6ba3f8a8964df..56c0bc4e5c6e2 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 @@ -2121,14 +2121,14 @@ public void processElement(ProcessContext c) { inputCoder.getValueCoder())) .setWindowingStrategyInternal(preCombineStrategy) .apply("PreCombineHot", Combine.perKey(hotPreCombine)) - .apply("StripNonce", ParDo.of( - new DoFn, AccumT>, - KV>>() { - @ProcessElement - public void processElement(ProcessContext c) { - c.output(KV.of( - c.element().getKey().getKey(), - InputOrAccum.accum(c.element().getValue()))); + .apply("StripNonce", MapElements.via( + new SimpleFunction, AccumT>, + KV>>() { + @Override + public KV> apply(KV, AccumT> elem) { + return KV.of( + elem.getKey().getKey(), + InputOrAccum.accum(elem.getValue())); } })) .setCoder(KvCoder.of(inputCoder.getKeyCoder(), inputOrAccumCoder)) @@ -2137,12 +2137,12 @@ public void processElement(ProcessContext c) { PCollection>> preprocessedCold = split .get(cold) .setCoder(inputCoder) - .apply("PrepareCold", ParDo.of( - new DoFn, KV>>() { - @ProcessElement - public void processElement(ProcessContext c) { - c.output(KV.of(c.element().getKey(), - InputOrAccum.input(c.element().getValue()))); + .apply("PrepareCold", MapElements.via( + new SimpleFunction, KV>>() { + @Override + public KV> apply(KV element) { + return KV.of(element.getKey(), + InputOrAccum.input(element.getValue())); } })) .setCoder(KvCoder.of(inputCoder.getKeyCoder(), inputOrAccumCoder)); 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 ac59c767504e1..195c5d17ed88a 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 @@ -107,10 +107,10 @@ public PerElement() { } public PCollection> apply(PCollection input) { return input - .apply("Init", ParDo.of(new DoFn>() { - @ProcessElement - public void processElement(ProcessContext c) { - c.output(KV.of(c.element(), (Void) null)); + .apply("Init", MapElements.via(new SimpleFunction>() { + @Override + public KV apply(T element) { + return KV.of(element, (Void) null); } })) .apply(Count.perKey()); 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 6f9e3d8ac0784..2837c40cc3a6a 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 @@ -29,7 +29,7 @@ * {@link PCollection} and merging the results. */ public class FlatMapElements -extends PTransform, PCollection> { +extends PTransform, PCollection> { /** * For a {@code SerializableFunction>} {@code fn}, * returns a {@link PTransform} that applies {@code fn} to every element of the input @@ -130,7 +130,7 @@ private FlatMapElements( } @Override - public PCollection apply(PCollection input) { + public PCollection apply(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 7e09d7e4dd3bf..f3f4f887078d6 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 @@ -173,13 +173,11 @@ public PCollection apply(PCollection> in) { @SuppressWarnings("unchecked") Coder elemCoder = ((IterableLikeCoder) inCoder).getElemCoder(); - return in.apply("FlattenIterables", ParDo.of( - new DoFn, T>() { - @ProcessElement - public void processElement(ProcessContext c) { - for (T i : c.element()) { - c.output(i); - } + return in.apply("FlattenIterables", FlatMapElements.via( + new SimpleFunction, Iterable>() { + @Override + public Iterable apply(Iterable element) { + return element; } })) .setCoder(elemCoder); 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 5ac1866a35904..2405adf41e4b7 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 @@ -58,10 +58,10 @@ private Keys() { } @Override public PCollection apply(PCollection> in) { return - in.apply("Keys", ParDo.of(new DoFn, K>() { - @ProcessElement - public void processElement(ProcessContext c) { - c.output(c.element().getKey()); + in.apply("Keys", MapElements.via(new SimpleFunction, K>() { + @Override + public K apply(KV kv) { + return kv.getKey(); } })); } 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 d4386d2a81077..2b81ebfdf1def 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 @@ -62,11 +62,10 @@ private KvSwap() { } @Override public PCollection> apply(PCollection> in) { return - in.apply("KvSwap", ParDo.of(new DoFn, KV>() { - @ProcessElement - public void processElement(ProcessContext c) { - KV e = c.element(); - c.output(KV.of(e.getValue(), e.getKey())); + in.apply("KvSwap", MapElements.via(new SimpleFunction, KV>() { + @Override + public KV apply(KV kv) { + return KV.of(kv.getValue(), kv.getKey()); } })); } 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 17ad6e74a13fd..73e4359831dbe 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 @@ -25,7 +25,7 @@ * {@code PTransform}s for mapping a simple function over the elements of a {@link PCollection}. */ public class MapElements -extends PTransform, PCollection> { +extends PTransform, PCollection> { /** * For a {@code SerializableFunction} {@code fn} and output type descriptor, @@ -44,8 +44,16 @@ public class MapElements * descriptor need not be provided. */ public static MissingOutputTypeDescriptor - via(SerializableFunction fn) { - return new MissingOutputTypeDescriptor<>(fn); + via(SerializableFunction fn) { + + // TypeDescriptor interacts poorly with the wildcards needed to correctly express + // covariance and contravariance in Java, so instead we cast it to an invariant + // function here. + @SuppressWarnings("unchecked") // safe covariant cast + SerializableFunction simplerFn = + (SerializableFunction) fn; + + return new MissingOutputTypeDescriptor<>(simplerFn); } /** @@ -103,7 +111,7 @@ private MapElements(SimpleFunction fn, Class fnClass) { } @Override - public PCollection apply(PCollection input) { + public PCollection apply(PCollection input) { return input.apply( "Map", ParDo.of( diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/RemoveDuplicates.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/RemoveDuplicates.java index bba4b51309570..2744b14c8913c 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/RemoveDuplicates.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/RemoveDuplicates.java @@ -85,10 +85,10 @@ public static WithRepresentativeValues withRepresentativeValueF @Override public PCollection apply(PCollection in) { return in - .apply("CreateIndex", ParDo.of(new DoFn>() { - @ProcessElement - public void processElement(ProcessContext c) { - c.output(KV.of(c.element(), (Void) null)); + .apply("CreateIndex", MapElements.via(new SimpleFunction>() { + @Override + public KV apply(T element) { + return KV.of(element, (Void) null); } })) .apply(Combine.perKey( 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 34342db53c6b8..d21d100764a07 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 @@ -58,10 +58,10 @@ private Values() { } @Override public PCollection apply(PCollection> in) { return - in.apply("Values", ParDo.of(new DoFn, V>() { - @ProcessElement - public void processElement(ProcessContext c) { - c.output(c.element().getValue()); + in.apply("Values", MapElements.via(new SimpleFunction, V>() { + @Override + public V apply(KV kv) { + return kv.getValue(); } })); } 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 2a44963e6fc2d..8b061f6bdc57f 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 @@ -113,11 +113,10 @@ public WithKeys withKeyType(TypeDescriptor keyType) { @Override public PCollection> apply(PCollection in) { PCollection> result = - in.apply("AddKeys", ParDo.of(new DoFn>() { - @ProcessElement - public void processElement(ProcessContext c) { - c.output(KV.of(fn.apply(c.element()), - c.element())); + in.apply("AddKeys", MapElements.via(new SimpleFunction>() { + @Override + public KV apply(V element) { + return KV.of(fn.apply(element), element); } })); 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 c1b0237e7a69e..9dd069cf99521 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 @@ -21,10 +21,10 @@ import org.apache.beam.sdk.annotations.Experimental.Kind; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.Coder.NonDeterministicException; -import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.GroupByKey; +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.display.DisplayData; import org.apache.beam.sdk.util.WindowingStrategy; import org.apache.beam.sdk.util.WindowingStrategy.AccumulationMode; @@ -645,10 +645,9 @@ public PCollection apply(PCollection input) { // We first apply a (trivial) transform to the input PCollection to produce a new // PCollection. This ensures that we don't modify the windowing strategy of the input // which may be used elsewhere. - .apply("Identity", ParDo.of(new DoFn() { - @ProcessElement - public void processElement(ProcessContext c) { - c.output(c.element()); + .apply("Identity", MapElements.via(new SimpleFunction() { + @Override public T apply(T element) { + return element; } })) // Then we modify the windowing strategy. 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 8b8649994a069..d7b3ac54de9d8 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 @@ -36,10 +36,10 @@ 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.Flatten; +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.PCollection; import org.apache.beam.sdk.values.PCollectionList; @@ -146,10 +146,10 @@ public void testMultipleApply() { private static PTransform, PCollection> addSuffix( final String suffix) { - return ParDo.of(new DoFn() { - @ProcessElement - public void processElement(DoFn.ProcessContext c) { - c.output(c.element() + suffix); + return MapElements.via(new SimpleFunction() { + @Override + public String apply(String input) { + return input + suffix; } }); } 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 f9bf472eab9d5..b9ba53ba6e130 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 @@ -91,7 +91,9 @@ public class WriteTest { // Static counts of the number of records per shard. private static List recordsPerShard = new ArrayList<>(); - private static final MapElements IDENTITY_MAP = + @SuppressWarnings("unchecked") // covariant cast + private static final PTransform, PCollection> IDENTITY_MAP = + (PTransform) MapElements.via(new SimpleFunction() { @Override public String apply(String input) { 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 e86a1289f82b0..7217bca663fae 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 @@ -233,7 +233,7 @@ public Integer apply(Integer input) { } @Test public void testSimpleFunctionDisplayData() { - SimpleFunction simpleFn = new SimpleFunction() { + SimpleFunction simpleFn = new SimpleFunction() { @Override public Integer apply(Integer input) { return input; @@ -255,17 +255,17 @@ public void populateDisplayData(DisplayData.Builder builder) { @Test @Category(RunnableOnService.class) public void testPrimitiveDisplayData() { - SimpleFunction mapFn = new SimpleFunction() { + SimpleFunction mapFn = new SimpleFunction() { @Override public Integer apply(Integer input) { return input; } }; - MapElements map = MapElements.via(mapFn); + MapElements map = MapElements.via(mapFn); DisplayDataEvaluator evaluator = DisplayDataEvaluator.create(); - Set displayData = evaluator.displayDataForPrimitiveTransforms(map); + Set displayData = evaluator.displayDataForPrimitiveTransforms(map); assertThat("MapElements should include the mapFn in its primitive display data", displayData, hasItem(hasDisplayItem("mapFn", mapFn.getClass()))); } 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 2383105d2e612..8a0c7880e97db 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 @@ -34,9 +34,11 @@ import org.apache.beam.sdk.io.kafka.KafkaCheckpointMark.PartitionMark; import org.apache.beam.sdk.options.PipelineOptions; 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.SerializableFunction; +import org.apache.beam.sdk.transforms.SimpleFunction; import org.apache.beam.sdk.util.CoderUtils; import org.apache.beam.sdk.util.ExposedByteArrayInputStream; import org.apache.beam.sdk.values.KV; @@ -1314,10 +1316,10 @@ private KafkaValueWrite(TypedWrite kvWriteTransform) { public PDone apply(PCollection input) { return input .apply("Kafka values with default key", - ParDo.of(new DoFn>() { - @ProcessElement - public void processElement(ProcessContext ctx) throws Exception { - ctx.output(KV.of(null, ctx.element())); + MapElements.via(new SimpleFunction>() { + @Override + public KV apply(V element) { + return KV.of(null, element); } })) .setCoder(KvCoder.of(VoidCoder.of(), kvWriteTransform.valueCoder)) From 235bf3b5c5e4231ab4c56680ff0a4b78aa58346c Mon Sep 17 00:00:00 2001 From: Pei He Date: Mon, 15 Aug 2016 12:22:11 -0700 Subject: [PATCH 032/346] Set Gcs upload buffer size to 1M in streaming mode in DataflowRunner --- .../beam/runners/dataflow/DataflowRunner.java | 6 ++++ .../runners/dataflow/DataflowRunnerTest.java | 31 +++++++++++++++++++ 2 files changed, 37 insertions(+) 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 62222897184bc..6f8180e0d3810 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 @@ -125,6 +125,7 @@ import com.google.api.services.dataflow.model.Job; import com.google.api.services.dataflow.model.ListJobsResponse; import com.google.api.services.dataflow.model.WorkerPool; +import com.google.cloud.hadoop.util.AbstractGoogleAsyncWriteChannel; import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Function; import com.google.common.base.Joiner; @@ -309,6 +310,11 @@ public static DataflowRunner fromOptions(PipelineOptions options) { + "' invalid. Please make sure the value is non-negative."); } + if (dataflowOptions.isStreaming() && dataflowOptions.getGcsUploadBufferSizeBytes() == null) { + dataflowOptions.setGcsUploadBufferSizeBytes( + AbstractGoogleAsyncWriteChannel.UPLOAD_PIPE_BUFFER_SIZE_DEFAULT); + } + return new DataflowRunner(dataflowOptions); } 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 d7deffdb03238..6f1653b026bec 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 @@ -89,6 +89,7 @@ import com.google.api.services.dataflow.model.DataflowPackage; import com.google.api.services.dataflow.model.Job; import com.google.api.services.dataflow.model.ListJobsResponse; +import com.google.cloud.hadoop.util.AbstractGoogleAsyncWriteChannel; import com.google.common.base.Throwables; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; @@ -795,6 +796,36 @@ public void testValidJobName() throws IOException { } } + @Test + public void testGcsUploadBufferSizeDefault() throws IOException { + DataflowPipelineOptions batchOptions = buildPipelineOptions(); + DataflowRunner.fromOptions(batchOptions); + assertNull(batchOptions.getGcsUploadBufferSizeBytes()); + + DataflowPipelineOptions streamingOptions = buildPipelineOptions(); + streamingOptions.setStreaming(true); + DataflowRunner.fromOptions(streamingOptions); + assertEquals( + AbstractGoogleAsyncWriteChannel.UPLOAD_PIPE_BUFFER_SIZE_DEFAULT, + streamingOptions.getGcsUploadBufferSizeBytes().intValue()); + } + + @Test + public void testGcsUploadBufferSize() throws IOException { + int gcsUploadBufferSizeBytes = 12345678; + DataflowPipelineOptions batchOptions = buildPipelineOptions(); + batchOptions.setGcsUploadBufferSizeBytes(gcsUploadBufferSizeBytes); + DataflowRunner.fromOptions(batchOptions); + assertEquals(gcsUploadBufferSizeBytes, batchOptions.getGcsUploadBufferSizeBytes().intValue()); + + DataflowPipelineOptions streamingOptions = buildPipelineOptions(); + streamingOptions.setStreaming(true); + streamingOptions.setGcsUploadBufferSizeBytes(gcsUploadBufferSizeBytes); + DataflowRunner.fromOptions(streamingOptions); + assertEquals( + gcsUploadBufferSizeBytes, streamingOptions.getGcsUploadBufferSizeBytes().intValue()); + } + /** * A fake PTransform for testing. */ From 530b9c0dc41840b04ca16cc49d26045e5fe29eb2 Mon Sep 17 00:00:00 2001 From: Pei He Date: Wed, 17 Aug 2016 13:56:37 -0700 Subject: [PATCH 033/346] addressed feedback --- .../beam/runners/dataflow/DataflowRunner.java | 6 ++++-- .../runners/dataflow/DataflowRunnerTest.java | 21 ++++++++++++------- 2 files changed, 18 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 6f8180e0d3810..1a845eae8d144 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 @@ -219,6 +219,9 @@ public class DataflowRunner extends PipelineRunner { // The limit of CreateJob request size. 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; + private final Set> pcollectionsRequiringIndexedFormat; /** @@ -311,8 +314,7 @@ public static DataflowRunner fromOptions(PipelineOptions options) { } if (dataflowOptions.isStreaming() && dataflowOptions.getGcsUploadBufferSizeBytes() == null) { - dataflowOptions.setGcsUploadBufferSizeBytes( - AbstractGoogleAsyncWriteChannel.UPLOAD_PIPE_BUFFER_SIZE_DEFAULT); + dataflowOptions.setGcsUploadBufferSizeBytes(GCS_UPLOAD_BUFFER_SIZE_BYTES_DEFAULT); } return new DataflowRunner(dataflowOptions); 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 6f1653b026bec..58b9878713bed 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 @@ -797,31 +797,38 @@ public void testValidJobName() throws IOException { } @Test - public void testGcsUploadBufferSizeDefault() throws IOException { + public void testGcsUploadBufferSizeIsUnsetForBatchWhenDefault() throws IOException { DataflowPipelineOptions batchOptions = buildPipelineOptions(); - DataflowRunner.fromOptions(batchOptions); + batchOptions.setRunner(DataflowRunner.class); + Pipeline.create(batchOptions); assertNull(batchOptions.getGcsUploadBufferSizeBytes()); + } + @Test + public void testGcsUploadBufferSizeIsSetForStreamingWhenDefault() throws IOException { DataflowPipelineOptions streamingOptions = buildPipelineOptions(); streamingOptions.setStreaming(true); - DataflowRunner.fromOptions(streamingOptions); + streamingOptions.setRunner(DataflowRunner.class); + Pipeline.create(streamingOptions); assertEquals( - AbstractGoogleAsyncWriteChannel.UPLOAD_PIPE_BUFFER_SIZE_DEFAULT, + DataflowRunner.GCS_UPLOAD_BUFFER_SIZE_BYTES_DEFAULT, streamingOptions.getGcsUploadBufferSizeBytes().intValue()); } @Test - public void testGcsUploadBufferSize() throws IOException { + public void testGcsUploadBufferSizeUnchangedWhenNotDefault() throws IOException { int gcsUploadBufferSizeBytes = 12345678; DataflowPipelineOptions batchOptions = buildPipelineOptions(); batchOptions.setGcsUploadBufferSizeBytes(gcsUploadBufferSizeBytes); - DataflowRunner.fromOptions(batchOptions); + batchOptions.setRunner(DataflowRunner.class); + Pipeline.create(batchOptions); assertEquals(gcsUploadBufferSizeBytes, batchOptions.getGcsUploadBufferSizeBytes().intValue()); DataflowPipelineOptions streamingOptions = buildPipelineOptions(); streamingOptions.setStreaming(true); streamingOptions.setGcsUploadBufferSizeBytes(gcsUploadBufferSizeBytes); - DataflowRunner.fromOptions(streamingOptions); + streamingOptions.setRunner(DataflowRunner.class); + Pipeline.create(streamingOptions); assertEquals( gcsUploadBufferSizeBytes, streamingOptions.getGcsUploadBufferSizeBytes().intValue()); } From aa541e798987bf790bec75cb12e03bd10811b6b7 Mon Sep 17 00:00:00 2001 From: Pei He Date: Wed, 17 Aug 2016 14:30:23 -0700 Subject: [PATCH 034/346] fix unused imports --- .../java/org/apache/beam/runners/dataflow/DataflowRunner.java | 1 - .../org/apache/beam/runners/dataflow/DataflowRunnerTest.java | 1 - 2 files changed, 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 1a845eae8d144..c4dd703f1232c 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 @@ -125,7 +125,6 @@ import com.google.api.services.dataflow.model.Job; import com.google.api.services.dataflow.model.ListJobsResponse; import com.google.api.services.dataflow.model.WorkerPool; -import com.google.cloud.hadoop.util.AbstractGoogleAsyncWriteChannel; import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Function; import com.google.common.base.Joiner; 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 58b9878713bed..92a6bcb4c7298 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 @@ -89,7 +89,6 @@ import com.google.api.services.dataflow.model.DataflowPackage; import com.google.api.services.dataflow.model.Job; import com.google.api.services.dataflow.model.ListJobsResponse; -import com.google.cloud.hadoop.util.AbstractGoogleAsyncWriteChannel; import com.google.common.base.Throwables; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; From bbd0e6bb6f6eba47c4e7772bc8bed008eeb7de9e Mon Sep 17 00:00:00 2001 From: Vikas Kedigehalli Date: Mon, 15 Aug 2016 15:28:07 -0700 Subject: [PATCH 035/346] DatastoreIO Sink as ParDo --- .../beam/sdk/io/gcp/datastore/V1Beta3.java | 376 +++++++----------- .../sdk/io/gcp/datastore/V1Beta3Test.java | 88 ++-- 2 files changed, 195 insertions(+), 269 deletions(-) diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/datastore/V1Beta3.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/datastore/V1Beta3.java index 052feb34a8479..0d2e2cb72cab9 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/datastore/V1Beta3.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/datastore/V1Beta3.java @@ -30,10 +30,6 @@ import static com.google.datastore.v1beta3.client.DatastoreHelper.makeValue; import org.apache.beam.sdk.annotations.Experimental; -import org.apache.beam.sdk.coders.Coder; -import org.apache.beam.sdk.coders.SerializableCoder; -import org.apache.beam.sdk.io.Sink.WriteOperation; -import org.apache.beam.sdk.io.Sink.Writer; import org.apache.beam.sdk.options.GcpOptions; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.Create; @@ -167,7 +163,8 @@ public class V1Beta3 { * Datastore has a limit of 500 mutations per batch operation, so we flush * changes to Datastore every 500 entities. */ - private static final int DATASTORE_BATCH_UPDATE_LIMIT = 500; + @VisibleForTesting + static final int DATASTORE_BATCH_UPDATE_LIMIT = 500; /** * Returns an empty {@link V1Beta3.Read} builder. Configure the source {@code projectId}, @@ -634,42 +631,8 @@ public void processElement(ProcessContext context) throws Exception { } } } - - /** - * A wrapper factory class for Datastore singleton classes {@link DatastoreFactory} and - * {@link QuerySplitter} - * - *

    {@link DatastoreFactory} and {@link QuerySplitter} are not java serializable, hence - * wrapping them under this class, which implements {@link Serializable}. - */ - @VisibleForTesting - static class V1Beta3DatastoreFactory implements Serializable { - - /** Builds a Datastore client for the given pipeline options and project. */ - public Datastore getDatastore(PipelineOptions pipelineOptions, String projectId) { - DatastoreOptions.Builder builder = - new DatastoreOptions.Builder() - .projectId(projectId) - .initializer( - new RetryHttpRequestInitializer() - ); - - Credential credential = pipelineOptions.as(GcpOptions.class).getGcpCredential(); - if (credential != null) { - builder.credential(credential); - } - - return DatastoreFactory.get().create(builder.build()); - } - - /** Builds a Datastore {@link QuerySplitter}. */ - public QuerySplitter getQuerySplitter() { - return DatastoreHelper.getQuerySplitter(); - } - } } - /** * Returns an empty {@link V1Beta3.Write} builder. Configure the destination * {@code projectId} using {@link V1Beta3.Write#withProjectId}. @@ -705,8 +668,8 @@ public Write withProjectId(String projectId) { @Override public PDone apply(PCollection input) { - return input.apply( - org.apache.beam.sdk.io.Write.to(new DatastoreSink(projectId))); + input.apply(ParDo.of(new DatastoreWriterFn(projectId))); + return PDone.in(input.getPipeline()); } @Override @@ -733,130 +696,127 @@ public void populateDisplayData(DisplayData.Builder builder) { .addIfNotNull(DisplayData.item("projectId", projectId) .withLabel("Output Project")); } - } - /** - * A {@link org.apache.beam.sdk.io.Sink} that writes data to Datastore. - */ - static class DatastoreSink extends org.apache.beam.sdk.io.Sink { - final String projectId; - - public DatastoreSink(String projectId) { - this.projectId = projectId; - } - - @Override - public void validate(PipelineOptions options) { - checkNotNull(projectId, "projectId"); - } - - @Override - public DatastoreWriteOperation createWriteOperation(PipelineOptions options) { - return new DatastoreWriteOperation(this); - } - - @Override - public void populateDisplayData(DisplayData.Builder builder) { - super.populateDisplayData(builder); - builder - .addIfNotNull(DisplayData.item("projectId", projectId) - .withLabel("Output Project")); - } - } + /** + * A {@link DoFn} that writes {@link Entity} objects to Cloud Datastore. Entities are written in + * batches, where the maximum batch size is {@link V1Beta3#DATASTORE_BATCH_UPDATE_LIMIT}. + * Entities are committed as upsert mutations (either update if the key already exists, or + * insert if it is a new key). If an entity does not have a complete key (i.e., it has no name + * or id), the bundle will fail. + * + *

    See + * Datastore: Entities, Properties, and Keys for information about entity keys and entities. + * + *

    Commits are non-transactional. If a commit fails because of a conflict over an entity + * group, the commit will be retried (up to {@link V1Beta3#DATASTORE_BATCH_UPDATE_LIMIT} + * times). + */ + @VisibleForTesting + static class DatastoreWriterFn extends DoFn { + private static final Logger LOG = LoggerFactory.getLogger(DatastoreWriterFn.class); + private final String projectId; + private transient Datastore datastore; + private final V1Beta3DatastoreFactory datastoreFactory; + // Current batch of entities to be written. + private final List entities = new ArrayList<>(); + /** + * Since a bundle is written in batches, we should retry the commit of a batch in order to + * prevent transient errors from causing the bundle to fail. + */ + private static final int MAX_RETRIES = 5; - /** - * A {@link WriteOperation} that will manage a parallel write to a Datastore sink. - */ - private static class DatastoreWriteOperation - extends WriteOperation { - private static final Logger LOG = LoggerFactory.getLogger(DatastoreWriteOperation.class); + /** + * Initial backoff time for exponential backoff for retry attempts. + */ + private static final int INITIAL_BACKOFF_MILLIS = 5000; - private final DatastoreSink sink; + public DatastoreWriterFn(String projectId) { + this(projectId, new V1Beta3DatastoreFactory()); + } - public DatastoreWriteOperation(DatastoreSink sink) { - this.sink = sink; - } + @VisibleForTesting + DatastoreWriterFn(String projectId, V1Beta3DatastoreFactory datastoreFactory) { + this.projectId = checkNotNull(projectId, "projectId"); + this.datastoreFactory = datastoreFactory; + } - @Override - public Coder getWriterResultCoder() { - return SerializableCoder.of(DatastoreWriteResult.class); - } + @StartBundle + public void startBundle(Context c) { + datastore = datastoreFactory.getDatastore(c.getPipelineOptions(), projectId); + } - @Override - public void initialize(PipelineOptions options) throws Exception {} + @ProcessElement + public void processElement(ProcessContext c) throws Exception { + // Verify that the entity to write has a complete key. + if (!isValidKey(c.element().getKey())) { + throw new IllegalArgumentException( + "Entities to be written to the Datastore must have complete keys"); + } - /** - * Finalizes the write. Logs the number of entities written to the Datastore. - */ - @Override - public void finalize(Iterable writerResults, PipelineOptions options) - throws Exception { - long totalEntities = 0; - for (DatastoreWriteResult result : writerResults) { - totalEntities += result.entitiesWritten; + entities.add(c.element()); + if (entities.size() >= V1Beta3.DATASTORE_BATCH_UPDATE_LIMIT) { + flushBatch(); + } } - LOG.info("Wrote {} elements.", totalEntities); - } - @Override - public DatastoreWriter createWriter(PipelineOptions options) throws Exception { - DatastoreOptions.Builder builder = - new DatastoreOptions.Builder() - .projectId(sink.projectId) - .initializer(new RetryHttpRequestInitializer()); - Credential credential = options.as(GcpOptions.class).getGcpCredential(); - if (credential != null) { - builder.credential(credential); + @FinishBundle + public void finishBundle(Context c) throws Exception { + if (entities.size() > 0) { + flushBatch(); + } } - Datastore datastore = DatastoreFactory.get().create(builder.build()); - return new DatastoreWriter(this, datastore); - } + /** + * Writes a batch of entities to Cloud Datastore. + * + *

    If a commit fails, it will be retried (up to {@link DatastoreWriterFn#MAX_RETRIES} + * times). All entities in the batch will be committed again, even if the commit was partially + * successful. If the retry limit is exceeded, the last exception from the Datastore will be + * thrown. + * + * @throws DatastoreException if the commit fails or IOException or InterruptedException if + * backing off between retries fails. + */ + private void flushBatch() throws DatastoreException, IOException, InterruptedException { + LOG.debug("Writing batch of {} entities", entities.size()); + Sleeper sleeper = Sleeper.DEFAULT; + BackOff backoff = new AttemptBoundedExponentialBackOff(MAX_RETRIES, INITIAL_BACKOFF_MILLIS); + + while (true) { + // Batch upsert entities. + try { + CommitRequest.Builder commitRequest = CommitRequest.newBuilder(); + for (Entity entity: entities) { + commitRequest.addMutations(makeUpsert(entity)); + } + commitRequest.setMode(CommitRequest.Mode.NON_TRANSACTIONAL); + datastore.commit(commitRequest.build()); + // Break if the commit threw no exception. + break; + } catch (DatastoreException exception) { + // Only log the code and message for potentially-transient errors. The entire exception + // will be propagated upon the last retry. + LOG.error("Error writing to the Datastore ({}): {}", exception.getCode(), + exception.getMessage()); + if (!BackOffUtils.next(sleeper, backoff)) { + LOG.error("Aborting after {} retries.", MAX_RETRIES); + throw exception; + } + } + } + LOG.debug("Successfully wrote {} entities", entities.size()); + entities.clear(); + } - @Override - public DatastoreSink getSink() { - return sink; + @Override + public void populateDisplayData(Builder builder) { + super.populateDisplayData(builder); + builder + .addIfNotNull(DisplayData.item("projectId", projectId) + .withLabel("Output Project")); + } } - } - - /** - * {@link Writer} that writes entities to a Datastore Sink. Entities are written in batches, - * where the maximum batch size is {@link V1Beta3#DATASTORE_BATCH_UPDATE_LIMIT}. Entities - * are committed as upsert mutations (either update if the key already exists, or insert if it is - * a new key). If an entity does not have a complete key (i.e., it has no name or id), the bundle - * will fail. - * - *

    See - * Datastore: Entities, Properties, and Keys for information about entity keys and upsert - * mutations. - * - *

    Commits are non-transactional. If a commit fails because of a conflict over an entity - * group, the commit will be retried (up to {@link V1Beta3#DATASTORE_BATCH_UPDATE_LIMIT} - * times). - * - *

    Visible for testing purposes. - */ - @VisibleForTesting - static class DatastoreWriter extends Writer { - private static final Logger LOG = LoggerFactory.getLogger(DatastoreWriter.class); - private final DatastoreWriteOperation writeOp; - private final Datastore datastore; - private long totalWritten = 0; - - // Visible for testing. - final List entities = new ArrayList<>(); - - /** - * Since a bundle is written in batches, we should retry the commit of a batch in order to - * prevent transient errors from causing the bundle to fail. - */ - private static final int MAX_RETRIES = 5; - - /** - * Initial backoff time for exponential backoff for retry attempts. - */ - private static final int INITIAL_BACKOFF_MILLIS = 5000; /** * Returns true if a Datastore key is complete. A key is complete if its last element @@ -870,100 +830,38 @@ static boolean isValidKey(Key key) { PathElement lastElement = elementList.get(elementList.size() - 1); return (lastElement.getId() != 0 || !lastElement.getName().isEmpty()); } + } - DatastoreWriter(DatastoreWriteOperation writeOp, Datastore datastore) { - this.writeOp = writeOp; - this.datastore = datastore; - } - - @Override - public void open(String uId) throws Exception {} - - /** - * Writes an entity to the Datastore. Writes are batched, up to {@link - * V1Beta3#DATASTORE_BATCH_UPDATE_LIMIT}. If an entity does not have a complete key, an - * {@link IllegalArgumentException} will be thrown. - */ - @Override - public void write(Entity value) throws Exception { - // Verify that the entity to write has a complete key. - if (!isValidKey(value.getKey())) { - throw new IllegalArgumentException( - "Entities to be written to the Datastore must have complete keys"); - } - - entities.add(value); + /** + * A wrapper factory class for Datastore singleton classes {@link DatastoreFactory} and + * {@link QuerySplitter} + * + *

    {@link DatastoreFactory} and {@link QuerySplitter} are not java serializable, hence + * wrapping them under this class, which implements {@link Serializable}. + */ + @VisibleForTesting + static class V1Beta3DatastoreFactory implements Serializable { - if (entities.size() >= V1Beta3.DATASTORE_BATCH_UPDATE_LIMIT) { - flushBatch(); - } - } + /** Builds a Datastore client for the given pipeline options and project. */ + public Datastore getDatastore(PipelineOptions pipelineOptions, String projectId) { + DatastoreOptions.Builder builder = + new DatastoreOptions.Builder() + .projectId(projectId) + .initializer( + new RetryHttpRequestInitializer() + ); - /** - * Flushes any pending batch writes and returns a DatastoreWriteResult. - */ - @Override - public DatastoreWriteResult close() throws Exception { - if (entities.size() > 0) { - flushBatch(); + Credential credential = pipelineOptions.as(GcpOptions.class).getGcpCredential(); + if (credential != null) { + builder.credential(credential); } - return new DatastoreWriteResult(totalWritten); - } - @Override - public DatastoreWriteOperation getWriteOperation() { - return writeOp; + return DatastoreFactory.get().create(builder.build()); } - /** - * Writes a batch of entities to the Datastore. - * - *

    If a commit fails, it will be retried (up to {@link DatastoreWriter#MAX_RETRIES} - * times). All entities in the batch will be committed again, even if the commit was partially - * successful. If the retry limit is exceeded, the last exception from the Datastore will be - * thrown. - * - * @throws DatastoreException if the commit fails or IOException or InterruptedException if - * backing off between retries fails. - */ - private void flushBatch() throws DatastoreException, IOException, InterruptedException { - LOG.debug("Writing batch of {} entities", entities.size()); - Sleeper sleeper = Sleeper.DEFAULT; - BackOff backoff = new AttemptBoundedExponentialBackOff(MAX_RETRIES, INITIAL_BACKOFF_MILLIS); - - while (true) { - // Batch upsert entities. - try { - CommitRequest.Builder commitRequest = CommitRequest.newBuilder(); - for (Entity entity: entities) { - commitRequest.addMutations(makeUpsert(entity)); - } - commitRequest.setMode(CommitRequest.Mode.NON_TRANSACTIONAL); - datastore.commit(commitRequest.build()); - // Break if the commit threw no exception. - break; - } catch (DatastoreException exception) { - // Only log the code and message for potentially-transient errors. The entire exception - // will be propagated upon the last retry. - LOG.error("Error writing to the Datastore ({}): {}", exception.getCode(), - exception.getMessage()); - if (!BackOffUtils.next(sleeper, backoff)) { - LOG.error("Aborting after {} retries.", MAX_RETRIES); - throw exception; - } - } - } - totalWritten += entities.size(); - LOG.debug("Successfully wrote {} entities", entities.size()); - entities.clear(); - } - } - - private static class DatastoreWriteResult implements Serializable { - final long entitiesWritten; - - public DatastoreWriteResult(long recordsWritten) { - this.entitiesWritten = recordsWritten; + /** Builds a Datastore {@link QuerySplitter}. */ + public QuerySplitter getQuerySplitter() { + return DatastoreHelper.getQuerySplitter(); } } } diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/V1Beta3Test.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/V1Beta3Test.java index 9947c6087a46a..8fa34dae119fe 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/V1Beta3Test.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/V1Beta3Test.java @@ -17,6 +17,7 @@ */ package org.apache.beam.sdk.io.gcp.datastore; +import static org.apache.beam.sdk.io.gcp.datastore.V1Beta3.DATASTORE_BATCH_UPDATE_LIMIT; import static org.apache.beam.sdk.io.gcp.datastore.V1Beta3.Read.DEFAULT_BUNDLE_SIZE_BYTES; import static org.apache.beam.sdk.io.gcp.datastore.V1Beta3.Read.QUERY_BATCH_LIMIT; import static org.apache.beam.sdk.io.gcp.datastore.V1Beta3.Read.getEstimatedSizeBytes; @@ -27,8 +28,8 @@ import static com.google.datastore.v1beta3.client.DatastoreHelper.makeFilter; import static com.google.datastore.v1beta3.client.DatastoreHelper.makeKey; import static com.google.datastore.v1beta3.client.DatastoreHelper.makeOrder; +import static com.google.datastore.v1beta3.client.DatastoreHelper.makeUpsert; import static com.google.datastore.v1beta3.client.DatastoreHelper.makeValue; -import static org.hamcrest.Matchers.containsInAnyOrder; import static org.hamcrest.Matchers.greaterThanOrEqualTo; import static org.hamcrest.Matchers.hasItem; import static org.hamcrest.Matchers.lessThanOrEqualTo; @@ -44,11 +45,12 @@ import static org.mockito.Mockito.verifyZeroInteractions; import static org.mockito.Mockito.when; -import org.apache.beam.sdk.io.gcp.datastore.V1Beta3.DatastoreWriter; import org.apache.beam.sdk.io.gcp.datastore.V1Beta3.Read.ReadFn; import org.apache.beam.sdk.io.gcp.datastore.V1Beta3.Read.SplitQueryFn; -import org.apache.beam.sdk.io.gcp.datastore.V1Beta3.Read.V1Beta3DatastoreFactory; import org.apache.beam.sdk.io.gcp.datastore.V1Beta3.Read.V1Beta3Options; +import org.apache.beam.sdk.io.gcp.datastore.V1Beta3.V1Beta3DatastoreFactory; +import org.apache.beam.sdk.io.gcp.datastore.V1Beta3.Write; +import org.apache.beam.sdk.io.gcp.datastore.V1Beta3.Write.DatastoreWriterFn; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.testing.RunnableOnService; import org.apache.beam.sdk.transforms.DoFnTester; @@ -61,7 +63,7 @@ import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.POutput; -import com.google.common.collect.Lists; +import com.google.datastore.v1beta3.CommitRequest; import com.google.datastore.v1beta3.Entity; import com.google.datastore.v1beta3.EntityResult; import com.google.datastore.v1beta3.Key; @@ -87,7 +89,6 @@ import org.mockito.stubbing.Answer; import java.util.ArrayList; -import java.util.Collections; import java.util.HashSet; import java.util.LinkedList; import java.util.List; @@ -285,33 +286,33 @@ public void testHasNameOrId() { Key key; // Complete with name, no ancestor key = makeKey("bird", "finch").build(); - assertTrue(DatastoreWriter.isValidKey(key)); + assertTrue(Write.isValidKey(key)); // Complete with id, no ancestor key = makeKey("bird", 123).build(); - assertTrue(DatastoreWriter.isValidKey(key)); + assertTrue(Write.isValidKey(key)); // Incomplete, no ancestor key = makeKey("bird").build(); - assertFalse(DatastoreWriter.isValidKey(key)); + assertFalse(Write.isValidKey(key)); // Complete with name and ancestor key = makeKey("bird", "owl").build(); key = makeKey(key, "bird", "horned").build(); - assertTrue(DatastoreWriter.isValidKey(key)); + assertTrue(Write.isValidKey(key)); // Complete with id and ancestor key = makeKey("bird", "owl").build(); key = makeKey(key, "bird", 123).build(); - assertTrue(DatastoreWriter.isValidKey(key)); + assertTrue(Write.isValidKey(key)); // Incomplete with ancestor key = makeKey("bird", "owl").build(); key = makeKey(key, "bird").build(); - assertFalse(DatastoreWriter.isValidKey(key)); + assertFalse(Write.isValidKey(key)); key = makeKey().build(); - assertFalse(DatastoreWriter.isValidKey(key)); + assertFalse(Write.isValidKey(key)); } /** @@ -321,35 +322,62 @@ public void testHasNameOrId() { public void testAddEntitiesWithIncompleteKeys() throws Exception { Key key = makeKey("bird").build(); Entity entity = Entity.newBuilder().setKey(key).build(); - DatastoreWriter writer = new DatastoreWriter(null, mockDatastore); + DatastoreWriterFn writer = new DatastoreWriterFn(PROJECT_ID, mockDatastoreFactory); + DoFnTester doFnTester = DoFnTester.of(writer); + doFnTester.setCloningBehavior(CloningBehavior.DO_NOT_CLONE); thrown.expect(IllegalArgumentException.class); thrown.expectMessage("Entities to be written to the Datastore must have complete keys"); - writer.write(entity); + doFnTester.processBundle(entity); + } + + /** Tests {@link DatastoreWriterFn} with entities less than one batch. */ + @Test + public void testDatatoreWriterFnWithOneBatch() throws Exception { + datastoreWriterFnTest(100); + } + + /** Tests {@link DatastoreWriterFn} with entities of more than one batches, but not a multiple. */ + @Test + public void testDatatoreWriterFnWithMultipleBatches() throws Exception { + datastoreWriterFnTest(DATASTORE_BATCH_UPDATE_LIMIT * 3 + 100); } /** - * Test that entities are added to the batch to update. + * Tests {@link DatastoreWriterFn} with entities of several batches, using an exact multiple of + * write batch size. */ @Test - public void testAddingEntities() throws Exception { - List expected = Lists.newArrayList( - Entity.newBuilder().setKey(makeKey("bird", "jay").build()).build(), - Entity.newBuilder().setKey(makeKey("bird", "condor").build()).build(), - Entity.newBuilder().setKey(makeKey("bird", "robin").build()).build()); - - List allEntities = Lists.newArrayList(expected); - Collections.shuffle(allEntities); - - DatastoreWriter writer = new DatastoreWriter(null, mockDatastore); - writer.open("test_id"); - for (Entity entity : allEntities) { - writer.write(entity); + public void testDatatoreWriterFnWithBatchesExactMultiple() throws Exception { + datastoreWriterFnTest(DATASTORE_BATCH_UPDATE_LIMIT * 2); + } + + // A helper method to test DatastoreWriterFn for various batch sizes. + private void datastoreWriterFnTest(int numEntities) throws Exception { + // Create the requested number of mutations. + List entities = new ArrayList<>(numEntities); + for (int i = 0; i < numEntities; ++i) { + entities.add(Entity.newBuilder().setKey(makeKey("key" + i, i + 1)).build()); } - assertEquals(expected.size(), writer.entities.size()); - assertThat(writer.entities, containsInAnyOrder(expected.toArray())); + DatastoreWriterFn datastoreWriter = new DatastoreWriterFn(PROJECT_ID, mockDatastoreFactory); + DoFnTester doFnTester = DoFnTester.of(datastoreWriter); + doFnTester.setCloningBehavior(CloningBehavior.DO_NOT_CLONE); + doFnTester.processBundle(entities); + + int start = 0; + while (start < numEntities) { + int end = Math.min(numEntities, start + DATASTORE_BATCH_UPDATE_LIMIT); + CommitRequest.Builder commitRequest = CommitRequest.newBuilder(); + commitRequest.setMode(CommitRequest.Mode.NON_TRANSACTIONAL); + for (Entity entity: entities.subList(start, end)) { + commitRequest.addMutations(makeUpsert(entity)); + } + // Verify all the batch requests were made with the expected entities. + verify(mockDatastore, times(1)).commit(commitRequest.build()); + start = end; + } } /** From 245c3ceabc8d7a596849ae02b386c522d2fe9537 Mon Sep 17 00:00:00 2001 From: Frank Yellin Date: Tue, 16 Aug 2016 14:40:41 -0700 Subject: [PATCH 036/346] Change name of result returned by BigQueryIO.Read --- .../java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java | 4 ++-- 1 file changed, 2 insertions(+), 2 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 aa168bdab0db0..ce0446788156c 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 @@ -165,7 +165,7 @@ *

    To read from a BigQuery table, apply a {@link BigQueryIO.Read} transformation. * This produces a {@link PCollection} of {@link TableRow TableRows} as output: *

    {@code
    - * PCollection shakespeare = pipeline.apply(
    + * PCollection weatherData = pipeline.apply(
      *     BigQueryIO.Read.from("clouddataflow-readonly:samples.weather_stations"));
      * }
    * @@ -176,7 +176,7 @@ * input transform. * *
    {@code
    - * PCollection shakespeare = pipeline.apply(
    + * PCollection meanTemperatureData = pipeline.apply(
      *     BigQueryIO.Read.fromQuery("SELECT year, mean_temp FROM [samples.weather_stations]"));
      * }
    * From 16bcf78adb3c4650f0f77d0659e029dc4312622c Mon Sep 17 00:00:00 2001 From: Pei He Date: Mon, 15 Aug 2016 17:23:20 -0700 Subject: [PATCH 037/346] Fix NPE in BigQueryIO.TransformingReader --- .../beam/sdk/testing/SourceTestUtils.java | 132 ++++++++++++++++++ .../beam/sdk/testing/SourceTestUtilsTest.java | 66 +++++++++ .../beam/sdk/io/gcp/bigquery/BigQueryIO.java | 12 +- .../sdk/io/gcp/bigquery/BigQueryIOTest.java | 31 ++++ 4 files changed, 235 insertions(+), 6 deletions(-) create mode 100644 sdks/java/core/src/test/java/org/apache/beam/sdk/testing/SourceTestUtilsTest.java diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/SourceTestUtils.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/SourceTestUtils.java index e0b88900dbc5d..9ce9c5e518da8 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/SourceTestUtils.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/SourceTestUtils.java @@ -17,6 +17,8 @@ */ package org.apache.beam.sdk.testing; +import static com.google.common.base.Preconditions.checkNotNull; + import static org.hamcrest.Matchers.containsInAnyOrder; import static org.hamcrest.Matchers.equalTo; import static org.junit.Assert.assertEquals; @@ -27,10 +29,15 @@ import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.io.BoundedSource; +import org.apache.beam.sdk.io.BoundedSource.BoundedReader; import org.apache.beam.sdk.io.Source; import org.apache.beam.sdk.options.PipelineOptions; +import org.apache.beam.sdk.transforms.display.DisplayData; import org.apache.beam.sdk.values.KV; +import com.google.common.collect.ImmutableList; + +import org.joda.time.Instant; import org.junit.Assert; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -38,6 +45,7 @@ import java.io.IOException; import java.util.ArrayList; import java.util.List; +import java.util.NoSuchElementException; import java.util.Objects; import java.util.concurrent.Callable; import java.util.concurrent.CountDownLatch; @@ -45,6 +53,8 @@ import java.util.concurrent.Executors; import java.util.concurrent.Future; +import javax.annotation.Nullable; + /** * Helper functions and test harnesses for checking correctness of {@link Source} * implementations. @@ -673,4 +683,126 @@ public KV, BoundedSource> call() throws Exception { numItemsToReadBeforeSplitting, fraction, options); return (res.numResidualItems > 0); } + + /** + * Returns an equivalent unsplittable {@code BoundedSource}. + * + *

    It forwards most methods to the given {@code boundedSource}, except: + *

      + *
    1. {@link BoundedSource#splitIntoBundles} rejects initial splitting + * by returning itself in a list. + *
    2. {@link BoundedReader#splitAtFraction} rejects dynamic splitting by returning null. + *
    + */ + public static BoundedSource toUnsplittableSource(BoundedSource boundedSource) { + return new UnsplittableSource<>(boundedSource); + } + + private static class UnsplittableSource extends BoundedSource { + + private final BoundedSource boundedSource; + + private UnsplittableSource(BoundedSource boundedSource) { + this.boundedSource = checkNotNull(boundedSource, "boundedSource"); + } + + @Override + public void populateDisplayData(DisplayData.Builder builder) { + this.boundedSource.populateDisplayData(builder); + } + + @Override + public List> splitIntoBundles( + long desiredBundleSizeBytes, PipelineOptions options) throws Exception { + return ImmutableList.of(this); + } + + @Override + public long getEstimatedSizeBytes(PipelineOptions options) throws Exception { + return boundedSource.getEstimatedSizeBytes(options); + } + + @Override + public boolean producesSortedKeys(PipelineOptions options) throws Exception { + return boundedSource.producesSortedKeys(options); + } + + @Override + public BoundedReader createReader(PipelineOptions options) throws IOException { + return new UnsplittableReader<>(boundedSource, boundedSource.createReader(options)); + } + + @Override + public void validate() { + boundedSource.validate(); + } + + @Override + public Coder getDefaultOutputCoder() { + return boundedSource.getDefaultOutputCoder(); + } + + private static class UnsplittableReader extends BoundedReader { + + private final BoundedSource boundedSource; + private final BoundedReader boundedReader; + + private UnsplittableReader(BoundedSource boundedSource, BoundedReader boundedReader) { + this.boundedSource = checkNotNull(boundedSource, "boundedSource"); + this.boundedReader = checkNotNull(boundedReader, "boundedReader"); + } + + @Override + public BoundedSource getCurrentSource() { + return boundedSource; + } + + @Override + public boolean start() throws IOException { + return boundedReader.start(); + } + + @Override + public boolean advance() throws IOException { + return boundedReader.advance(); + } + + @Override + public T getCurrent() throws NoSuchElementException { + return boundedReader.getCurrent(); + } + + @Override + public void close() throws IOException { + boundedReader.close(); + } + + @Override + @Nullable + public BoundedSource splitAtFraction(double fraction) { + return null; + } + + @Override + @Nullable + public Double getFractionConsumed() { + return boundedReader.getFractionConsumed(); + } + + @Override + public long getSplitPointsConsumed() { + return boundedReader.getSplitPointsConsumed(); + } + + @Override + public long getSplitPointsRemaining() { + return boundedReader.getSplitPointsRemaining(); + } + + @Override + public Instant getCurrentTimestamp() throws NoSuchElementException { + return boundedReader.getCurrentTimestamp(); + } + } + } } diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/SourceTestUtilsTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/SourceTestUtilsTest.java new file mode 100644 index 0000000000000..f2b332bb7dd3a --- /dev/null +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/SourceTestUtilsTest.java @@ -0,0 +1,66 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.sdk.testing; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNull; + +import org.apache.beam.sdk.io.BoundedSource; +import org.apache.beam.sdk.io.BoundedSource.BoundedReader; +import org.apache.beam.sdk.io.CountingSource; +import org.apache.beam.sdk.options.PipelineOptions; +import org.apache.beam.sdk.options.PipelineOptionsFactory; + +import com.google.common.collect.Sets; + +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +import java.util.List; +import java.util.Set; + +/** + * Tests for {@link SourceTestUtils}. + */ +@RunWith(JUnit4.class) +public class SourceTestUtilsTest { + + @Test + public void testToUnsplittableSource() throws Exception { + PipelineOptions options = PipelineOptionsFactory.create(); + BoundedSource baseSource = CountingSource.upTo(100); + BoundedSource unsplittableSource = SourceTestUtils.toUnsplittableSource(baseSource); + List splits = unsplittableSource.splitIntoBundles(1, options); + assertEquals(splits.size(), 1); + assertEquals(splits.get(0), unsplittableSource); + + BoundedReader unsplittableReader = unsplittableSource.createReader(options); + assertEquals(0, unsplittableReader.getFractionConsumed(), 1e-15); + + Set expected = Sets.newHashSet(SourceTestUtils.readFromSource(baseSource, options)); + Set actual = Sets.newHashSet(); + actual.addAll(SourceTestUtils.readNItemsFromUnstartedReader(unsplittableReader, 40)); + assertNull(unsplittableReader.splitAtFraction(0.5)); + actual.addAll(SourceTestUtils.readRemainingFromReader(unsplittableReader, true /* started */)); + assertEquals(1, unsplittableReader.getFractionConsumed(), 1e-15); + + assertEquals(100, actual.size()); + assertEquals(Sets.newHashSet(expected), Sets.newHashSet(actual)); + } +} 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 ce0446788156c..e61dcca993a00 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 @@ -1123,9 +1123,9 @@ static class TransformingSource extends BoundedSource { BoundedSource boundedSource, SerializableFunction function, Coder outputCoder) { - this.boundedSource = boundedSource; - this.function = function; - this.outputCoder = outputCoder; + this.boundedSource = checkNotNull(boundedSource, "boundedSource"); + this.function = checkNotNull(function, "function"); + this.outputCoder = checkNotNull(outputCoder, "outputCoder"); } @Override @@ -1170,7 +1170,7 @@ private class TransformingReader extends BoundedReader { private final BoundedReader boundedReader; private TransformingReader(BoundedReader boundedReader) { - this.boundedReader = boundedReader; + this.boundedReader = checkNotNull(boundedReader, "boundedReader"); } @Override @@ -1201,8 +1201,8 @@ public void close() throws IOException { @Override public synchronized BoundedSource splitAtFraction(double fraction) { - return new TransformingSource<>( - boundedReader.splitAtFraction(fraction), function, outputCoder); + BoundedSource split = boundedReader.splitAtFraction(fraction); + return split == null ? null : new TransformingSource<>(split, function, outputCoder); } @Override 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 fcaa054da4565..ca60696960e8d 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 @@ -1215,6 +1215,37 @@ public String apply(Long input) { stringSource, stringSource.splitIntoBundles(100, options), options); } + @Test + public void testTransformingSourceUnsplittable() throws Exception { + int numElements = 10000; + @SuppressWarnings("deprecation") + BoundedSource longSource = + SourceTestUtils.toUnsplittableSource(CountingSource.upTo(numElements)); + SerializableFunction toStringFn = + new SerializableFunction() { + @Override + public String apply(Long input) { + return input.toString(); + } + }; + BoundedSource stringSource = + new TransformingSource<>(longSource, toStringFn, StringUtf8Coder.of()); + + List expected = Lists.newArrayList(); + for (int i = 0; i < numElements; i++) { + expected.add(String.valueOf(i)); + } + + PipelineOptions options = PipelineOptionsFactory.create(); + Assert.assertThat( + SourceTestUtils.readFromSource(stringSource, options), CoreMatchers.is(expected)); + SourceTestUtils.assertSplitAtFractionBehavior( + stringSource, 100, 0.3, ExpectedSplitOutcome.MUST_BE_CONSISTENT_IF_SUCCEEDS, options); + + SourceTestUtils.assertSourcesEqualReferenceSource( + stringSource, stringSource.splitIntoBundles(100, options), options); + } + @Test @Category(RunnableOnService.class) public void testPassThroughThenCleanup() throws Exception { From 178898fea91625fb880cdb5a1b7515d537793257 Mon Sep 17 00:00:00 2001 From: Thomas Groh Date: Thu, 11 Aug 2016 15:46:10 -0700 Subject: [PATCH 038/346] Add inEarlyPanesInGlobalWindow as a PAssert Extractor This is for use in asserting the contents of speculative panes in the global window. --- .../org/apache/beam/sdk/testing/PAssert.java | 18 ++++++++++++++++++ .../beam/sdk/testing/PaneExtractors.java | 18 ++++++++++++++++++ 2 files changed, 36 insertions(+) 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 e07ee3dad4f4b..3f1a741a4dff8 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 @@ -49,6 +49,7 @@ import org.apache.beam.sdk.transforms.windowing.GlobalWindow; import org.apache.beam.sdk.transforms.windowing.GlobalWindows; import org.apache.beam.sdk.transforms.windowing.Never; +import org.apache.beam.sdk.transforms.windowing.PaneInfo.Timing; import org.apache.beam.sdk.transforms.windowing.Trigger; import org.apache.beam.sdk.transforms.windowing.Window; import org.apache.beam.sdk.transforms.windowing.Window.ClosingBehavior; @@ -175,6 +176,13 @@ public interface IterableAssert { */ IterableAssert inCombinedNonLatePanes(BoundedWindow window); + /** + * Creates a new {@link IterableAssert} like this one, but with the assertion restricted to only + * run on panes in the {@link GlobalWindow} that were emitted before the {@link GlobalWindow} + * closed. These panes have {@link Timing#EARLY}. + */ + IterableAssert inEarlyGlobalWindowPanes(); + /** * Asserts that the iterable in question contains the provided elements. * @@ -381,6 +389,11 @@ public PCollectionContentsAssert inCombinedNonLatePanes(BoundedWindow window) return withPane(window, PaneExtractors.nonLatePanes()); } + @Override + public IterableAssert inEarlyGlobalWindowPanes() { + return withPane(GlobalWindow.INSTANCE, PaneExtractors.earlyPanes()); + } + private PCollectionContentsAssert withPane( BoundedWindow window, SimpleFunction>, Iterable> paneExtractor) { @@ -557,6 +570,11 @@ public PCollectionSingletonIterableAssert inCombinedNonLatePanes(BoundedWindo return withPanes(window, PaneExtractors.>nonLatePanes()); } + @Override + public IterableAssert inEarlyGlobalWindowPanes() { + return withPanes(GlobalWindow.INSTANCE, PaneExtractors.>earlyPanes()); + } + private PCollectionSingletonIterableAssert withPanes( BoundedWindow window, SimpleFunction>>, Iterable>> 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 f699bfc83ccec..899612b3ef2a1 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 @@ -59,6 +59,10 @@ static SimpleFunction>, Iterable> nonLatePanes( return new ExtractNonLatePanes<>(); } + static SimpleFunction>, Iterable> earlyPanes() { + return new ExtractEarlyPanes<>(); + } + static SimpleFunction>, Iterable> allPanes() { return new ExtractAllPanes<>(); } @@ -137,4 +141,18 @@ public Iterable apply(Iterable> input) { return outputs; } } + + private static class ExtractEarlyPanes + extends SimpleFunction>, Iterable> { + @Override + public Iterable apply(Iterable> input) { + List outputs = new ArrayList<>(); + for (WindowedValue value : input) { + if (value.getPane().getTiming() == PaneInfo.Timing.EARLY) { + outputs.add(value.getValue()); + } + } + return outputs; + } + } } From 6c82321147033d3aa71fd906b675ffd4b3d4f5c5 Mon Sep 17 00:00:00 2001 From: Thomas Groh Date: Mon, 15 Aug 2016 19:43:28 -0700 Subject: [PATCH 039/346] Add TestStream to the Testing package This is a source suitable for use with tests that have interesting triggering behavior. It is an Unbounded source that emits elements in bundles, and advances the watermark and processing time appropriately. --- runners/direct-java/pom.xml | 3 + .../apache/beam/sdk/testing/TestStream.java | 326 ++++++++++++++++++ .../beam/sdk/testing/TestStreamTest.java | 169 +++++++++ 3 files changed, 498 insertions(+) create mode 100644 sdks/java/core/src/main/java/org/apache/beam/sdk/testing/TestStream.java create mode 100644 sdks/java/core/src/test/java/org/apache/beam/sdk/testing/TestStreamTest.java diff --git a/runners/direct-java/pom.xml b/runners/direct-java/pom.xml index e06883f6e65d7..8b0f91d858a8a 100644 --- a/runners/direct-java/pom.xml +++ b/runners/direct-java/pom.xml @@ -85,6 +85,9 @@ org.apache.beam:beam-sdks-java-core org.apache.beam:beam-runners-java-core + + org/apache/beam/sdk/testing/TestStreamTest.java + [ 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 new file mode 100644 index 0000000000000..6d11f72bb9791 --- /dev/null +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/TestStream.java @@ -0,0 +1,326 @@ +/* + * 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 static com.google.common.base.Preconditions.checkNotNull; + +import org.apache.beam.sdk.Pipeline; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.coders.DurationCoder; +import org.apache.beam.sdk.coders.InstantCoder; +import org.apache.beam.sdk.coders.IterableCoder; +import org.apache.beam.sdk.coders.StandardCoder; +import org.apache.beam.sdk.runners.PipelineRunner; +import org.apache.beam.sdk.transforms.PTransform; +import org.apache.beam.sdk.transforms.windowing.BoundedWindow; +import org.apache.beam.sdk.util.PropertyNames; +import org.apache.beam.sdk.util.VarInt; +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.TimestampedValue; +import org.apache.beam.sdk.values.TimestampedValue.TimestampedValueCoder; + +import com.google.auto.value.AutoValue; +import com.google.common.annotations.VisibleForTesting; +import com.google.common.collect.ImmutableList; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; + +import org.joda.time.Duration; +import org.joda.time.Instant; +import org.joda.time.ReadableDuration; + +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.util.Collections; +import java.util.List; + +/** + * A testing input that generates an unbounded {@link PCollection} of elements, advancing the + * watermark and processing time as elements are emitted. After all of the specified elements are + * emitted, ceases to produce output. + * + *

    Each call to a {@link TestStream.Builder} method will only be reflected in the state of the + * {@link Pipeline} after each method before it has completed and no more progress can be made by + * the {@link Pipeline}. A {@link PipelineRunner} must ensure that no more progress can be made in + * the {@link Pipeline} before advancing the state of the {@link TestStream}. + */ +public final class TestStream extends PTransform> { + private final List> events; + private final Coder coder; + + /** + * Create a new {@link TestStream.Builder} with no elements and watermark equal to {@link + * BoundedWindow#TIMESTAMP_MIN_VALUE}. + */ + public static Builder create(Coder coder) { + return new Builder<>(coder); + } + + private TestStream(Coder coder, List> events) { + this.coder = coder; + this.events = checkNotNull(events); + } + + public Coder> getEventCoder() { + return EventCoder.of(coder); + } + + /** + * An incomplete {@link TestStream}. Elements added to this builder will be produced in sequence + * when the pipeline created by the {@link TestStream} is run. + */ + public static class Builder { + private final Coder coder; + private final ImmutableList.Builder> events; + private Instant currentWatermark; + + private Builder(Coder coder) { + this.coder = coder; + events = ImmutableList.builder(); + + currentWatermark = BoundedWindow.TIMESTAMP_MIN_VALUE; + } + + /** + * Adds the specified elements to the source with timestamp equal to the current watermark. + * + * @return this {@link TestStream.Builder} + */ + @SafeVarargs + public final Builder addElements(T element, T... elements) { + TimestampedValue firstElement = TimestampedValue.of(element, currentWatermark); + @SuppressWarnings("unchecked") + TimestampedValue[] remainingElements = new TimestampedValue[elements.length]; + for (int i = 0; i < elements.length; i++) { + remainingElements[i] = TimestampedValue.of(elements[i], currentWatermark); + } + return addElements(firstElement, remainingElements); + } + + /** + * Adds the specified elements to the source with the provided timestamps. + * + * @return this {@link TestStream.Builder} + */ + @SafeVarargs + public final Builder addElements( + TimestampedValue element, TimestampedValue... elements) { + events.add(ElementEvent.add(element, elements)); + return this; + } + + /** + * Advance the watermark of this source to the specified instant. + * + *

    The watermark must advance monotonically and to at most {@link + * BoundedWindow#TIMESTAMP_MAX_VALUE}. + * + * @return this {@link TestStream.Builder} + */ + public Builder advanceWatermarkTo(Instant newWatermark) { + checkArgument( + newWatermark.isAfter(currentWatermark), "The watermark must monotonically advance"); + checkArgument( + newWatermark.isBefore(BoundedWindow.TIMESTAMP_MAX_VALUE), + "The Watermark cannot progress beyond the maximum. Got: %s. Maximum: %s", + newWatermark, + BoundedWindow.TIMESTAMP_MAX_VALUE); + events.add(WatermarkEvent.advanceTo(newWatermark)); + currentWatermark = newWatermark; + return this; + } + + /** + * Advance the processing time by the specified amount. + * + * @return this {@link TestStream.Builder} + */ + public Builder advanceProcessingTime(Duration amount) { + checkArgument( + amount.getMillis() > 0, + "Must advance the processing time by a positive amount. Got: ", + amount); + events.add(ProcessingTimeEvent.advanceBy(amount)); + return this; + } + + /** + * Advance the watermark to infinity, completing this {@link TestStream}. Future calls to the + * same builder will not affect the returned {@link TestStream}. + */ + public TestStream advanceWatermarkToInfinity() { + events.add(WatermarkEvent.advanceTo(BoundedWindow.TIMESTAMP_MAX_VALUE)); + return new TestStream<>(coder, events.build()); + } + } + + /** + * An event in a {@link TestStream}. A marker interface for all events that happen while + * evaluating a {@link TestStream}. + */ + public interface Event { + EventType getType(); + } + + /** + * The types of {@link Event} that are supported by {@link TestStream}. + */ + public enum EventType { + ELEMENT, + WATERMARK, + PROCESSING_TIME + } + + /** A {@link Event} that produces elements. */ + @AutoValue + public abstract static class ElementEvent implements Event { + public abstract Iterable> getElements(); + + @SafeVarargs + static Event add(TimestampedValue element, TimestampedValue... elements) { + return add(ImmutableList.>builder().add(element).add(elements).build()); + } + + static Event add(Iterable> elements) { + return new AutoValue_TestStream_ElementEvent<>(EventType.ELEMENT, elements); + } + } + + /** A {@link Event} that advances the watermark. */ + @AutoValue + public abstract static class WatermarkEvent implements Event { + public abstract Instant getWatermark(); + + static Event advanceTo(Instant newWatermark) { + return new AutoValue_TestStream_WatermarkEvent<>(EventType.WATERMARK, newWatermark); + } + } + + /** A {@link Event} that advances the processing time clock. */ + @AutoValue + public abstract static class ProcessingTimeEvent implements Event { + public abstract Duration getProcessingTimeAdvance(); + + static Event advanceBy(Duration amount) { + return new AutoValue_TestStream_ProcessingTimeEvent<>(EventType.PROCESSING_TIME, amount); + } + } + + @Override + public PCollection apply(PBegin input) { + return PCollection.createPrimitiveOutputInternal( + input.getPipeline(), WindowingStrategy.globalDefault(), IsBounded.UNBOUNDED) + .setCoder(coder); + } + + public List> getStreamEvents() { + return events; + } + + /** + * A {@link Coder} that encodes and decodes {@link TestStream.Event Events}. + * + * @param the type of elements in {@link ElementEvent ElementEvents} encoded and decoded by + * this {@link EventCoder} + */ + @VisibleForTesting + static final class EventCoder extends StandardCoder> { + private static final Coder DURATION_CODER = DurationCoder.of(); + private static final Coder INSTANT_CODER = InstantCoder.of(); + private final Coder valueCoder; + private final Coder>> elementCoder; + + public static EventCoder of(Coder valueCoder) { + return new EventCoder<>(valueCoder); + } + + @JsonCreator + public static EventCoder of( + @JsonProperty(PropertyNames.COMPONENT_ENCODINGS) List> components) { + checkArgument( + components.size() == 1, + "Was expecting exactly one component coder, got %s", + components.size()); + return new EventCoder<>((Coder) components.get(0)); + } + + private EventCoder(Coder valueCoder) { + this.valueCoder = valueCoder; + this.elementCoder = IterableCoder.of(TimestampedValueCoder.of(valueCoder)); + } + + @Override + public void encode( + Event value, OutputStream outStream, Context context) + throws IOException { + VarInt.encode(value.getType().ordinal(), outStream); + switch (value.getType()) { + case ELEMENT: + Iterable> elems = ((ElementEvent) value).getElements(); + elementCoder.encode(elems, outStream, context); + break; + case WATERMARK: + Instant ts = ((WatermarkEvent) value).getWatermark(); + INSTANT_CODER.encode(ts, outStream, context); + break; + case PROCESSING_TIME: + Duration processingAdvance = ((ProcessingTimeEvent) value).getProcessingTimeAdvance(); + DURATION_CODER.encode(processingAdvance, outStream, context); + break; + default: + throw new AssertionError("Unreachable"); + } + } + + @Override + public Event decode( + InputStream inStream, Context context) throws IOException { + switch (EventType.values()[VarInt.decodeInt(inStream)]) { + case ELEMENT: + Iterable> elements = elementCoder.decode(inStream, context); + return ElementEvent.add(elements); + case WATERMARK: + return WatermarkEvent.advanceTo(INSTANT_CODER.decode(inStream, context)); + case PROCESSING_TIME: + return ProcessingTimeEvent.advanceBy( + DURATION_CODER.decode(inStream, context).toDuration()); + default: + throw new AssertionError("Unreachable"); + } + } + + @Override + public List> getCoderArguments() { + return Collections.singletonList(valueCoder); + } + + @Override + public void verifyDeterministic() throws NonDeterministicException { + elementCoder.verifyDeterministic(); + DURATION_CODER.verifyDeterministic(); + INSTANT_CODER.verifyDeterministic(); + } + } +} 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 new file mode 100644 index 0000000000000..09bccfa171268 --- /dev/null +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/TestStreamTest.java @@ -0,0 +1,169 @@ +/* + * 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 org.hamcrest.Matchers.allOf; +import static org.hamcrest.Matchers.greaterThanOrEqualTo; +import static org.hamcrest.Matchers.lessThanOrEqualTo; +import static org.junit.Assert.assertThat; + +import org.apache.beam.sdk.coders.VarIntCoder; +import org.apache.beam.sdk.coders.VarLongCoder; +import org.apache.beam.sdk.transforms.Count; +import org.apache.beam.sdk.transforms.Flatten; +import org.apache.beam.sdk.transforms.GroupByKey; +import org.apache.beam.sdk.transforms.SerializableFunction; +import org.apache.beam.sdk.transforms.Sum; +import org.apache.beam.sdk.transforms.Values; +import org.apache.beam.sdk.transforms.WithKeys; +import org.apache.beam.sdk.transforms.windowing.AfterPane; +import org.apache.beam.sdk.transforms.windowing.AfterProcessingTime; +import org.apache.beam.sdk.transforms.windowing.AfterWatermark; +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.transforms.windowing.Window.ClosingBehavior; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.TimestampedValue; + +import org.joda.time.Duration; +import org.joda.time.Instant; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +import java.io.Serializable; + +/** + * Tests for {@link TestStream}. + */ +@RunWith(JUnit4.class) +public class TestStreamTest implements Serializable { + @Test + @Category(NeedsRunner.class) + public void testLateDataAccumulating() { + Instant instant = new Instant(0); + TestStream source = TestStream.create(VarIntCoder.of()) + .addElements(TimestampedValue.of(1, instant), + TimestampedValue.of(2, instant), + TimestampedValue.of(3, instant)) + .advanceWatermarkTo(instant.plus(Duration.standardMinutes(6))) + // These elements are late but within the allowed lateness + .addElements(TimestampedValue.of(4, instant), TimestampedValue.of(5, instant)) + .advanceWatermarkTo(instant.plus(Duration.standardMinutes(20))) + // These elements are droppably late + .addElements(TimestampedValue.of(-1, instant), + TimestampedValue.of(-2, instant), + TimestampedValue.of(-3, instant)) + .advanceWatermarkToInfinity(); + + TestPipeline p = TestPipeline.create(); + PCollection windowed = p + .apply(source) + .apply(Window.into(FixedWindows.of(Duration.standardMinutes(5))).triggering( + AfterWatermark.pastEndOfWindow() + .withEarlyFirings(AfterProcessingTime.pastFirstElementInPane() + .plusDelayOf(Duration.standardMinutes(2))) + .withLateFirings(AfterPane.elementCountAtLeast(1))) + .accumulatingFiredPanes() + .withAllowedLateness(Duration.standardMinutes(5), ClosingBehavior.FIRE_ALWAYS)); + PCollection triggered = windowed.apply(WithKeys.of(1)) + .apply(GroupByKey.create()) + .apply(Values.>create()) + .apply(Flatten.iterables()); + PCollection count = windowed.apply(Count.globally().withoutDefaults()); + PCollection sum = windowed.apply(Sum.integersGlobally().withoutDefaults()); + + IntervalWindow window = new IntervalWindow(instant, instant.plus(Duration.standardMinutes(5L))); + PAssert.that(triggered) + .inFinalPane(window) + .containsInAnyOrder(1, 2, 3, 4, 5); + PAssert.that(triggered) + .inOnTimePane(window) + .containsInAnyOrder(1, 2, 3); + PAssert.that(count) + .inWindow(window) + .satisfies(new SerializableFunction, Void>() { + @Override + public Void apply(Iterable input) { + for (Long count : input) { + assertThat(count, allOf(greaterThanOrEqualTo(3L), lessThanOrEqualTo(5L))); + } + return null; + } + }); + PAssert.that(sum) + .inWindow(window) + .satisfies(new SerializableFunction, Void>() { + @Override + public Void apply(Iterable input) { + for (Integer sum : input) { + assertThat(sum, allOf(greaterThanOrEqualTo(6), lessThanOrEqualTo(15))); + } + return null; + } + }); + + p.run(); + } + + @Test + @Category(NeedsRunner.class) + public void testProcessingTimeTrigger() { + TestStream source = TestStream.create(VarLongCoder.of()) + .addElements(TimestampedValue.of(1L, new Instant(1000L)), + TimestampedValue.of(2L, new Instant(2000L))) + .advanceProcessingTime(Duration.standardMinutes(12)) + .addElements(TimestampedValue.of(3L, new Instant(3000L))) + .advanceProcessingTime(Duration.standardMinutes(6)) + .advanceWatermarkToInfinity(); + + TestPipeline p = TestPipeline.create(); + PCollection sum = p.apply(source) + .apply(Window.triggering(AfterWatermark.pastEndOfWindow() + .withEarlyFirings(AfterProcessingTime.pastFirstElementInPane() + .plusDelayOf(Duration.standardMinutes(5)))).accumulatingFiredPanes() + .withAllowedLateness(Duration.ZERO)) + .apply(Sum.longsGlobally()); + + PAssert.that(sum).inEarlyGlobalWindowPanes().containsInAnyOrder(3L, 6L); + + p.run(); + } + + @Test + public void testEncodeDecode() throws Exception { + TestStream.Event elems = + TestStream.ElementEvent.add( + TimestampedValue.of(1, new Instant()), + TimestampedValue.of(-10, new Instant()), + TimestampedValue.of(Integer.MAX_VALUE, new Instant())); + TestStream.Event wm = TestStream.WatermarkEvent.advanceTo(new Instant(100)); + TestStream.Event procTime = + TestStream.ProcessingTimeEvent.advanceBy(Duration.millis(90548)); + + TestStream.EventCoder coder = TestStream.EventCoder.of(VarIntCoder.of()); + + CoderProperties.coderSerializable(coder); + CoderProperties.coderDecodeEncodeEqual(coder, elems); + CoderProperties.coderDecodeEncodeEqual(coder, wm); + CoderProperties.coderDecodeEncodeEqual(coder, procTime); + } +} From f37dba846deed1fa5ec687819391dc62f7565b10 Mon Sep 17 00:00:00 2001 From: Thomas Groh Date: Mon, 15 Aug 2016 19:45:58 -0700 Subject: [PATCH 040/346] Implement TestStream in the DirectRunner --- runners/direct-java/pom.xml | 3 - .../beam/runners/direct/DirectOptions.java | 45 +--- .../beam/runners/direct/DirectRunner.java | 47 +++- .../runners/direct/EvaluationContext.java | 10 +- ...FixedThreadPoolExecutorServiceFactory.java | 45 ---- .../beam/runners/direct/NanosOffsetClock.java | 13 -- .../direct/TestStreamEvaluatorFactory.java | 204 ++++++++++++++++++ .../direct/TransformEvaluatorRegistry.java | 11 + .../direct/WriteWithShardingFactory.java | 2 +- .../runners/direct/EvaluationContextTest.java | 1 + .../apache/beam/sdk/testing/TestStream.java | 114 +++++++--- .../beam/sdk/testing/TestStreamTest.java | 159 ++++++++++++++ 12 files changed, 508 insertions(+), 146 deletions(-) delete mode 100644 runners/direct-java/src/main/java/org/apache/beam/runners/direct/FixedThreadPoolExecutorServiceFactory.java create mode 100644 runners/direct-java/src/main/java/org/apache/beam/runners/direct/TestStreamEvaluatorFactory.java diff --git a/runners/direct-java/pom.xml b/runners/direct-java/pom.xml index 8b0f91d858a8a..e06883f6e65d7 100644 --- a/runners/direct-java/pom.xml +++ b/runners/direct-java/pom.xml @@ -85,9 +85,6 @@ org.apache.beam:beam-sdks-java-core org.apache.beam:beam-runners-java-core - - org/apache/beam/sdk/testing/TestStreamTest.java - [ diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectOptions.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectOptions.java index 3901c04cebac1..798fda4c4e9dc 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectOptions.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectOptions.java @@ -17,59 +17,16 @@ */ package org.apache.beam.runners.direct; -import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.options.ApplicationNameOptions; import org.apache.beam.sdk.options.Default; import org.apache.beam.sdk.options.Description; -import org.apache.beam.sdk.options.Hidden; import org.apache.beam.sdk.options.PipelineOptions; -import org.apache.beam.sdk.options.Validation.Required; -import org.apache.beam.sdk.transforms.PTransform; - -import com.fasterxml.jackson.annotation.JsonIgnore; - -import java.util.concurrent.ExecutorService; -import java.util.concurrent.Executors; /** * Options that can be used to configure the {@link org.apache.beam.runners.direct.DirectRunner}. */ public interface DirectOptions extends PipelineOptions, ApplicationNameOptions { - /** - * Gets the {@link ExecutorServiceFactory} to use to create instances of {@link ExecutorService} - * to execute {@link PTransform PTransforms}. - * - *

    Note that {@link ExecutorService ExecutorServices} returned by the factory must ensure that - * it cannot enter a state in which it will not schedule additional pending work unless currently - * scheduled work completes, as this may cause the {@link Pipeline} to cease processing. - * - *

    Defaults to a {@link FixedThreadPoolExecutorServiceFactory}, which produces instances of - * {@link Executors#newCachedThreadPool()}. - */ - @JsonIgnore - @Required - @Hidden - @Default.InstanceFactory(FixedThreadPoolExecutorServiceFactory.class) - ExecutorServiceFactory getExecutorServiceFactory(); - - void setExecutorServiceFactory(ExecutorServiceFactory executorService); - - /** - * Gets the {@link Clock} used by this pipeline. The clock is used in place of accessing the - * system time when time values are required by the evaluator. - */ - @Default.InstanceFactory(NanosOffsetClock.Factory.class) - @JsonIgnore - @Required - @Hidden - @Description( - "The processing time source used by the pipeline. When the current time is " - + "needed by the evaluator, the result of clock#now() is used.") - Clock getClock(); - - void setClock(Clock clock); - - @Default.Boolean(false) + @Default.Boolean(true) @Description( "If the pipeline should shut down producers which have reached the maximum " + "representable watermark. If this is set to true, a pipeline in which all PTransforms " 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 f2b781ec5e358..68184dec7d852 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 @@ -19,6 +19,7 @@ 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; import org.apache.beam.sdk.AggregatorRetrievalException; import org.apache.beam.sdk.AggregatorValues; @@ -29,6 +30,7 @@ import org.apache.beam.sdk.io.Write; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.runners.PipelineRunner; +import org.apache.beam.sdk.testing.TestStream; import org.apache.beam.sdk.transforms.Aggregator; import org.apache.beam.sdk.transforms.AppliedPTransform; import org.apache.beam.sdk.transforms.GroupByKey; @@ -46,6 +48,7 @@ import org.apache.beam.sdk.values.PValue; import com.google.common.base.MoreObjects; +import com.google.common.base.Supplier; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; @@ -58,6 +61,7 @@ import java.util.HashMap; import java.util.Map; import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; /** * An In-Memory implementation of the Dataflow Programming Model. Supports Unbounded @@ -76,8 +80,9 @@ public class DirectRunner private static Map, PTransformOverrideFactory> defaultTransformOverrides = ImmutableMap., PTransformOverrideFactory>builder() - .put(GroupByKey.class, new DirectGroupByKeyOverrideFactory()) .put(CreatePCollectionView.class, new ViewOverrideFactory()) + .put(GroupByKey.class, new DirectGroupByKeyOverrideFactory()) + .put(TestStream.class, new DirectTestStreamFactory()) .put(Write.Bound.class, new WriteWithShardingFactory()) .build(); @@ -175,6 +180,8 @@ public static interface PCollectionViewWriter { //////////////////////////////////////////////////////////////////////////////////////////////// private final DirectOptions options; + private Supplier executorServiceSupplier = new FixedThreadPoolSupplier(); + private Supplier clockSupplier = new NanosOffsetClockSupplier(); public static DirectRunner fromOptions(PipelineOptions options) { return new DirectRunner(options.as(DirectOptions.class)); @@ -191,6 +198,14 @@ public DirectOptions getPipelineOptions() { return options; } + Supplier getClockSupplier() { + return clockSupplier; + } + + void setClockSupplier(Supplier supplier) { + this.clockSupplier = supplier; + } + @Override public OutputT apply( PTransform transform, InputT input) { @@ -223,6 +238,7 @@ public DirectPipelineResult run(Pipeline pipeline) { EvaluationContext context = EvaluationContext.create( getPipelineOptions(), + clockSupplier.get(), createBundleFactory(getPipelineOptions()), consumerTrackingVisitor.getRootTransforms(), consumerTrackingVisitor.getValueToConsumers(), @@ -230,14 +246,15 @@ public DirectPipelineResult run(Pipeline pipeline) { consumerTrackingVisitor.getViews()); // independent executor service for each run - ExecutorService executorService = - context.getPipelineOptions().getExecutorServiceFactory().create(); + ExecutorService executorService = executorServiceSupplier.get(); + + TransformEvaluatorRegistry registry = TransformEvaluatorRegistry.defaultRegistry(); PipelineExecutor executor = ExecutorServiceParallelExecutor.create( executorService, consumerTrackingVisitor.getValueToConsumers(), keyedPValueVisitor.getKeyedPValues(), - TransformEvaluatorRegistry.defaultRegistry(), + registry, defaultModelEnforcements(options), context); executor.start(consumerTrackingVisitor.getRootTransforms()); @@ -392,4 +409,26 @@ public State waitUntilFinish(Duration duration) throws IOException { "DirectPipelineResult does not support waitUntilFinish."); } } + + /** + * A {@link Supplier} that creates a {@link ExecutorService} based on + * {@link Executors#newFixedThreadPool(int)}. + */ + private static class FixedThreadPoolSupplier implements Supplier { + @Override + public ExecutorService get() { + return Executors.newFixedThreadPool(Runtime.getRuntime().availableProcessors()); + } + } + + + /** + * A {@link Supplier} that creates a {@link NanosOffsetClock}. + */ + private static class NanosOffsetClockSupplier implements Supplier { + @Override + public Clock get() { + return NanosOffsetClock.create(); + } + } } 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 94f28e2acb951..b9f159a2d5988 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 @@ -102,24 +102,26 @@ class EvaluationContext { public static EvaluationContext create( DirectOptions options, + Clock clock, BundleFactory bundleFactory, Collection> rootTransforms, Map>> valueToConsumers, Map, String> stepNames, Collection> views) { return new EvaluationContext( - options, bundleFactory, rootTransforms, valueToConsumers, stepNames, views); + options, clock, bundleFactory, rootTransforms, valueToConsumers, stepNames, views); } private EvaluationContext( DirectOptions options, + Clock clock, BundleFactory bundleFactory, Collection> rootTransforms, Map>> valueToConsumers, Map, String> stepNames, Collection> views) { this.options = checkNotNull(options); - this.clock = options.getClock(); + this.clock = clock; this.bundleFactory = checkNotNull(bundleFactory); checkNotNull(rootTransforms); checkNotNull(valueToConsumers); @@ -433,4 +435,8 @@ public boolean isDone() { public Instant now() { return clock.now(); } + + Clock getClock() { + return clock; + } } diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/FixedThreadPoolExecutorServiceFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/FixedThreadPoolExecutorServiceFactory.java deleted file mode 100644 index 74c42926b115f..0000000000000 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/FixedThreadPoolExecutorServiceFactory.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 org.apache.beam.runners.direct; - -import org.apache.beam.sdk.options.DefaultValueFactory; -import org.apache.beam.sdk.options.PipelineOptions; - -import java.util.concurrent.ExecutorService; -import java.util.concurrent.Executors; - -/** - * A {@link ExecutorServiceFactory} that produces fixed thread pools via - * {@link Executors#newFixedThreadPool(int)}, with the number of threads equal to the available - * processors as provided by {@link Runtime#availableProcessors()}. - */ -class FixedThreadPoolExecutorServiceFactory - implements DefaultValueFactory, ExecutorServiceFactory { - private static final FixedThreadPoolExecutorServiceFactory INSTANCE = - new FixedThreadPoolExecutorServiceFactory(); - - @Override - public ExecutorServiceFactory create(PipelineOptions options) { - return INSTANCE; - } - - @Override - public ExecutorService create() { - return Executors.newFixedThreadPool(Runtime.getRuntime().availableProcessors()); - } -} diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/NanosOffsetClock.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/NanosOffsetClock.java index ffdee9d14af2e..77fa1967c4345 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/NanosOffsetClock.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/NanosOffsetClock.java @@ -17,9 +17,6 @@ */ package org.apache.beam.runners.direct; -import org.apache.beam.sdk.options.DefaultValueFactory; -import org.apache.beam.sdk.options.PipelineOptions; - import org.joda.time.Instant; import java.util.concurrent.TimeUnit; @@ -46,14 +43,4 @@ public Instant now() { baseMillis + (TimeUnit.MILLISECONDS.convert( System.nanoTime() - nanosAtBaseMillis, TimeUnit.NANOSECONDS))); } - - /** - * Creates instances of {@link NanosOffsetClock}. - */ - public static class Factory implements DefaultValueFactory { - @Override - public Clock create(PipelineOptions options) { - return new NanosOffsetClock(); - } - } } 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 new file mode 100644 index 0000000000000..90a83b0b840f4 --- /dev/null +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TestStreamEvaluatorFactory.java @@ -0,0 +1,204 @@ +/* + * 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 com.google.common.base.Preconditions.checkState; + +import org.apache.beam.runners.direct.DirectRunner.CommittedBundle; +import org.apache.beam.runners.direct.DirectRunner.UncommittedBundle; +import org.apache.beam.sdk.Pipeline; +import org.apache.beam.sdk.runners.PipelineRunner; +import org.apache.beam.sdk.testing.TestStream; +import org.apache.beam.sdk.testing.TestStream.ElementEvent; +import org.apache.beam.sdk.testing.TestStream.Event; +import org.apache.beam.sdk.testing.TestStream.EventType; +import org.apache.beam.sdk.testing.TestStream.ProcessingTimeEvent; +import org.apache.beam.sdk.testing.TestStream.WatermarkEvent; +import org.apache.beam.sdk.transforms.AppliedPTransform; +import org.apache.beam.sdk.transforms.PTransform; +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.values.PBegin; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.PCollection.IsBounded; +import org.apache.beam.sdk.values.PInput; +import org.apache.beam.sdk.values.POutput; +import org.apache.beam.sdk.values.TimestampedValue; + +import com.google.common.base.Supplier; + +import org.joda.time.Duration; +import org.joda.time.Instant; + +import java.util.List; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicReference; + +import javax.annotation.Nullable; + +/** + * The {@link TransformEvaluatorFactory} for the {@link TestStream} primitive. + */ +class TestStreamEvaluatorFactory implements TransformEvaluatorFactory { + private final AtomicBoolean inUse = new AtomicBoolean(false); + private final AtomicReference> evaluator = new AtomicReference<>(); + + @Nullable + @Override + public TransformEvaluator forApplication( + AppliedPTransform application, + @Nullable CommittedBundle inputBundle, + EvaluationContext evaluationContext) throws Exception { + return createEvaluator((AppliedPTransform) application, evaluationContext); + } + + @Override + public void cleanup() throws Exception {} + + private TransformEvaluator createEvaluator( + AppliedPTransform, TestStream> application, + EvaluationContext evaluationContext) { + if (evaluator.get() == null) { + Evaluator createdEvaluator = new Evaluator<>(application, evaluationContext, inUse); + evaluator.compareAndSet(null, createdEvaluator); + } + if (inUse.compareAndSet(false, true)) { + return evaluator.get(); + } else { + return null; + } + } + + private static class Evaluator implements TransformEvaluator { + private final AppliedPTransform, TestStream> application; + private final EvaluationContext context; + private final AtomicBoolean inUse; + private final List> events; + private int index; + private Instant currentWatermark; + + private Evaluator( + AppliedPTransform, TestStream> application, + EvaluationContext context, + AtomicBoolean inUse) { + this.application = application; + this.context = context; + this.inUse = inUse; + this.events = application.getTransform().getEvents(); + index = 0; + currentWatermark = BoundedWindow.TIMESTAMP_MIN_VALUE; + } + + @Override + public void processElement(WindowedValue element) throws Exception { + } + + @Override + public TransformResult finishBundle() throws Exception { + if (index >= events.size()) { + return StepTransformResult.withHold(application, BoundedWindow.TIMESTAMP_MAX_VALUE).build(); + } + Event event = events.get(index); + if (event.getType().equals(EventType.WATERMARK)) { + currentWatermark = ((WatermarkEvent) event).getWatermark(); + } + StepTransformResult.Builder result = + StepTransformResult.withHold(application, currentWatermark); + if (event.getType().equals(EventType.ELEMENT)) { + UncommittedBundle bundle = context.createRootBundle(application.getOutput()); + for (TimestampedValue elem : ((ElementEvent) event).getElements()) { + bundle.add(WindowedValue.timestampedValueInGlobalWindow(elem.getValue(), + elem.getTimestamp())); + } + result.addOutput(bundle); + } + if (event.getType().equals(EventType.PROCESSING_TIME)) { + ((TestClock) context.getClock()) + .advance(((ProcessingTimeEvent) event).getProcessingTimeAdvance()); + } + index++; + checkState(inUse.compareAndSet(true, false), + "The InUse flag of a %s was changed while the source evaluator was executing. " + + "%s cannot be split or evaluated in parallel.", + TestStream.class.getSimpleName(), + TestStream.class.getSimpleName()); + return result.build(); + } + } + + private static class TestClock implements Clock { + private final AtomicReference currentTime = + new AtomicReference<>(BoundedWindow.TIMESTAMP_MIN_VALUE); + + public void advance(Duration amount) { + Instant now = currentTime.get(); + currentTime.compareAndSet(now, now.plus(amount)); + } + + @Override + public Instant now() { + return currentTime.get(); + } + } + + private static class TestClockSupplier implements Supplier { + @Override + public Clock get() { + return new TestClock(); + } + } + + static class DirectTestStreamFactory implements PTransformOverrideFactory { + @Override + public PTransform override( + PTransform transform) { + if (transform instanceof TestStream) { + return (PTransform) + new DirectTestStream((TestStream) transform); + } + return transform; + } + + private static class DirectTestStream extends PTransform> { + private final TestStream original; + + private DirectTestStream(TestStream transform) { + this.original = transform; + } + + @Override + public PCollection apply(PBegin input) { + setup(input.getPipeline()); + return PCollection.createPrimitiveOutputInternal( + input.getPipeline(), WindowingStrategy.globalDefault(), IsBounded.UNBOUNDED) + .setCoder(original.getValueCoder()); + } + + private void setup(Pipeline p) { + PipelineRunner runner = p.getRunner(); + checkState(runner instanceof DirectRunner, + "%s can only be used when running with the %s", + getClass().getSimpleName(), + DirectRunner.class.getSimpleName()); + ((DirectRunner) runner).setClockSupplier(new TestClockSupplier()); + } + } + } +} 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 b46923775c484..c35e8b14c84a2 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 @@ -23,6 +23,7 @@ import org.apache.beam.runners.direct.DirectGroupByKey.DirectGroupByKeyOnly; import org.apache.beam.runners.direct.DirectRunner.CommittedBundle; import org.apache.beam.sdk.io.Read; +import org.apache.beam.sdk.testing.TestStream; import org.apache.beam.sdk.transforms.AppliedPTransform; import org.apache.beam.sdk.transforms.Flatten.FlattenPCollectionList; import org.apache.beam.sdk.transforms.PTransform; @@ -61,6 +62,7 @@ public static TransformEvaluatorRegistry defaultRegistry() { // Runner-specific primitives used in expansion of GroupByKey .put(DirectGroupByKeyOnly.class, new GroupByKeyOnlyEvaluatorFactory()) .put(DirectGroupAlsoByWindow.class, new GroupAlsoByWindowEvaluatorFactory()) + .put(TestStream.class, new TestStreamEvaluatorFactory()) .build(); return new TransformEvaluatorRegistry(primitives); } @@ -117,4 +119,13 @@ public void cleanup() throws Exception { throw toThrow; } } + + /** + * A factory to create Transform Evaluator Registries. + */ + public static class Factory { + public TransformEvaluatorRegistry create() { + return TransformEvaluatorRegistry.defaultRegistry(); + } + } } 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 c2157b8634ce0..1ab3403f45bc2 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 @@ public PTransform extends PTransform, PDone> { + private static class DynamicallyReshardedWrite extends PTransform, PDone> { private final transient Write.Bound original; private DynamicallyReshardedWrite(Bound original) { 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 d4b577349166d..7ac0caadc55bc 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 @@ -117,6 +117,7 @@ public void setup() { context = EvaluationContext.create( runner.getPipelineOptions(), + NanosOffsetClock.create(), ImmutableListBundleFactory.create(), rootTransforms, valueToConsumers, 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 6d11f72bb9791..e2eda32228556 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 @@ -32,10 +32,8 @@ import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.util.PropertyNames; import org.apache.beam.sdk.util.VarInt; -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.TimestampedValue; import org.apache.beam.sdk.values.TimestampedValue.TimestampedValueCoder; @@ -83,30 +81,30 @@ private TestStream(Coder coder, List> events) { this.events = checkNotNull(events); } - public Coder> getEventCoder() { - return EventCoder.of(coder); - } - /** * An incomplete {@link TestStream}. Elements added to this builder will be produced in sequence * when the pipeline created by the {@link TestStream} is run. */ public static class Builder { private final Coder coder; - private final ImmutableList.Builder> events; - private Instant currentWatermark; + private final ImmutableList> events; + private final Instant currentWatermark; private Builder(Coder coder) { - this.coder = coder; - events = ImmutableList.builder(); + this(coder, ImmutableList.>of(), BoundedWindow.TIMESTAMP_MIN_VALUE); + } - currentWatermark = BoundedWindow.TIMESTAMP_MIN_VALUE; + private Builder(Coder coder, ImmutableList> events, Instant currentWatermark) { + this.coder = coder; + this.events = events; + this.currentWatermark = currentWatermark; } /** * Adds the specified elements to the source with timestamp equal to the current watermark. * - * @return this {@link TestStream.Builder} + * @return A {@link TestStream.Builder} like this one that will add the provided elements + * after all earlier events have completed. */ @SafeVarargs public final Builder addElements(T element, T... elements) { @@ -122,22 +120,40 @@ public final Builder addElements(T element, T... elements) { /** * Adds the specified elements to the source with the provided timestamps. * - * @return this {@link TestStream.Builder} + * @return A {@link TestStream.Builder} like this one that will add the provided elements + * after all earlier events have completed. */ @SafeVarargs public final Builder addElements( TimestampedValue element, TimestampedValue... elements) { - events.add(ElementEvent.add(element, elements)); - return this; + checkArgument( + element.getTimestamp().isBefore(BoundedWindow.TIMESTAMP_MAX_VALUE), + "Elements must have timestamps before %s. Got: %s", + BoundedWindow.TIMESTAMP_MAX_VALUE, + element.getTimestamp()); + for (TimestampedValue multiElement : elements) { + checkArgument( + multiElement.getTimestamp().isBefore(BoundedWindow.TIMESTAMP_MAX_VALUE), + "Elements must have timestamps before %s. Got: %s", + BoundedWindow.TIMESTAMP_MAX_VALUE, + multiElement.getTimestamp()); + } + ImmutableList> newEvents = + ImmutableList.>builder() + .addAll(events) + .add(ElementEvent.add(element, elements)) + .build(); + return new Builder(coder, newEvents, currentWatermark); } /** * Advance the watermark of this source to the specified instant. * - *

    The watermark must advance monotonically and to at most {@link - * BoundedWindow#TIMESTAMP_MAX_VALUE}. + *

    The watermark must advance monotonically and cannot advance to {@link + * BoundedWindow#TIMESTAMP_MAX_VALUE} or beyond. * - * @return this {@link TestStream.Builder} + * @return A {@link TestStream.Builder} like this one that will advance the watermark to the + * specified point after all earlier events have completed. */ public Builder advanceWatermarkTo(Instant newWatermark) { checkArgument( @@ -147,23 +163,30 @@ public Builder advanceWatermarkTo(Instant newWatermark) { "The Watermark cannot progress beyond the maximum. Got: %s. Maximum: %s", newWatermark, BoundedWindow.TIMESTAMP_MAX_VALUE); - events.add(WatermarkEvent.advanceTo(newWatermark)); - currentWatermark = newWatermark; - return this; + ImmutableList> newEvents = ImmutableList.>builder() + .addAll(events) + .add(WatermarkEvent.advanceTo(newWatermark)) + .build(); + return new Builder(coder, newEvents, newWatermark); } /** * Advance the processing time by the specified amount. * - * @return this {@link TestStream.Builder} + * @return A {@link TestStream.Builder} like this one that will advance the processing time by + * the specified amount after all earlier events have completed. */ public Builder advanceProcessingTime(Duration amount) { checkArgument( amount.getMillis() > 0, "Must advance the processing time by a positive amount. Got: ", amount); - events.add(ProcessingTimeEvent.advanceBy(amount)); - return this; + ImmutableList> newEvents = + ImmutableList.>builder() + .addAll(events) + .add(ProcessingTimeEvent.advanceBy(amount)) + .build(); + return new Builder(coder, newEvents, currentWatermark); } /** @@ -171,8 +194,12 @@ public Builder advanceProcessingTime(Duration amount) { * same builder will not affect the returned {@link TestStream}. */ public TestStream advanceWatermarkToInfinity() { - events.add(WatermarkEvent.advanceTo(BoundedWindow.TIMESTAMP_MAX_VALUE)); - return new TestStream<>(coder, events.build()); + ImmutableList> newEvents = + ImmutableList.>builder() + .addAll(events) + .add(WatermarkEvent.advanceTo(BoundedWindow.TIMESTAMP_MAX_VALUE)) + .build(); + return new TestStream<>(coder, newEvents); } } @@ -230,12 +257,30 @@ static Event advanceBy(Duration amount) { @Override public PCollection apply(PBegin input) { - return PCollection.createPrimitiveOutputInternal( - input.getPipeline(), WindowingStrategy.globalDefault(), IsBounded.UNBOUNDED) - .setCoder(coder); + throw new IllegalStateException( + String.format( + "Pipeline Runner %s does not provide a required override for %s", + input.getPipeline().getRunner().getClass().getSimpleName(), + getClass().getSimpleName())); + } + + public Coder getValueCoder() { + return coder; + } + + /** + * Returns a coder suitable for encoding {@link TestStream.Event}. + */ + public Coder> getEventCoder() { + return EventCoder.of(coder); } - public List> getStreamEvents() { + /** + * Returns the sequence of {@link Event Events} in this {@link TestStream}. + * + *

    For use by {@link PipelineRunner} authors. + */ + public List> getEvents() { return events; } @@ -243,7 +288,7 @@ public List> getStreamEvents() { * A {@link Coder} that encodes and decodes {@link TestStream.Event Events}. * * @param the type of elements in {@link ElementEvent ElementEvents} encoded and decoded by - * this {@link EventCoder} + * this {@link EventCoder} */ @VisibleForTesting static final class EventCoder extends StandardCoder> { @@ -290,14 +335,15 @@ public void encode( DURATION_CODER.encode(processingAdvance, outStream, context); break; default: - throw new AssertionError("Unreachable"); + throw new AssertionError("Unreachable: Unsupported Event Type " + value.getType()); } } @Override public Event decode( InputStream inStream, Context context) throws IOException { - switch (EventType.values()[VarInt.decodeInt(inStream)]) { + EventType eventType = EventType.values()[VarInt.decodeInt(inStream)]; + switch (eventType) { case ELEMENT: Iterable> elements = elementCoder.decode(inStream, context); return ElementEvent.add(elements); @@ -307,7 +353,7 @@ public Event decode( return ProcessingTimeEvent.advanceBy( DURATION_CODER.decode(inStream, context).toDuration()); default: - throw new AssertionError("Unreachable"); + throw new AssertionError("Unreachable: Unsupported Event Type " + eventType); } } 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 09bccfa171268..df37d7f967a54 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 @@ -23,8 +23,13 @@ import static org.hamcrest.Matchers.lessThanOrEqualTo; import static org.junit.Assert.assertThat; +import org.apache.beam.sdk.Pipeline; +import org.apache.beam.sdk.coders.StringUtf8Coder; import org.apache.beam.sdk.coders.VarIntCoder; import org.apache.beam.sdk.coders.VarLongCoder; +import org.apache.beam.sdk.options.PipelineOptions; +import org.apache.beam.sdk.options.PipelineOptionsFactory; +import org.apache.beam.sdk.testing.TestStream.Builder; import org.apache.beam.sdk.transforms.Count; import org.apache.beam.sdk.transforms.Flatten; import org.apache.beam.sdk.transforms.GroupByKey; @@ -35,8 +40,12 @@ import org.apache.beam.sdk.transforms.windowing.AfterPane; import org.apache.beam.sdk.transforms.windowing.AfterProcessingTime; import org.apache.beam.sdk.transforms.windowing.AfterWatermark; +import org.apache.beam.sdk.transforms.windowing.BoundedWindow; +import org.apache.beam.sdk.transforms.windowing.DefaultTrigger; 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.Never; import org.apache.beam.sdk.transforms.windowing.Window; import org.apache.beam.sdk.transforms.windowing.Window.ClosingBehavior; import org.apache.beam.sdk.values.PCollection; @@ -44,8 +53,10 @@ 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.rules.ExpectedException; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; @@ -56,6 +67,8 @@ */ @RunWith(JUnit4.class) public class TestStreamTest implements Serializable { + @Rule public transient ExpectedException thrown = ExpectedException.none(); + @Test @Category(NeedsRunner.class) public void testLateDataAccumulating() { @@ -148,6 +161,152 @@ public void testProcessingTimeTrigger() { p.run(); } + @Test + @Category(NeedsRunner.class) + public void testDiscardingMode() { + TestStream stream = + TestStream.create(StringUtf8Coder.of()) + .advanceWatermarkTo(new Instant(0)) + .addElements( + TimestampedValue.of("firstPane", new Instant(100)), + TimestampedValue.of("alsoFirstPane", new Instant(200))) + .addElements(TimestampedValue.of("onTimePane", new Instant(500))) + .advanceWatermarkTo(new Instant(1001L)) + .addElements( + TimestampedValue.of("finalLatePane", new Instant(750)), + 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 = + p.apply(stream) + .apply( + Window.into(windowFn) + .triggering( + AfterWatermark.pastEndOfWindow() + .withEarlyFirings(AfterPane.elementCountAtLeast(2)) + .withLateFirings(Never.ever())) + .discardingFiredPanes() + .withAllowedLateness(allowedLateness)) + .apply(WithKeys.of(1)) + .apply(GroupByKey.create()) + .apply(Values.>create()) + .apply(Flatten.iterables()); + + IntervalWindow window = windowFn.assignWindow(new Instant(100)); + PAssert.that(values) + .inWindow(window) + .containsInAnyOrder( + "firstPane", "alsoFirstPane", "onTimePane", "finalLatePane", "alsoFinalLatePane"); + PAssert.that(values) + .inCombinedNonLatePanes(window) + .containsInAnyOrder("firstPane", "alsoFirstPane", "onTimePane"); + PAssert.that(values).inOnTimePane(window).containsInAnyOrder("onTimePane"); + PAssert.that(values) + .inFinalPane(window) + .containsInAnyOrder("finalLatePane", "alsoFinalLatePane"); + + p.run(); + } + + @Test + @Category(NeedsRunner.class) + public void testFirstElementLate() { + Instant lateElementTimestamp = new Instant(-1_000_000); + TestStream stream = + TestStream.create(StringUtf8Coder.of()) + .advanceWatermarkTo(new Instant(0)) + .addElements(TimestampedValue.of("late", lateElementTimestamp)) + .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) + .apply(Window.into(windowFn).triggering(DefaultTrigger.of()) + .discardingFiredPanes() + .withAllowedLateness(allowedLateness)) + .apply(WithKeys.of(1)) + .apply(GroupByKey.create()) + .apply(Values.>create()) + .apply(Flatten.iterables()); + + PAssert.that(values).inWindow(windowFn.assignWindow(lateElementTimestamp)).empty(); + PAssert.that(values) + .inWindow(windowFn.assignWindow(new Instant(100))) + .containsInAnyOrder("onTime"); + + p.run(); + } + + @Test + @Category(NeedsRunner.class) + public void testElementsAtAlmostPositiveInfinity() { + Instant endOfGlobalWindow = GlobalWindow.INSTANCE.maxTimestamp(); + TestStream stream = TestStream.create(StringUtf8Coder.of()) + .addElements(TimestampedValue.of("foo", endOfGlobalWindow), + 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)) + .apply(WithKeys.of(1)) + .apply(GroupByKey.create()) + .apply(Values.>create()) + .apply(Flatten.iterables()); + + PAssert.that(windowedValues) + .inWindow(windows.assignWindow(GlobalWindow.INSTANCE.maxTimestamp())) + .containsInAnyOrder("foo", "bar"); + p.run(); + } + + @Test + public void testElementAtPositiveInfinityThrows() { + Builder stream = + TestStream.create(VarIntCoder.of()) + .addElements(TimestampedValue.of(-1, BoundedWindow.TIMESTAMP_MAX_VALUE.minus(1L))); + thrown.expect(IllegalArgumentException.class); + stream.addElements(TimestampedValue.of(1, BoundedWindow.TIMESTAMP_MAX_VALUE)); + } + + @Test + public void testAdvanceWatermarkNonMonotonicThrows() { + Builder stream = + TestStream.create(VarIntCoder.of()) + .advanceWatermarkTo(new Instant(0L)); + thrown.expect(IllegalArgumentException.class); + stream.advanceWatermarkTo(new Instant(-1L)); + } + + @Test + public void testAdvanceWatermarkEqualToPositiveInfinityThrows() { + Builder stream = + TestStream.create(VarIntCoder.of()) + .advanceWatermarkTo(BoundedWindow.TIMESTAMP_MAX_VALUE.minus(1L)); + thrown.expect(IllegalArgumentException.class); + stream.advanceWatermarkTo(BoundedWindow.TIMESTAMP_MAX_VALUE); + } + + @Test + public void testUnsupportedRunnerThrows() { + PipelineOptions opts = PipelineOptionsFactory.create(); + opts.setRunner(CrashingRunner.class); + + Pipeline p = Pipeline.create(opts); + + thrown.expect(IllegalStateException.class); + thrown.expectMessage("does not provide a required override"); + thrown.expectMessage(TestStream.class.getSimpleName()); + thrown.expectMessage(CrashingRunner.class.getSimpleName()); + p.apply(TestStream.create(VarIntCoder.of()).advanceWatermarkToInfinity()); + } + @Test public void testEncodeDecode() throws Exception { TestStream.Event elems = From dab9efcfc658ac2223d5202989bc0d204039f043 Mon Sep 17 00:00:00 2001 From: sammcveety Date: Thu, 18 Aug 2016 18:02:06 -0400 Subject: [PATCH 041/346] Incorporate private IP option for Dataflow runner --- pom.xml | 2 +- .../dataflow/DataflowPipelineTranslator.java | 7 +++++++ .../options/DataflowPipelineWorkerPoolOptions.java | 14 ++++++++++++++ 3 files changed, 22 insertions(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index f39f94aca693c..58adbe7a86cc9 100644 --- a/pom.xml +++ b/pom.xml @@ -105,7 +105,7 @@ v2-rev295-1.22.0 0.0.2 v2-rev8-1.22.0 - v1b3-rev30-1.22.0 + v1b3-rev36-1.22.0 0.5.160222 1.0.0-beta.2 1.0.0-beta 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 0ca92c849f431..b8d94452872c1 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 @@ -425,6 +425,13 @@ public Job translate(List packages) { workerPool.setMachineType(options.getWorkerMachineType()); } + if (options.getUsePublicIps() != null) { + if (options.getUsePublicIps()) { + workerPool.setIpConfiguration("WORKER_IP_PUBLIC"); + } else { + workerPool.setIpConfiguration("WORKER_IP_PRIVATE"); + } + } workerPool.setPackages(packages); workerPool.setNumWorkers(options.getNumWorkers()); 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 e3a1a0f5bdefb..9b7a9ceb7f742 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 @@ -29,6 +29,8 @@ import java.util.List; +import javax.annotation.Nullable; + /** * Options that are used to configure the Dataflow pipeline worker pool. */ @@ -260,4 +262,16 @@ public String getTeardownPolicyName() { + "https://cloud.google.com/compute/docs/reference/latest/diskTypes") String getWorkerDiskType(); void setWorkerDiskType(String value); + + /** + * Specifies whether worker pools should be started with public IP addresses. + * + *

    WARNING: This feature is experimental. You must be whitelisted to use it. + */ + @Description("Specifies whether worker pools should be started with public IP addresses. WARNING:" + + "This feature is experimental. You must be whitelisted to use it.") + @Experimental + @JsonIgnore + @Nullable Boolean getUsePublicIps(); + void setUsePublicIps(@Nullable Boolean value); } From a4053ac1787a45ba71b7054f2f54e804c6d7c3ac Mon Sep 17 00:00:00 2001 From: Vikas Kedigehalli Date: Wed, 17 Aug 2016 18:19:52 -0700 Subject: [PATCH 042/346] Datastore Sink support for writing Mutations This generalizes Write to Write and Delete cleanly. --- .../beam/sdk/io/gcp/datastore/V1Beta3.java | 436 ++++++++++++------ .../sdk/io/gcp/datastore/V1Beta3Test.java | 237 ++++++++-- 2 files changed, 508 insertions(+), 165 deletions(-) diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/datastore/V1Beta3.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/datastore/V1Beta3.java index 0d2e2cb72cab9..8503b668e25af 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/datastore/V1Beta3.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/datastore/V1Beta3.java @@ -24,6 +24,7 @@ import static com.google.datastore.v1beta3.PropertyFilter.Operator.EQUAL; import static com.google.datastore.v1beta3.PropertyOrder.Direction.DESCENDING; import static com.google.datastore.v1beta3.QueryResultBatch.MoreResultsType.NOT_FINISHED; +import static com.google.datastore.v1beta3.client.DatastoreHelper.makeDelete; import static com.google.datastore.v1beta3.client.DatastoreHelper.makeFilter; import static com.google.datastore.v1beta3.client.DatastoreHelper.makeOrder; import static com.google.datastore.v1beta3.client.DatastoreHelper.makeUpsert; @@ -36,8 +37,10 @@ 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.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.Values; import org.apache.beam.sdk.transforms.display.DisplayData; import org.apache.beam.sdk.transforms.display.DisplayData.Builder; @@ -60,6 +63,7 @@ import com.google.datastore.v1beta3.EntityResult; import com.google.datastore.v1beta3.Key; import com.google.datastore.v1beta3.Key.PathElement; +import com.google.datastore.v1beta3.Mutation; import com.google.datastore.v1beta3.PartitionId; import com.google.datastore.v1beta3.Query; import com.google.datastore.v1beta3.QueryResultBatch; @@ -84,7 +88,7 @@ import javax.annotation.Nullable; /** - *

    {@link V1Beta3} provides an API to Read and Write {@link PCollection PCollections} of + *

    {@link V1Beta3} provides an API to Read, Write and Delete {@link PCollection PCollections} of * Google Cloud Datastore version v1beta3 * {@link Entity} objects. * @@ -129,7 +133,25 @@ * p.run(); * } * - *

    {@link Entity Entities} in the {@code PCollection} to be written must have complete + *

    To delete a {@link PCollection} of {@link Entity Entities} from Datastore, use + * {@link V1Beta3#deleteEntity()}, specifying the Cloud Datastore project to write to: + * + *

     {@code
    + * PCollection entities = ...;
    + * entities.apply(DatastoreIO.v1beta3().deleteEntity().withProjectId(projectId));
    + * p.run();
    + * } 
    + * + *

    To delete entities associated with a {@link PCollection} of {@link Key Keys} from Datastore, + * use {@link V1Beta3#deleteKey}, specifying the Cloud Datastore project to write to: + * + *

     {@code
    + * PCollection entities = ...;
    + * entities.apply(DatastoreIO.v1beta3().deleteKey().withProjectId(projectId));
    + * p.run();
    + * } 
    + * + *

    {@link Entity Entities} in the {@code PCollection} to be written or deleted must have complete * {@link Key Keys}. Complete {@code Keys} specify the {@code name} and {@code id} of the * {@code Entity}, where incomplete {@code Keys} do not. A {@code namespace} other than * {@code projectId} default may be used by specifying it in the {@code Entity} {@code Keys}. @@ -139,9 +161,9 @@ * keyBuilder.getPartitionIdBuilder().setNamespace(namespace); * } * - *

    {@code Entities} will be committed as upsert (update or insert) mutations. Please read - * Entities, Properties, and - * Keys for more information about {@code Entity} keys. + *

    {@code Entities} will be committed as upsert (update or insert) or delete mutations. Please + * read Entities, Properties, + * and Keys for more information about {@code Entity} keys. * *

    Permissions

    * Permission requirements depend on the {@code PipelineRunner} that is used to execute the @@ -641,21 +663,34 @@ public Write write() { return new Write(null); } + /** + * Returns an empty {@link DeleteEntity} builder. Configure the destination + * {@code projectId} using {@link DeleteEntity#withProjectId}. + */ + public DeleteEntity deleteEntity() { + return new DeleteEntity(null); + } + + /** + * Returns an empty {@link DeleteKey} builder. Configure the destination + * {@code projectId} using {@link DeleteKey#withProjectId}. + */ + public DeleteKey deleteKey() { + return new DeleteKey(null); + } + /** * A {@link PTransform} that writes {@link Entity} objects to Cloud Datastore. * * @see DatastoreIO */ - public static class Write extends PTransform, PDone> { - @Nullable - private final String projectId; - + public static class Write extends Mutate { /** * Note that {@code projectId} is only {@code @Nullable} as a matter of build order, but if * it is {@code null} at instantiation time, an error will be thrown. */ - public Write(@Nullable String projectId) { - this.projectId = projectId; + Write(@Nullable String projectId) { + super(projectId, new UpsertFn()); } /** @@ -665,27 +700,99 @@ public Write withProjectId(String projectId) { checkNotNull(projectId, "projectId"); return new Write(projectId); } + } - @Override - public PDone apply(PCollection input) { - input.apply(ParDo.of(new DatastoreWriterFn(projectId))); - return PDone.in(input.getPipeline()); + /** + * A {@link PTransform} that deletes {@link Entity Entities} from Cloud Datastore. + * + * @see DatastoreIO + */ + public static class DeleteEntity extends Mutate { + /** + * Note that {@code projectId} is only {@code @Nullable} as a matter of build order, but if + * it is {@code null} at instantiation time, an error will be thrown. + */ + DeleteEntity(@Nullable String projectId) { + super(projectId, new DeleteEntityFn()); } - @Override - public void validate(PCollection input) { + /** + * Returns a new {@link DeleteEntity} that deletes entities from the Cloud Datastore for the + * specified project. + */ + public DeleteEntity withProjectId(String projectId) { + checkNotNull(projectId, "projectId"); + return new DeleteEntity(projectId); + } + } + + /** + * A {@link PTransform} that deletes {@link Entity Entities} associated with the given + * {@link Key Keys} from Cloud Datastore. + * + * @see DatastoreIO + */ + public static class DeleteKey extends Mutate { + /** + * Note that {@code projectId} is only {@code @Nullable} as a matter of build order, but if + * it is {@code null} at instantiation time, an error will be thrown. + */ + DeleteKey(@Nullable String projectId) { + super(projectId, new DeleteKeyFn()); + } + + /** + * Returns a new {@link DeleteKey} that deletes entities from the Cloud Datastore for the + * specified project. + */ + public DeleteKey withProjectId(String projectId) { checkNotNull(projectId, "projectId"); + return new DeleteKey(projectId); } + } + /** + * A {@link PTransform} that writes mutations to Cloud Datastore. + * + *

    It requires a {@link DoFn} that tranforms an object of type {@code T} to a {@link Mutation}. + * {@code T} is usually either an {@link Entity} or a {@link Key} + * Note: Only idempotent Cloud Datastore mutation operations (upsert and delete) should + * be used by the {@code DoFn} provided, as the commits are retried when failures occur. + */ + private abstract static class Mutate extends PTransform, PDone> { @Nullable - public String getProjectId() { - return projectId; + private final String projectId; + /** A function that transforms each {@code T} into a mutation. */ + private final SimpleFunction mutationFn; + + /** + * Note that {@code projectId} is only {@code @Nullable} as a matter of build order, but if + * it is {@code null} at instantiation time, an error will be thrown. + */ + Mutate(@Nullable String projectId, SimpleFunction mutationFn) { + this.projectId = projectId; + this.mutationFn = checkNotNull(mutationFn); + } + + @Override + public PDone apply(PCollection input) { + input.apply("Convert to Mutation", MapElements.via(mutationFn)) + .apply("Write Mutation to Datastore", ParDo.of(new DatastoreWriterFn(projectId))); + + return PDone.in(input.getPipeline()); + } + + @Override + public void validate(PCollection input) { + checkNotNull(projectId, "projectId"); + checkNotNull(mutationFn, "mutationFn"); } @Override public String toString() { return MoreObjects.toStringHelper(getClass()) .add("projectId", projectId) + .add("mutationFn", mutationFn.getClass().getName()) .toString(); } @@ -694,141 +801,200 @@ public void populateDisplayData(DisplayData.Builder builder) { super.populateDisplayData(builder); builder .addIfNotNull(DisplayData.item("projectId", projectId) - .withLabel("Output Project")); + .withLabel("Output Project")) + .include(mutationFn); } + public String getProjectId() { + return projectId; + } + } + + /** + * {@link DoFn} that writes {@link Mutation}s to Cloud Datastore. Mutations are written in + * batches, where the maximum batch size is {@link V1Beta3#DATASTORE_BATCH_UPDATE_LIMIT}. + * + *

    See + * Datastore: Entities, Properties, and Keys for information about entity keys and mutations. + * + *

    Commits are non-transactional. If a commit fails because of a conflict over an entity + * group, the commit will be retried (up to {@link V1Beta3#DATASTORE_BATCH_UPDATE_LIMIT} + * times). This means that the mutation operation should be idempotent. Thus, the writer should + * only be used for {code upsert} and {@code delete} mutation operations, as these are the only + * two Cloud Datastore mutations that are idempotent. + */ + @VisibleForTesting + static class DatastoreWriterFn extends DoFn { + private static final Logger LOG = LoggerFactory.getLogger(DatastoreWriterFn.class); + private final String projectId; + private transient Datastore datastore; + private final V1Beta3DatastoreFactory datastoreFactory; + // Current batch of mutations to be written. + private final List mutations = new ArrayList<>(); + /** + * Since a bundle is written in batches, we should retry the commit of a batch in order to + * prevent transient errors from causing the bundle to fail. + */ + private static final int MAX_RETRIES = 5; + /** - * A {@link DoFn} that writes {@link Entity} objects to Cloud Datastore. Entities are written in - * batches, where the maximum batch size is {@link V1Beta3#DATASTORE_BATCH_UPDATE_LIMIT}. - * Entities are committed as upsert mutations (either update if the key already exists, or - * insert if it is a new key). If an entity does not have a complete key (i.e., it has no name - * or id), the bundle will fail. - * - *

    See - * Datastore: Entities, Properties, and Keys for information about entity keys and entities. - * - *

    Commits are non-transactional. If a commit fails because of a conflict over an entity - * group, the commit will be retried (up to {@link V1Beta3#DATASTORE_BATCH_UPDATE_LIMIT} - * times). + * Initial backoff time for exponential backoff for retry attempts. */ - @VisibleForTesting - static class DatastoreWriterFn extends DoFn { - private static final Logger LOG = LoggerFactory.getLogger(DatastoreWriterFn.class); - private final String projectId; - private transient Datastore datastore; - private final V1Beta3DatastoreFactory datastoreFactory; - // Current batch of entities to be written. - private final List entities = new ArrayList<>(); - /** - * Since a bundle is written in batches, we should retry the commit of a batch in order to - * prevent transient errors from causing the bundle to fail. - */ - private static final int MAX_RETRIES = 5; + private static final int INITIAL_BACKOFF_MILLIS = 5000; - /** - * Initial backoff time for exponential backoff for retry attempts. - */ - private static final int INITIAL_BACKOFF_MILLIS = 5000; + DatastoreWriterFn(String projectId) { + this(projectId, new V1Beta3DatastoreFactory()); + } - public DatastoreWriterFn(String projectId) { - this(projectId, new V1Beta3DatastoreFactory()); - } + @VisibleForTesting + DatastoreWriterFn(String projectId, V1Beta3DatastoreFactory datastoreFactory) { + this.projectId = checkNotNull(projectId, "projectId"); + this.datastoreFactory = datastoreFactory; + } - @VisibleForTesting - DatastoreWriterFn(String projectId, V1Beta3DatastoreFactory datastoreFactory) { - this.projectId = checkNotNull(projectId, "projectId"); - this.datastoreFactory = datastoreFactory; - } + @StartBundle + public void startBundle(Context c) { + datastore = datastoreFactory.getDatastore(c.getPipelineOptions(), projectId); + } - @StartBundle - public void startBundle(Context c) { - datastore = datastoreFactory.getDatastore(c.getPipelineOptions(), projectId); + @ProcessElement + public void processElement(ProcessContext c) throws Exception { + mutations.add(c.element()); + if (mutations.size() >= V1Beta3.DATASTORE_BATCH_UPDATE_LIMIT) { + flushBatch(); } + } - @ProcessElement - public void processElement(ProcessContext c) throws Exception { - // Verify that the entity to write has a complete key. - if (!isValidKey(c.element().getKey())) { - throw new IllegalArgumentException( - "Entities to be written to the Datastore must have complete keys"); - } - - entities.add(c.element()); - if (entities.size() >= V1Beta3.DATASTORE_BATCH_UPDATE_LIMIT) { - flushBatch(); - } + @FinishBundle + public void finishBundle(Context c) throws Exception { + if (mutations.size() > 0) { + flushBatch(); } + } - @FinishBundle - public void finishBundle(Context c) throws Exception { - if (entities.size() > 0) { - flushBatch(); - } - } + /** + * Writes a batch of mutations to Cloud Datastore. + * + *

    If a commit fails, it will be retried (up to {@link DatastoreWriterFn#MAX_RETRIES} + * times). All mutations in the batch will be committed again, even if the commit was partially + * successful. If the retry limit is exceeded, the last exception from the Datastore will be + * thrown. + * + * @throws DatastoreException if the commit fails or IOException or InterruptedException if + * backing off between retries fails. + */ + private void flushBatch() throws DatastoreException, IOException, InterruptedException { + LOG.debug("Writing batch of {} mutations", mutations.size()); + Sleeper sleeper = Sleeper.DEFAULT; + BackOff backoff = new AttemptBoundedExponentialBackOff(MAX_RETRIES, INITIAL_BACKOFF_MILLIS); - /** - * Writes a batch of entities to Cloud Datastore. - * - *

    If a commit fails, it will be retried (up to {@link DatastoreWriterFn#MAX_RETRIES} - * times). All entities in the batch will be committed again, even if the commit was partially - * successful. If the retry limit is exceeded, the last exception from the Datastore will be - * thrown. - * - * @throws DatastoreException if the commit fails or IOException or InterruptedException if - * backing off between retries fails. - */ - private void flushBatch() throws DatastoreException, IOException, InterruptedException { - LOG.debug("Writing batch of {} entities", entities.size()); - Sleeper sleeper = Sleeper.DEFAULT; - BackOff backoff = new AttemptBoundedExponentialBackOff(MAX_RETRIES, INITIAL_BACKOFF_MILLIS); - - while (true) { - // Batch upsert entities. - try { - CommitRequest.Builder commitRequest = CommitRequest.newBuilder(); - for (Entity entity: entities) { - commitRequest.addMutations(makeUpsert(entity)); - } - commitRequest.setMode(CommitRequest.Mode.NON_TRANSACTIONAL); - datastore.commit(commitRequest.build()); - // Break if the commit threw no exception. - break; - } catch (DatastoreException exception) { - // Only log the code and message for potentially-transient errors. The entire exception - // will be propagated upon the last retry. - LOG.error("Error writing to the Datastore ({}): {}", exception.getCode(), - exception.getMessage()); - if (!BackOffUtils.next(sleeper, backoff)) { - LOG.error("Aborting after {} retries.", MAX_RETRIES); - throw exception; - } + while (true) { + // Batch upsert entities. + try { + CommitRequest.Builder commitRequest = CommitRequest.newBuilder(); + commitRequest.addAllMutations(mutations); + commitRequest.setMode(CommitRequest.Mode.NON_TRANSACTIONAL); + datastore.commit(commitRequest.build()); + // Break if the commit threw no exception. + break; + } catch (DatastoreException exception) { + // Only log the code and message for potentially-transient errors. The entire exception + // will be propagated upon the last retry. + LOG.error("Error writing to the Datastore ({}): {}", exception.getCode(), + exception.getMessage()); + if (!BackOffUtils.next(sleeper, backoff)) { + LOG.error("Aborting after {} retries.", MAX_RETRIES); + throw exception; } } - LOG.debug("Successfully wrote {} entities", entities.size()); - entities.clear(); } + LOG.debug("Successfully wrote {} mutations", mutations.size()); + mutations.clear(); + } - @Override - public void populateDisplayData(Builder builder) { - super.populateDisplayData(builder); - builder - .addIfNotNull(DisplayData.item("projectId", projectId) - .withLabel("Output Project")); - } + @Override + public void populateDisplayData(Builder builder) { + super.populateDisplayData(builder); + builder + .addIfNotNull(DisplayData.item("projectId", projectId) + .withLabel("Output Project")); } + } - /** - * Returns true if a Datastore key is complete. A key is complete if its last element - * has either an id or a name. - */ - static boolean isValidKey(Key key) { - List elementList = key.getPathList(); - if (elementList.isEmpty()) { - return false; - } - PathElement lastElement = elementList.get(elementList.size() - 1); - return (lastElement.getId() != 0 || !lastElement.getName().isEmpty()); + /** + * Returns true if a Datastore key is complete. A key is complete if its last element + * has either an id or a name. + */ + static boolean isValidKey(Key key) { + List elementList = key.getPathList(); + if (elementList.isEmpty()) { + return false; + } + PathElement lastElement = elementList.get(elementList.size() - 1); + return (lastElement.getId() != 0 || !lastElement.getName().isEmpty()); + } + + /** + * A function that constructs an upsert {@link Mutation} from an {@link Entity}. + */ + @VisibleForTesting + static class UpsertFn extends SimpleFunction { + @Override + public Mutation apply(Entity entity) { + // Verify that the entity to write has a complete key. + checkArgument(isValidKey(entity.getKey()), + "Entities to be written to the Datastore must have complete keys:\n%s", entity); + + return makeUpsert(entity).build(); + } + + @Override + public void populateDisplayData(Builder builder) { + builder.add(DisplayData.item("upsertFn", this.getClass()) + .withLabel("Create Upsert Mutation")); + } + } + + /** + * A function that constructs a delete {@link Mutation} from an {@link Entity}. + */ + @VisibleForTesting + static class DeleteEntityFn extends SimpleFunction { + @Override + public Mutation apply(Entity entity) { + // Verify that the entity to delete has a complete key. + checkArgument(isValidKey(entity.getKey()), + "Entities to be deleted from the Datastore must have complete keys:\n%s", entity); + + return makeDelete(entity.getKey()).build(); + } + + @Override + public void populateDisplayData(Builder builder) { + builder.add(DisplayData.item("deleteEntityFn", this.getClass()) + .withLabel("Create Delete Mutation")); + } + } + + /** + * A function that constructs a delete {@link Mutation} from a {@link Key}. + */ + @VisibleForTesting + static class DeleteKeyFn extends SimpleFunction { + @Override + public Mutation apply(Key key) { + // Verify that the entity to delete has a complete key. + checkArgument(isValidKey(key), + "Keys to be deleted from the Datastore must be complete:\n%s", key); + + return makeDelete(key).build(); + } + + @Override + public void populateDisplayData(Builder builder) { + builder.add(DisplayData.item("deleteKeyFn", this.getClass()) + .withLabel("Create Delete Mutation")); } } diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/V1Beta3Test.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/V1Beta3Test.java index 8fa34dae119fe..b0c6c185f812e 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/V1Beta3Test.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/V1Beta3Test.java @@ -22,9 +22,11 @@ import static org.apache.beam.sdk.io.gcp.datastore.V1Beta3.Read.QUERY_BATCH_LIMIT; import static org.apache.beam.sdk.io.gcp.datastore.V1Beta3.Read.getEstimatedSizeBytes; import static org.apache.beam.sdk.io.gcp.datastore.V1Beta3.Read.makeRequest; +import static org.apache.beam.sdk.io.gcp.datastore.V1Beta3.isValidKey; import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasDisplayItem; import static com.google.datastore.v1beta3.PropertyFilter.Operator.EQUAL; import static com.google.datastore.v1beta3.PropertyOrder.Direction.DESCENDING; +import static com.google.datastore.v1beta3.client.DatastoreHelper.makeDelete; import static com.google.datastore.v1beta3.client.DatastoreHelper.makeFilter; import static com.google.datastore.v1beta3.client.DatastoreHelper.makeKey; import static com.google.datastore.v1beta3.client.DatastoreHelper.makeOrder; @@ -45,12 +47,17 @@ import static org.mockito.Mockito.verifyZeroInteractions; import static org.mockito.Mockito.when; +import org.apache.beam.sdk.io.gcp.datastore.V1Beta3.DatastoreWriterFn; +import org.apache.beam.sdk.io.gcp.datastore.V1Beta3.DeleteEntity; +import org.apache.beam.sdk.io.gcp.datastore.V1Beta3.DeleteEntityFn; +import org.apache.beam.sdk.io.gcp.datastore.V1Beta3.DeleteKey; +import org.apache.beam.sdk.io.gcp.datastore.V1Beta3.DeleteKeyFn; import org.apache.beam.sdk.io.gcp.datastore.V1Beta3.Read.ReadFn; import org.apache.beam.sdk.io.gcp.datastore.V1Beta3.Read.SplitQueryFn; import org.apache.beam.sdk.io.gcp.datastore.V1Beta3.Read.V1Beta3Options; +import org.apache.beam.sdk.io.gcp.datastore.V1Beta3.UpsertFn; import org.apache.beam.sdk.io.gcp.datastore.V1Beta3.V1Beta3DatastoreFactory; import org.apache.beam.sdk.io.gcp.datastore.V1Beta3.Write; -import org.apache.beam.sdk.io.gcp.datastore.V1Beta3.Write.DatastoreWriterFn; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.testing.RunnableOnService; import org.apache.beam.sdk.transforms.DoFnTester; @@ -67,6 +74,7 @@ import com.google.datastore.v1beta3.Entity; import com.google.datastore.v1beta3.EntityResult; import com.google.datastore.v1beta3.Key; +import com.google.datastore.v1beta3.Mutation; import com.google.datastore.v1beta3.PartitionId; import com.google.datastore.v1beta3.Query; import com.google.datastore.v1beta3.QueryResultBatch; @@ -233,7 +241,7 @@ public void testWriteDoesNotAllowNullProject() throws Exception { @Test public void testWriteValidationFailsWithNoProject() throws Exception { - V1Beta3.Write write = DatastoreIO.v1beta3().write(); + Write write = DatastoreIO.v1beta3().write(); thrown.expect(NullPointerException.class); thrown.expectMessage("projectId"); @@ -242,24 +250,89 @@ public void testWriteValidationFailsWithNoProject() throws Exception { } @Test - public void testSinkValidationSucceedsWithProject() throws Exception { - V1Beta3.Write write = DatastoreIO.v1beta3().write().withProjectId(PROJECT_ID); + public void testWriteValidationSucceedsWithProject() throws Exception { + Write write = DatastoreIO.v1beta3().write().withProjectId(PROJECT_ID); write.validate(null); } @Test public void testWriteDisplayData() { - V1Beta3.Write write = DatastoreIO.v1beta3().write() - .withProjectId(PROJECT_ID); + Write write = DatastoreIO.v1beta3().write().withProjectId(PROJECT_ID); DisplayData displayData = DisplayData.from(write); assertThat(displayData, hasDisplayItem("projectId", PROJECT_ID)); } + @Test + public void testDeleteEntityDoesNotAllowNullProject() throws Exception { + thrown.expect(NullPointerException.class); + thrown.expectMessage("projectId"); + + DatastoreIO.v1beta3().deleteEntity().withProjectId(null); + } + + @Test + public void testDeleteEntityValidationFailsWithNoProject() throws Exception { + DeleteEntity deleteEntity = DatastoreIO.v1beta3().deleteEntity(); + + thrown.expect(NullPointerException.class); + thrown.expectMessage("projectId"); + + deleteEntity.validate(null); + } + + @Test + public void testDeleteEntityValidationSucceedsWithProject() throws Exception { + DeleteEntity deleteEntity = DatastoreIO.v1beta3().deleteEntity().withProjectId(PROJECT_ID); + deleteEntity.validate(null); + } + + @Test + public void testDeleteEntityDisplayData() { + DeleteEntity deleteEntity = DatastoreIO.v1beta3().deleteEntity().withProjectId(PROJECT_ID); + + DisplayData displayData = DisplayData.from(deleteEntity); + + assertThat(displayData, hasDisplayItem("projectId", PROJECT_ID)); + } + + @Test + public void testDeleteKeyDoesNotAllowNullProject() throws Exception { + thrown.expect(NullPointerException.class); + thrown.expectMessage("projectId"); + + DatastoreIO.v1beta3().deleteKey().withProjectId(null); + } + + @Test + public void testDeleteKeyValidationFailsWithNoProject() throws Exception { + DeleteKey deleteKey = DatastoreIO.v1beta3().deleteKey(); + + thrown.expect(NullPointerException.class); + thrown.expectMessage("projectId"); + + deleteKey.validate(null); + } + + @Test + public void testDeleteKeyValidationSucceedsWithProject() throws Exception { + DeleteKey deleteKey = DatastoreIO.v1beta3().deleteKey().withProjectId(PROJECT_ID); + deleteKey.validate(null); + } + + @Test + public void testDeleteKeyDisplayData() { + DeleteKey deleteKey = DatastoreIO.v1beta3().deleteKey().withProjectId(PROJECT_ID); + + DisplayData displayData = DisplayData.from(deleteKey); + + assertThat(displayData, hasDisplayItem("projectId", PROJECT_ID)); + } + @Test @Category(RunnableOnService.class) - public void testSinkPrimitiveDisplayData() { + public void testWritePrimitiveDisplayData() { DisplayDataEvaluator evaluator = DisplayDataEvaluator.create(); PTransform, ?> write = DatastoreIO.v1beta3().write().withProjectId("myProject"); @@ -267,6 +340,39 @@ public void testSinkPrimitiveDisplayData() { Set displayData = evaluator.displayDataForPrimitiveTransforms(write); assertThat("DatastoreIO write should include the project in its primitive display data", displayData, hasItem(hasDisplayItem("projectId"))); + assertThat("DatastoreIO write should include the upsertFn in its primitive display data", + displayData, hasItem(hasDisplayItem("upsertFn"))); + + } + + @Test + @Category(RunnableOnService.class) + public void testDeleteEntityPrimitiveDisplayData() { + DisplayDataEvaluator evaluator = DisplayDataEvaluator.create(); + PTransform, ?> write = + DatastoreIO.v1beta3().deleteEntity().withProjectId("myProject"); + + Set displayData = evaluator.displayDataForPrimitiveTransforms(write); + assertThat("DatastoreIO write should include the project in its primitive display data", + displayData, hasItem(hasDisplayItem("projectId"))); + assertThat("DatastoreIO write should include the deleteEntityFn in its primitive display data", + displayData, hasItem(hasDisplayItem("deleteEntityFn"))); + + } + + @Test + @Category(RunnableOnService.class) + public void testDeleteKeyPrimitiveDisplayData() { + DisplayDataEvaluator evaluator = DisplayDataEvaluator.create(); + PTransform, ?> write = + DatastoreIO.v1beta3().deleteKey().withProjectId("myProject"); + + Set displayData = evaluator.displayDataForPrimitiveTransforms(write); + assertThat("DatastoreIO write should include the project in its primitive display data", + displayData, hasItem(hasDisplayItem("projectId"))); + assertThat("DatastoreIO write should include the deleteKeyFn in its primitive display data", + displayData, hasItem(hasDisplayItem("deleteKeyFn"))); + } /** @@ -286,33 +392,33 @@ public void testHasNameOrId() { Key key; // Complete with name, no ancestor key = makeKey("bird", "finch").build(); - assertTrue(Write.isValidKey(key)); + assertTrue(isValidKey(key)); // Complete with id, no ancestor key = makeKey("bird", 123).build(); - assertTrue(Write.isValidKey(key)); + assertTrue(isValidKey(key)); // Incomplete, no ancestor key = makeKey("bird").build(); - assertFalse(Write.isValidKey(key)); + assertFalse(isValidKey(key)); // Complete with name and ancestor key = makeKey("bird", "owl").build(); key = makeKey(key, "bird", "horned").build(); - assertTrue(Write.isValidKey(key)); + assertTrue(isValidKey(key)); // Complete with id and ancestor key = makeKey("bird", "owl").build(); key = makeKey(key, "bird", 123).build(); - assertTrue(Write.isValidKey(key)); + assertTrue(isValidKey(key)); // Incomplete with ancestor key = makeKey("bird", "owl").build(); key = makeKey(key, "bird").build(); - assertFalse(Write.isValidKey(key)); + assertFalse(isValidKey(key)); key = makeKey().build(); - assertFalse(Write.isValidKey(key)); + assertFalse(isValidKey(key)); } /** @@ -322,14 +428,86 @@ public void testHasNameOrId() { public void testAddEntitiesWithIncompleteKeys() throws Exception { Key key = makeKey("bird").build(); Entity entity = Entity.newBuilder().setKey(key).build(); - DatastoreWriterFn writer = new DatastoreWriterFn(PROJECT_ID, mockDatastoreFactory); - DoFnTester doFnTester = DoFnTester.of(writer); - doFnTester.setCloningBehavior(CloningBehavior.DO_NOT_CLONE); + UpsertFn upsertFn = new UpsertFn(); thrown.expect(IllegalArgumentException.class); thrown.expectMessage("Entities to be written to the Datastore must have complete keys"); - doFnTester.processBundle(entity); + upsertFn.apply(entity); + } + + @Test + /** + * Test that entities with valid keys are transformed to upsert mutations. + */ + public void testAddEntities() throws Exception { + Key key = makeKey("bird", "finch").build(); + Entity entity = Entity.newBuilder().setKey(key).build(); + UpsertFn upsertFn = new UpsertFn(); + + Mutation exceptedMutation = makeUpsert(entity).build(); + assertEquals(upsertFn.apply(entity), exceptedMutation); + } + + /** + * Test that entities with incomplete keys cannot be deleted. + */ + @Test + public void testDeleteEntitiesWithIncompleteKeys() throws Exception { + Key key = makeKey("bird").build(); + Entity entity = Entity.newBuilder().setKey(key).build(); + DeleteEntityFn deleteEntityFn = new DeleteEntityFn(); + + thrown.expect(IllegalArgumentException.class); + thrown.expectMessage("Entities to be deleted from the Datastore must have complete keys"); + + deleteEntityFn.apply(entity); + } + + /** + * Test that entities with valid keys are transformed to delete mutations. + */ + @Test + public void testDeleteEntities() throws Exception { + Key key = makeKey("bird", "finch").build(); + Entity entity = Entity.newBuilder().setKey(key).build(); + DeleteEntityFn deleteEntityFn = new DeleteEntityFn(); + + Mutation exceptedMutation = makeDelete(entity.getKey()).build(); + assertEquals(deleteEntityFn.apply(entity), exceptedMutation); + } + + /** + * Test that incomplete keys cannot be deleted. + */ + @Test + public void testDeleteIncompleteKeys() throws Exception { + Key key = makeKey("bird").build(); + DeleteKeyFn deleteKeyFn = new DeleteKeyFn(); + + thrown.expect(IllegalArgumentException.class); + thrown.expectMessage("Keys to be deleted from the Datastore must be complete"); + + deleteKeyFn.apply(key); + } + + /** + * Test that valid keys are transformed to delete mutations. + */ + @Test + public void testDeleteKeys() throws Exception { + Key key = makeKey("bird", "finch").build(); + DeleteKeyFn deleteKeyFn = new DeleteKeyFn(); + + Mutation exceptedMutation = makeDelete(key).build(); + assertEquals(deleteKeyFn.apply(key), exceptedMutation); + } + + @Test + public void testDatastoreWriteFnDisplayData() { + DatastoreWriterFn datastoreWriter = new DatastoreWriterFn(PROJECT_ID); + DisplayData displayData = DisplayData.from(datastoreWriter); + assertThat(displayData, hasDisplayItem("projectId", PROJECT_ID)); } /** Tests {@link DatastoreWriterFn} with entities less than one batch. */ @@ -354,27 +532,26 @@ public void testDatatoreWriterFnWithBatchesExactMultiple() throws Exception { } // A helper method to test DatastoreWriterFn for various batch sizes. - private void datastoreWriterFnTest(int numEntities) throws Exception { + private void datastoreWriterFnTest(int numMutations) throws Exception { // Create the requested number of mutations. - List entities = new ArrayList<>(numEntities); - for (int i = 0; i < numEntities; ++i) { - entities.add(Entity.newBuilder().setKey(makeKey("key" + i, i + 1)).build()); + List mutations = new ArrayList<>(numMutations); + for (int i = 0; i < numMutations; ++i) { + mutations.add( + makeUpsert(Entity.newBuilder().setKey(makeKey("key" + i, i + 1)).build()).build()); } DatastoreWriterFn datastoreWriter = new DatastoreWriterFn(PROJECT_ID, mockDatastoreFactory); - DoFnTester doFnTester = DoFnTester.of(datastoreWriter); + DoFnTester doFnTester = DoFnTester.of(datastoreWriter); doFnTester.setCloningBehavior(CloningBehavior.DO_NOT_CLONE); - doFnTester.processBundle(entities); + doFnTester.processBundle(mutations); int start = 0; - while (start < numEntities) { - int end = Math.min(numEntities, start + DATASTORE_BATCH_UPDATE_LIMIT); + while (start < numMutations) { + int end = Math.min(numMutations, start + DATASTORE_BATCH_UPDATE_LIMIT); CommitRequest.Builder commitRequest = CommitRequest.newBuilder(); commitRequest.setMode(CommitRequest.Mode.NON_TRANSACTIONAL); - for (Entity entity: entities.subList(start, end)) { - commitRequest.addMutations(makeUpsert(entity)); - } - // Verify all the batch requests were made with the expected entities. + commitRequest.addAllMutations(mutations.subList(start, end)); + // Verify all the batch requests were made with the expected mutations. verify(mockDatastore, times(1)).commit(commitRequest.build()); start = end; } From c996c1eb0d66accec9b678d260169c02043999f3 Mon Sep 17 00:00:00 2001 From: Thomas Groh Date: Fri, 19 Aug 2016 10:24:00 -0700 Subject: [PATCH 043/346] Mark JAXBContext as Volatile in JAXBCoder This corrects the double-checked locking and lazy intialization --- .../build-tools/src/main/resources/beam/findbugs-filter.xml | 6 ------ .../src/main/java/org/apache/beam/sdk/coders/JAXBCoder.java | 2 +- 2 files changed, 1 insertion(+), 7 deletions(-) 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 d135cbb671d9b..cd50408fd395e 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 @@ -37,12 +37,6 @@ - - - - - - diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/JAXBCoder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/JAXBCoder.java index 2284aafe9add9..689f993a1cdeb 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/JAXBCoder.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/JAXBCoder.java @@ -46,7 +46,7 @@ public class JAXBCoder extends AtomicCoder { private final Class jaxbClass; - private transient JAXBContext jaxbContext; + private transient volatile JAXBContext jaxbContext; public Class getJAXBClass() { return jaxbClass; From 4ad78b23f2e07d294ae85cd9e9141a4592ed84f0 Mon Sep 17 00:00:00 2001 From: Luke Cwik Date: Thu, 18 Aug 2016 16:35:33 -0700 Subject: [PATCH 044/346] Modify example dependencies to only add runners as optional dependencies. Also support excluding runners by disabling the include-runners profile. Tested ITs BQ and WordCount pass on Dataflow WordCount passes on Flink and Spark --- examples/java/pom.xml | 129 ++++++++++++------ .../org/apache/beam/examples/WordCount.java | 3 +- examples/java8/pom.xml | 40 +++++- 3 files changed, 125 insertions(+), 47 deletions(-) diff --git a/examples/java/pom.xml b/examples/java/pom.xml index 6efbc54406d91..21d7a3a197cdb 100644 --- a/examples/java/pom.xml +++ b/examples/java/pom.xml @@ -38,6 +38,78 @@ 1.6.2 + + + + include-runners + + true + + + + org.apache.beam + beam-runners-direct-java + ${project.version} + runtime + true + + + + org.apache.beam + beam-runners-flink_2.10 + ${project.version} + runtime + true + + + + org.apache.beam + beam-runners-google-cloud-dataflow-java + ${project.version} + runtime + true + + + + org.apache.beam + beam-runners-spark + ${project.version} + runtime + true + + + + org.apache.spark + spark-core_2.10 + ${spark.version} + runtime + true + + + + org.apache.spark + spark-streaming_2.10 + ${spark.version} + runtime + true + + + + + @@ -269,53 +341,11 @@ slf4j-api - - org.apache.beam - beam-runners-direct-java - ${project.version} - runtime - true - - - - org.apache.beam - beam-runners-google-cloud-dataflow-java - ${project.version} - runtime - true - - - - org.apache.beam - beam-runners-flink_2.10 - ${project.version} - runtime - - - - org.apache.beam - beam-runners-spark - ${project.version} - - - - org.apache.spark - spark-core_2.10 - ${spark.version} - runtime - - - - org.apache.spark - spark-streaming_2.10 - ${spark.version} - runtime - - org.slf4j slf4j-jdk14 runtime + true + + + + org.apache.beam + beam-runners-direct-java + ${project.version} + test + + org.mockito mockito-all 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 72ad35f04e549..42d30bb5fa94a 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/WordCount.java +++ b/examples/java/src/main/java/org/apache/beam/examples/WordCount.java @@ -17,7 +17,6 @@ */ package org.apache.beam.examples; -import org.apache.beam.runners.spark.SparkRunner; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.io.TextIO; import org.apache.beam.sdk.options.Default; @@ -211,7 +210,7 @@ public String create(PipelineOptions options) { public static class InputFactory implements DefaultValueFactory { @Override public String create(PipelineOptions options) { - if (options.getRunner().isAssignableFrom(SparkRunner.class)) { + if (options.getRunner().getName().contains("SparkRunner")) { return Resources.getResource("LICENSE").getPath(); } else { return "gs://apache-beam-samples/apache/LICENSE"; diff --git a/examples/java8/pom.xml b/examples/java8/pom.xml index 609dcfac09fe8..f3f2c3c83f17c 100644 --- a/examples/java8/pom.xml +++ b/examples/java8/pom.xml @@ -35,8 +35,11 @@ jar - + + 1.6.2 + + include-runners - true + + true + org.apache.beam @@ -71,6 +76,14 @@ true + + org.apache.beam + beam-runners-google-cloud-dataflow-java + ${project.version} + runtime + true + + org.apache.beam beam-runners-spark @@ -78,6 +91,22 @@ runtime true + + + org.apache.spark + spark-core_2.10 + ${spark.version} + runtime + true + + + + org.apache.spark + spark-streaming_2.10 + ${spark.version} + runtime + true + @@ -164,6 +193,13 @@ slf4j-api + + org.slf4j + slf4j-jdk14 + runtime + true + + org.apache.avro avro From 5c1b9f13bc406883160826b9dc40de8061a60284 Mon Sep 17 00:00:00 2001 From: Vikas Kedigehalli Date: Fri, 19 Aug 2016 16:18:37 -0700 Subject: [PATCH 045/346] DatastoreIO v1beta3 to v1 --- examples/java/pom.xml | 4 +- .../beam/examples/complete/AutoComplete.java | 12 +- .../examples/cookbook/DatastoreWordCount.java | 24 +-- pom.xml | 8 +- runners/google-cloud-dataflow-java/pom.xml | 2 +- sdks/java/io/google-cloud-platform/pom.xml | 4 +- .../sdk/io/gcp/datastore/DatastoreIO.java | 8 +- .../{V1Beta3.java => DatastoreV1.java} | 175 +++++++++--------- ...{V1Beta3Test.java => DatastoreV1Test.java} | 162 ++++++++-------- .../{V1Beta3ReadIT.java => V1ReadIT.java} | 42 ++--- ...ta3TestOptions.java => V1TestOptions.java} | 4 +- .../{V1Beta3TestUtil.java => V1TestUtil.java} | 74 ++++---- .../{V1Beta3WriteIT.java => V1WriteIT.java} | 22 +-- 13 files changed, 271 insertions(+), 270 deletions(-) rename sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/datastore/{V1Beta3.java => DatastoreV1.java} (85%) rename sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/{V1Beta3Test.java => DatastoreV1Test.java} (82%) rename sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/{V1Beta3ReadIT.java => V1ReadIT.java} (69%) rename sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/{V1Beta3TestOptions.java => V1TestOptions.java} (92%) rename sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/{V1Beta3TestUtil.java => V1TestUtil.java} (82%) rename sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/{V1Beta3WriteIT.java => V1WriteIT.java} (76%) diff --git a/examples/java/pom.xml b/examples/java/pom.xml index 21d7a3a197cdb..096bc4e32fefd 100644 --- a/examples/java/pom.xml +++ b/examples/java/pom.xml @@ -323,12 +323,12 @@ com.google.cloud.datastore - datastore-v1beta3-proto-client + datastore-v1-proto-client com.google.cloud.datastore - datastore-v1beta3-protos + datastore-v1-protos 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 baae100bcf38a..120c64fae54ac 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 @@ -18,8 +18,8 @@ package org.apache.beam.examples.complete; import static com.google.common.base.Preconditions.checkArgument; -import static com.google.datastore.v1beta3.client.DatastoreHelper.makeKey; -import static com.google.datastore.v1beta3.client.DatastoreHelper.makeValue; +import static com.google.datastore.v1.client.DatastoreHelper.makeKey; +import static com.google.datastore.v1.client.DatastoreHelper.makeValue; import org.apache.beam.examples.common.ExampleBigQueryTableOptions; import org.apache.beam.examples.common.ExampleOptions; @@ -59,9 +59,9 @@ import com.google.api.services.bigquery.model.TableRow; import com.google.api.services.bigquery.model.TableSchema; import com.google.common.base.MoreObjects; -import com.google.datastore.v1beta3.Entity; -import com.google.datastore.v1beta3.Key; -import com.google.datastore.v1beta3.Value; +import com.google.datastore.v1.Entity; +import com.google.datastore.v1.Key; +import com.google.datastore.v1.Value; import org.joda.time.Duration; @@ -488,7 +488,7 @@ public static void main(String[] args) throws IOException { toWrite .apply("FormatForDatastore", ParDo.of(new FormatForDatastore(options.getKind(), options.getDatastoreAncestorKey()))) - .apply(DatastoreIO.v1beta3().write().withProjectId(MoreObjects.firstNonNull( + .apply(DatastoreIO.v1().write().withProjectId(MoreObjects.firstNonNull( options.getOutputProject(), options.getProject()))); } if (options.getOutputToBigQuery()) { diff --git a/examples/java/src/main/java/org/apache/beam/examples/cookbook/DatastoreWordCount.java b/examples/java/src/main/java/org/apache/beam/examples/cookbook/DatastoreWordCount.java index 21220b8fac8f9..215e2ffc78981 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/cookbook/DatastoreWordCount.java +++ b/examples/java/src/main/java/org/apache/beam/examples/cookbook/DatastoreWordCount.java @@ -17,16 +17,16 @@ */ package org.apache.beam.examples.cookbook; -import static com.google.datastore.v1beta3.client.DatastoreHelper.getString; -import static com.google.datastore.v1beta3.client.DatastoreHelper.makeFilter; -import static com.google.datastore.v1beta3.client.DatastoreHelper.makeKey; -import static com.google.datastore.v1beta3.client.DatastoreHelper.makeValue; +import static com.google.datastore.v1.client.DatastoreHelper.getString; +import static com.google.datastore.v1.client.DatastoreHelper.makeFilter; +import static com.google.datastore.v1.client.DatastoreHelper.makeKey; +import static com.google.datastore.v1.client.DatastoreHelper.makeValue; import org.apache.beam.examples.WordCount; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.io.TextIO; import org.apache.beam.sdk.io.gcp.datastore.DatastoreIO; -import org.apache.beam.sdk.io.gcp.datastore.V1Beta3; +import org.apache.beam.sdk.io.gcp.datastore.DatastoreV1; import org.apache.beam.sdk.options.Default; import org.apache.beam.sdk.options.Description; import org.apache.beam.sdk.options.PipelineOptions; @@ -36,11 +36,11 @@ import org.apache.beam.sdk.transforms.MapElements; import org.apache.beam.sdk.transforms.ParDo; -import com.google.datastore.v1beta3.Entity; -import com.google.datastore.v1beta3.Key; -import com.google.datastore.v1beta3.PropertyFilter; -import com.google.datastore.v1beta3.Query; -import com.google.datastore.v1beta3.Value; +import com.google.datastore.v1.Entity; +import com.google.datastore.v1.Key; +import com.google.datastore.v1.PropertyFilter; +import com.google.datastore.v1.Query; +import com.google.datastore.v1.Value; import java.util.Map; import java.util.UUID; @@ -194,7 +194,7 @@ public static void writeDataToDatastore(Options options) { Pipeline p = Pipeline.create(options); p.apply("ReadLines", TextIO.Read.from(options.getInput())) .apply(ParDo.of(new CreateEntityFn(options.getNamespace(), options.getKind()))) - .apply(DatastoreIO.v1beta3().write().withProjectId(options.getProject())); + .apply(DatastoreIO.v1().write().withProjectId(options.getProject())); p.run(); } @@ -225,7 +225,7 @@ public static void readDataFromDatastore(Options options) { Query query = makeAncestorKindQuery(options); // For Datastore sources, the read namespace can be set on the entire query. - V1Beta3.Read read = DatastoreIO.v1beta3().read() + DatastoreV1.Read read = DatastoreIO.v1().read() .withProjectId(options.getProject()) .withQuery(query) .withNamespace(options.getNamespace()); diff --git a/pom.xml b/pom.xml index 58adbe7a86cc9..f9e04795417db 100644 --- a/pom.xml +++ b/pom.xml @@ -107,8 +107,8 @@ v2-rev8-1.22.0 v1b3-rev36-1.22.0 0.5.160222 - 1.0.0-beta.2 - 1.0.0-beta + 1.1.0 + 1.0.1 1.0-rc2 1.1 1.22.0 @@ -450,13 +450,13 @@ com.google.cloud.datastore - datastore-v1beta3-proto-client + datastore-v1-proto-client ${datastore.client.version} com.google.cloud.datastore - datastore-v1beta3-protos + datastore-v1-protos ${datastore.proto.version} diff --git a/runners/google-cloud-dataflow-java/pom.xml b/runners/google-cloud-dataflow-java/pom.xml index 00b5a9ba67c25..0044823068c04 100644 --- a/runners/google-cloud-dataflow-java/pom.xml +++ b/runners/google-cloud-dataflow-java/pom.xml @@ -409,7 +409,7 @@ com.google.cloud.datastore - datastore-v1beta3-protos + datastore-v1-protos test diff --git a/sdks/java/io/google-cloud-platform/pom.xml b/sdks/java/io/google-cloud-platform/pom.xml index 1596a66b82be4..8075335bd7126 100644 --- a/sdks/java/io/google-cloud-platform/pom.xml +++ b/sdks/java/io/google-cloud-platform/pom.xml @@ -115,12 +115,12 @@ com.google.cloud.datastore - datastore-v1beta3-proto-client + datastore-v1-proto-client com.google.cloud.datastore - datastore-v1beta3-protos + datastore-v1-protos diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/datastore/DatastoreIO.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/datastore/DatastoreIO.java index bde0aba8a1038..5abf0153ac10a 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/datastore/DatastoreIO.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/datastore/DatastoreIO.java @@ -24,7 +24,7 @@ * Google Cloud Datastore over different * versions of the Datastore Client libraries. * - *

    To use the v1beta3 version see {@link V1Beta3}. + *

    To use the v1 version see {@link DatastoreV1}. */ @Experimental(Experimental.Kind.SOURCE_SINK) public class DatastoreIO { @@ -32,10 +32,10 @@ public class DatastoreIO { private DatastoreIO() {} /** - * Returns a {@link V1Beta3} that provides an API for accessing Datastore through v1beta3 version + * Returns a {@link DatastoreV1} that provides an API for accessing Datastore through v1 version * of Datastore Client library. */ - public static V1Beta3 v1beta3() { - return new V1Beta3(); + public static DatastoreV1 v1() { + return new DatastoreV1(); } } diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/datastore/V1Beta3.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/datastore/DatastoreV1.java similarity index 85% rename from sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/datastore/V1Beta3.java rename to sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/datastore/DatastoreV1.java index 8503b668e25af..852595aa79ab1 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/datastore/V1Beta3.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/datastore/DatastoreV1.java @@ -21,14 +21,14 @@ import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.base.Preconditions.checkNotNull; import static com.google.common.base.Verify.verify; -import static com.google.datastore.v1beta3.PropertyFilter.Operator.EQUAL; -import static com.google.datastore.v1beta3.PropertyOrder.Direction.DESCENDING; -import static com.google.datastore.v1beta3.QueryResultBatch.MoreResultsType.NOT_FINISHED; -import static com.google.datastore.v1beta3.client.DatastoreHelper.makeDelete; -import static com.google.datastore.v1beta3.client.DatastoreHelper.makeFilter; -import static com.google.datastore.v1beta3.client.DatastoreHelper.makeOrder; -import static com.google.datastore.v1beta3.client.DatastoreHelper.makeUpsert; -import static com.google.datastore.v1beta3.client.DatastoreHelper.makeValue; +import static com.google.datastore.v1.PropertyFilter.Operator.EQUAL; +import static com.google.datastore.v1.PropertyOrder.Direction.DESCENDING; +import static com.google.datastore.v1.QueryResultBatch.MoreResultsType.NOT_FINISHED; +import static com.google.datastore.v1.client.DatastoreHelper.makeDelete; +import static com.google.datastore.v1.client.DatastoreHelper.makeFilter; +import static com.google.datastore.v1.client.DatastoreHelper.makeOrder; +import static com.google.datastore.v1.client.DatastoreHelper.makeUpsert; +import static com.google.datastore.v1.client.DatastoreHelper.makeValue; import org.apache.beam.sdk.annotations.Experimental; import org.apache.beam.sdk.options.GcpOptions; @@ -58,23 +58,23 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.base.MoreObjects; import com.google.common.collect.ImmutableList; -import com.google.datastore.v1beta3.CommitRequest; -import com.google.datastore.v1beta3.Entity; -import com.google.datastore.v1beta3.EntityResult; -import com.google.datastore.v1beta3.Key; -import com.google.datastore.v1beta3.Key.PathElement; -import com.google.datastore.v1beta3.Mutation; -import com.google.datastore.v1beta3.PartitionId; -import com.google.datastore.v1beta3.Query; -import com.google.datastore.v1beta3.QueryResultBatch; -import com.google.datastore.v1beta3.RunQueryRequest; -import com.google.datastore.v1beta3.RunQueryResponse; -import com.google.datastore.v1beta3.client.Datastore; -import com.google.datastore.v1beta3.client.DatastoreException; -import com.google.datastore.v1beta3.client.DatastoreFactory; -import com.google.datastore.v1beta3.client.DatastoreHelper; -import com.google.datastore.v1beta3.client.DatastoreOptions; -import com.google.datastore.v1beta3.client.QuerySplitter; +import com.google.datastore.v1.CommitRequest; +import com.google.datastore.v1.Entity; +import com.google.datastore.v1.EntityResult; +import com.google.datastore.v1.Key; +import com.google.datastore.v1.Key.PathElement; +import com.google.datastore.v1.Mutation; +import com.google.datastore.v1.PartitionId; +import com.google.datastore.v1.Query; +import com.google.datastore.v1.QueryResultBatch; +import com.google.datastore.v1.RunQueryRequest; +import com.google.datastore.v1.RunQueryResponse; +import com.google.datastore.v1.client.Datastore; +import com.google.datastore.v1.client.DatastoreException; +import com.google.datastore.v1.client.DatastoreFactory; +import com.google.datastore.v1.client.DatastoreHelper; +import com.google.datastore.v1.client.DatastoreOptions; +import com.google.datastore.v1.client.QuerySplitter; import com.google.protobuf.Int32Value; import org.slf4j.Logger; @@ -88,21 +88,21 @@ import javax.annotation.Nullable; /** - *

    {@link V1Beta3} provides an API to Read, Write and Delete {@link PCollection PCollections} of - * Google Cloud Datastore version v1beta3 + *

    {@link DatastoreV1} provides an API to Read, Write and Delete {@link PCollection PCollections} + * of Google Cloud Datastore version v1 * {@link Entity} objects. * - *

    This API currently requires an authentication workaround. To use {@link V1Beta3}, users + *

    This API currently requires an authentication workaround. To use {@link DatastoreV1}, users * must use the {@code gcloud} command line tool to get credentials for Datastore: *

      * $ gcloud auth login
      * 
    * - *

    To read a {@link PCollection} from a query to Datastore, use {@link V1Beta3#read} and - * its methods {@link V1Beta3.Read#withProjectId} and {@link V1Beta3.Read#withQuery} to + *

    To read a {@link PCollection} from a query to Datastore, use {@link DatastoreV1#read} and + * its methods {@link DatastoreV1.Read#withProjectId} and {@link DatastoreV1.Read#withQuery} to * specify the project to query and the query to read from. You can optionally provide a namespace - * to query within using {@link V1Beta3.Read#withNamespace}. You could also optionally specify - * how many splits you want for the query using {@link V1Beta3.Read#withNumQuerySplits}. + * to query within using {@link DatastoreV1.Read#withNamespace}. You could also optionally specify + * how many splits you want for the query using {@link DatastoreV1.Read#withNumQuerySplits}. * *

    For example: * @@ -114,40 +114,40 @@ * * Pipeline p = Pipeline.create(options); * PCollection entities = p.apply( - * DatastoreIO.v1beta3().read() + * DatastoreIO.v1().read() * .withProjectId(projectId) * .withQuery(query)); * } * *

    Note: Normally, a Cloud Dataflow job will read from Cloud Datastore in parallel across * many workers. However, when the {@link Query} is configured with a limit using - * {@link com.google.datastore.v1beta3.Query.Builder#setLimit(Int32Value)}, then + * {@link com.google.datastore.v1.Query.Builder#setLimit(Int32Value)}, then * all returned results will be read by a single Dataflow worker in order to ensure correct data. * - *

    To write a {@link PCollection} to a Datastore, use {@link V1Beta3#write}, + *

    To write a {@link PCollection} to a Datastore, use {@link DatastoreV1#write}, * specifying the Cloud Datastore project to write to: * *

     {@code
      * PCollection entities = ...;
    - * entities.apply(DatastoreIO.v1beta3().write().withProjectId(projectId));
    + * entities.apply(DatastoreIO.v1().write().withProjectId(projectId));
      * p.run();
      * } 
    * *

    To delete a {@link PCollection} of {@link Entity Entities} from Datastore, use - * {@link V1Beta3#deleteEntity()}, specifying the Cloud Datastore project to write to: + * {@link DatastoreV1#deleteEntity()}, specifying the Cloud Datastore project to write to: * *

     {@code
      * PCollection entities = ...;
    - * entities.apply(DatastoreIO.v1beta3().deleteEntity().withProjectId(projectId));
    + * entities.apply(DatastoreIO.v1().deleteEntity().withProjectId(projectId));
      * p.run();
      * } 
    * *

    To delete entities associated with a {@link PCollection} of {@link Key Keys} from Datastore, - * use {@link V1Beta3#deleteKey}, specifying the Cloud Datastore project to write to: + * use {@link DatastoreV1#deleteKey}, specifying the Cloud Datastore project to write to: * *

     {@code
      * PCollection entities = ...;
    - * entities.apply(DatastoreIO.v1beta3().deleteKey().withProjectId(projectId));
    + * entities.apply(DatastoreIO.v1().deleteKey().withProjectId(projectId));
      * p.run();
      * } 
    * @@ -176,10 +176,10 @@ * @see org.apache.beam.sdk.runners.PipelineRunner */ @Experimental(Experimental.Kind.SOURCE_SINK) -public class V1Beta3 { +public class DatastoreV1 { // A package-private constructor to prevent direct instantiation from outside of this package - V1Beta3() {} + DatastoreV1() {} /** * Datastore has a limit of 500 mutations per batch operation, so we flush @@ -189,13 +189,13 @@ public class V1Beta3 { static final int DATASTORE_BATCH_UPDATE_LIMIT = 500; /** - * Returns an empty {@link V1Beta3.Read} builder. Configure the source {@code projectId}, + * Returns an empty {@link DatastoreV1.Read} builder. Configure the source {@code projectId}, * {@code query}, and optionally {@code namespace} and {@code numQuerySplits} using - * {@link V1Beta3.Read#withProjectId}, {@link V1Beta3.Read#withQuery}, - * {@link V1Beta3.Read#withNamespace}, {@link V1Beta3.Read#withNumQuerySplits}. + * {@link DatastoreV1.Read#withProjectId}, {@link DatastoreV1.Read#withQuery}, + * {@link DatastoreV1.Read#withNamespace}, {@link DatastoreV1.Read#withNumQuerySplits}. */ - public V1Beta3.Read read() { - return new V1Beta3.Read(null, null, null, 0); + public DatastoreV1.Read read() { + return new DatastoreV1.Read(null, null, null, 0); } /** @@ -329,37 +329,38 @@ private Read(@Nullable String projectId, @Nullable Query query, @Nullable String } /** - * Returns a new {@link V1Beta3.Read} that reads from the Datastore for the specified project. + * Returns a new {@link DatastoreV1.Read} that reads from the Datastore for the specified + * project. */ - public V1Beta3.Read withProjectId(String projectId) { + public DatastoreV1.Read withProjectId(String projectId) { checkNotNull(projectId, "projectId"); - return new V1Beta3.Read(projectId, query, namespace, numQuerySplits); + return new DatastoreV1.Read(projectId, query, namespace, numQuerySplits); } /** - * Returns a new {@link V1Beta3.Read} that reads the results of the specified query. + * Returns a new {@link DatastoreV1.Read} that reads the results of the specified query. * *

    Note: Normally, {@code DatastoreIO} will read from Cloud Datastore in parallel * across many workers. However, when the {@link Query} is configured with a limit using * {@link Query.Builder#setLimit}, then all results will be read by a single worker in order * to ensure correct results. */ - public V1Beta3.Read withQuery(Query query) { + public DatastoreV1.Read withQuery(Query query) { checkNotNull(query, "query"); checkArgument(!query.hasLimit() || query.getLimit().getValue() > 0, "Invalid query limit %s: must be positive", query.getLimit().getValue()); - return new V1Beta3.Read(projectId, query, namespace, numQuerySplits); + return new DatastoreV1.Read(projectId, query, namespace, numQuerySplits); } /** - * Returns a new {@link V1Beta3.Read} that reads from the given namespace. + * Returns a new {@link DatastoreV1.Read} that reads from the given namespace. */ - public V1Beta3.Read withNamespace(String namespace) { - return new V1Beta3.Read(projectId, query, namespace, numQuerySplits); + public DatastoreV1.Read withNamespace(String namespace) { + return new DatastoreV1.Read(projectId, query, namespace, numQuerySplits); } /** - * Returns a new {@link V1Beta3.Read} that reads by splitting the given {@code query} into + * Returns a new {@link DatastoreV1.Read} that reads by splitting the given {@code query} into * {@code numQuerySplits}. * *

    The semantics for the query splitting is defined below: @@ -374,8 +375,8 @@ public V1Beta3.Read withNamespace(String namespace) { * splits. In such cases we just use whatever the Datastore returns. * */ - public V1Beta3.Read withNumQuerySplits(int numQuerySplits) { - return new V1Beta3.Read(projectId, query, namespace, + public DatastoreV1.Read withNumQuerySplits(int numQuerySplits) { + return new DatastoreV1.Read(projectId, query, namespace, Math.min(Math.max(numQuerySplits, 0), NUM_QUERY_SPLITS_MAX)); } @@ -400,7 +401,7 @@ public String getNamespace() { */ @Override public PCollection apply(PBegin input) { - V1Beta3Options v1Beta3Options = V1Beta3Options.from(getProjectId(), getQuery(), + V1Options v1Options = V1Options.from(getProjectId(), getQuery(), getNamespace()); /* @@ -422,7 +423,7 @@ public PCollection apply(PBegin input) { */ PCollection> queries = input .apply(Create.of(query)) - .apply(ParDo.of(new SplitQueryFn(v1Beta3Options, numQuerySplits))); + .apply(ParDo.of(new SplitQueryFn(v1Options, numQuerySplits))); PCollection shardedQueries = queries .apply(GroupByKey.create()) @@ -430,7 +431,7 @@ public PCollection apply(PBegin input) { .apply(Flatten.iterables()); PCollection entities = shardedQueries - .apply(ParDo.of(new ReadFn(v1Beta3Options))); + .apply(ParDo.of(new ReadFn(v1Options))); return entities; } @@ -463,23 +464,23 @@ public String toString() { } /** - * A class for v1beta3 Datastore related options. + * A class for v1 Datastore related options. */ @VisibleForTesting - static class V1Beta3Options implements Serializable { + static class V1Options implements Serializable { private final Query query; private final String projectId; @Nullable private final String namespace; - private V1Beta3Options(String projectId, Query query, @Nullable String namespace) { + private V1Options(String projectId, Query query, @Nullable String namespace) { this.projectId = checkNotNull(projectId, "projectId"); this.query = checkNotNull(query, "query"); this.namespace = namespace; } - public static V1Beta3Options from(String projectId, Query query, @Nullable String namespace) { - return new V1Beta3Options(projectId, query, namespace); + public static V1Options from(String projectId, Query query, @Nullable String namespace) { + return new V1Options(projectId, query, namespace); } public Query getQuery() { @@ -502,23 +503,23 @@ public String getNamespace() { */ @VisibleForTesting static class SplitQueryFn extends DoFn> { - private final V1Beta3Options options; + private final V1Options options; // number of splits to make for a given query private final int numSplits; - private final V1Beta3DatastoreFactory datastoreFactory; + private final V1DatastoreFactory datastoreFactory; // Datastore client private transient Datastore datastore; // Query splitter private transient QuerySplitter querySplitter; - public SplitQueryFn(V1Beta3Options options, int numSplits) { - this(options, numSplits, new V1Beta3DatastoreFactory()); + public SplitQueryFn(V1Options options, int numSplits) { + this(options, numSplits, new V1DatastoreFactory()); } @VisibleForTesting - SplitQueryFn(V1Beta3Options options, int numSplits, - V1Beta3DatastoreFactory datastoreFactory) { + SplitQueryFn(V1Options options, int numSplits, + V1DatastoreFactory datastoreFactory) { this.options = options; this.numSplits = numSplits; this.datastoreFactory = datastoreFactory; @@ -582,17 +583,17 @@ public void populateDisplayData(Builder builder) { */ @VisibleForTesting static class ReadFn extends DoFn { - private final V1Beta3Options options; - private final V1Beta3DatastoreFactory datastoreFactory; + private final V1Options options; + private final V1DatastoreFactory datastoreFactory; // Datastore client private transient Datastore datastore; - public ReadFn(V1Beta3Options options) { - this(options, new V1Beta3DatastoreFactory()); + public ReadFn(V1Options options) { + this(options, new V1DatastoreFactory()); } @VisibleForTesting - ReadFn(V1Beta3Options options, V1Beta3DatastoreFactory datastoreFactory) { + ReadFn(V1Options options, V1DatastoreFactory datastoreFactory) { this.options = options; this.datastoreFactory = datastoreFactory; } @@ -656,8 +657,8 @@ public void processElement(ProcessContext context) throws Exception { } /** - * Returns an empty {@link V1Beta3.Write} builder. Configure the destination - * {@code projectId} using {@link V1Beta3.Write#withProjectId}. + * Returns an empty {@link DatastoreV1.Write} builder. Configure the destination + * {@code projectId} using {@link DatastoreV1.Write#withProjectId}. */ public Write write() { return new Write(null); @@ -812,14 +813,14 @@ public String getProjectId() { /** * {@link DoFn} that writes {@link Mutation}s to Cloud Datastore. Mutations are written in - * batches, where the maximum batch size is {@link V1Beta3#DATASTORE_BATCH_UPDATE_LIMIT}. + * batches, where the maximum batch size is {@link DatastoreV1#DATASTORE_BATCH_UPDATE_LIMIT}. * *

    See * Datastore: Entities, Properties, and Keys for information about entity keys and mutations. * *

    Commits are non-transactional. If a commit fails because of a conflict over an entity - * group, the commit will be retried (up to {@link V1Beta3#DATASTORE_BATCH_UPDATE_LIMIT} + * group, the commit will be retried (up to {@link DatastoreV1#DATASTORE_BATCH_UPDATE_LIMIT} * times). This means that the mutation operation should be idempotent. Thus, the writer should * only be used for {code upsert} and {@code delete} mutation operations, as these are the only * two Cloud Datastore mutations that are idempotent. @@ -829,7 +830,7 @@ static class DatastoreWriterFn extends DoFn { private static final Logger LOG = LoggerFactory.getLogger(DatastoreWriterFn.class); private final String projectId; private transient Datastore datastore; - private final V1Beta3DatastoreFactory datastoreFactory; + private final V1DatastoreFactory datastoreFactory; // Current batch of mutations to be written. private final List mutations = new ArrayList<>(); /** @@ -844,11 +845,11 @@ static class DatastoreWriterFn extends DoFn { private static final int INITIAL_BACKOFF_MILLIS = 5000; DatastoreWriterFn(String projectId) { - this(projectId, new V1Beta3DatastoreFactory()); + this(projectId, new V1DatastoreFactory()); } @VisibleForTesting - DatastoreWriterFn(String projectId, V1Beta3DatastoreFactory datastoreFactory) { + DatastoreWriterFn(String projectId, V1DatastoreFactory datastoreFactory) { this.projectId = checkNotNull(projectId, "projectId"); this.datastoreFactory = datastoreFactory; } @@ -861,7 +862,7 @@ public void startBundle(Context c) { @ProcessElement public void processElement(ProcessContext c) throws Exception { mutations.add(c.element()); - if (mutations.size() >= V1Beta3.DATASTORE_BATCH_UPDATE_LIMIT) { + if (mutations.size() >= DatastoreV1.DATASTORE_BATCH_UPDATE_LIMIT) { flushBatch(); } } @@ -1006,7 +1007,7 @@ public void populateDisplayData(Builder builder) { * wrapping them under this class, which implements {@link Serializable}. */ @VisibleForTesting - static class V1Beta3DatastoreFactory implements Serializable { + static class V1DatastoreFactory implements Serializable { /** Builds a Datastore client for the given pipeline options and project. */ public Datastore getDatastore(PipelineOptions pipelineOptions, String projectId) { diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/V1Beta3Test.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/DatastoreV1Test.java similarity index 82% rename from sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/V1Beta3Test.java rename to sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/DatastoreV1Test.java index b0c6c185f812e..31b5da43aa1ea 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/V1Beta3Test.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/DatastoreV1Test.java @@ -17,21 +17,21 @@ */ package org.apache.beam.sdk.io.gcp.datastore; -import static org.apache.beam.sdk.io.gcp.datastore.V1Beta3.DATASTORE_BATCH_UPDATE_LIMIT; -import static org.apache.beam.sdk.io.gcp.datastore.V1Beta3.Read.DEFAULT_BUNDLE_SIZE_BYTES; -import static org.apache.beam.sdk.io.gcp.datastore.V1Beta3.Read.QUERY_BATCH_LIMIT; -import static org.apache.beam.sdk.io.gcp.datastore.V1Beta3.Read.getEstimatedSizeBytes; -import static org.apache.beam.sdk.io.gcp.datastore.V1Beta3.Read.makeRequest; -import static org.apache.beam.sdk.io.gcp.datastore.V1Beta3.isValidKey; +import static org.apache.beam.sdk.io.gcp.datastore.DatastoreV1.DATASTORE_BATCH_UPDATE_LIMIT; +import static org.apache.beam.sdk.io.gcp.datastore.DatastoreV1.Read.DEFAULT_BUNDLE_SIZE_BYTES; +import static org.apache.beam.sdk.io.gcp.datastore.DatastoreV1.Read.QUERY_BATCH_LIMIT; +import static org.apache.beam.sdk.io.gcp.datastore.DatastoreV1.Read.getEstimatedSizeBytes; +import static org.apache.beam.sdk.io.gcp.datastore.DatastoreV1.Read.makeRequest; +import static org.apache.beam.sdk.io.gcp.datastore.DatastoreV1.isValidKey; import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasDisplayItem; -import static com.google.datastore.v1beta3.PropertyFilter.Operator.EQUAL; -import static com.google.datastore.v1beta3.PropertyOrder.Direction.DESCENDING; -import static com.google.datastore.v1beta3.client.DatastoreHelper.makeDelete; -import static com.google.datastore.v1beta3.client.DatastoreHelper.makeFilter; -import static com.google.datastore.v1beta3.client.DatastoreHelper.makeKey; -import static com.google.datastore.v1beta3.client.DatastoreHelper.makeOrder; -import static com.google.datastore.v1beta3.client.DatastoreHelper.makeUpsert; -import static com.google.datastore.v1beta3.client.DatastoreHelper.makeValue; +import static com.google.datastore.v1.PropertyFilter.Operator.EQUAL; +import static com.google.datastore.v1.PropertyOrder.Direction.DESCENDING; +import static com.google.datastore.v1.client.DatastoreHelper.makeDelete; +import static com.google.datastore.v1.client.DatastoreHelper.makeFilter; +import static com.google.datastore.v1.client.DatastoreHelper.makeKey; +import static com.google.datastore.v1.client.DatastoreHelper.makeOrder; +import static com.google.datastore.v1.client.DatastoreHelper.makeUpsert; +import static com.google.datastore.v1.client.DatastoreHelper.makeValue; import static org.hamcrest.Matchers.greaterThanOrEqualTo; import static org.hamcrest.Matchers.hasItem; import static org.hamcrest.Matchers.lessThanOrEqualTo; @@ -47,17 +47,17 @@ import static org.mockito.Mockito.verifyZeroInteractions; import static org.mockito.Mockito.when; -import org.apache.beam.sdk.io.gcp.datastore.V1Beta3.DatastoreWriterFn; -import org.apache.beam.sdk.io.gcp.datastore.V1Beta3.DeleteEntity; -import org.apache.beam.sdk.io.gcp.datastore.V1Beta3.DeleteEntityFn; -import org.apache.beam.sdk.io.gcp.datastore.V1Beta3.DeleteKey; -import org.apache.beam.sdk.io.gcp.datastore.V1Beta3.DeleteKeyFn; -import org.apache.beam.sdk.io.gcp.datastore.V1Beta3.Read.ReadFn; -import org.apache.beam.sdk.io.gcp.datastore.V1Beta3.Read.SplitQueryFn; -import org.apache.beam.sdk.io.gcp.datastore.V1Beta3.Read.V1Beta3Options; -import org.apache.beam.sdk.io.gcp.datastore.V1Beta3.UpsertFn; -import org.apache.beam.sdk.io.gcp.datastore.V1Beta3.V1Beta3DatastoreFactory; -import org.apache.beam.sdk.io.gcp.datastore.V1Beta3.Write; +import org.apache.beam.sdk.io.gcp.datastore.DatastoreV1.DatastoreWriterFn; +import org.apache.beam.sdk.io.gcp.datastore.DatastoreV1.DeleteEntity; +import org.apache.beam.sdk.io.gcp.datastore.DatastoreV1.DeleteEntityFn; +import org.apache.beam.sdk.io.gcp.datastore.DatastoreV1.DeleteKey; +import org.apache.beam.sdk.io.gcp.datastore.DatastoreV1.DeleteKeyFn; +import org.apache.beam.sdk.io.gcp.datastore.DatastoreV1.Read.ReadFn; +import org.apache.beam.sdk.io.gcp.datastore.DatastoreV1.Read.SplitQueryFn; +import org.apache.beam.sdk.io.gcp.datastore.DatastoreV1.Read.V1Options; +import org.apache.beam.sdk.io.gcp.datastore.DatastoreV1.UpsertFn; +import org.apache.beam.sdk.io.gcp.datastore.DatastoreV1.V1DatastoreFactory; +import org.apache.beam.sdk.io.gcp.datastore.DatastoreV1.Write; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.testing.RunnableOnService; import org.apache.beam.sdk.transforms.DoFnTester; @@ -70,18 +70,18 @@ import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.POutput; -import com.google.datastore.v1beta3.CommitRequest; -import com.google.datastore.v1beta3.Entity; -import com.google.datastore.v1beta3.EntityResult; -import com.google.datastore.v1beta3.Key; -import com.google.datastore.v1beta3.Mutation; -import com.google.datastore.v1beta3.PartitionId; -import com.google.datastore.v1beta3.Query; -import com.google.datastore.v1beta3.QueryResultBatch; -import com.google.datastore.v1beta3.RunQueryRequest; -import com.google.datastore.v1beta3.RunQueryResponse; -import com.google.datastore.v1beta3.client.Datastore; -import com.google.datastore.v1beta3.client.QuerySplitter; +import com.google.datastore.v1.CommitRequest; +import com.google.datastore.v1.Entity; +import com.google.datastore.v1.EntityResult; +import com.google.datastore.v1.Key; +import com.google.datastore.v1.Mutation; +import com.google.datastore.v1.PartitionId; +import com.google.datastore.v1.Query; +import com.google.datastore.v1.QueryResultBatch; +import com.google.datastore.v1.RunQueryRequest; +import com.google.datastore.v1.RunQueryResponse; +import com.google.datastore.v1.client.Datastore; +import com.google.datastore.v1.client.QuerySplitter; import com.google.protobuf.Int32Value; import org.junit.Before; @@ -103,29 +103,29 @@ import java.util.Set; /** - * Tests for {@link V1Beta3}. + * Tests for {@link DatastoreV1}. */ @RunWith(JUnit4.class) -public class V1Beta3Test { +public class DatastoreV1Test { private static final String PROJECT_ID = "testProject"; private static final String NAMESPACE = "testNamespace"; private static final String KIND = "testKind"; private static final Query QUERY; - private static final V1Beta3Options v1Beta3Options; + private static final V1Options V_1_OPTIONS; static { Query.Builder q = Query.newBuilder(); q.addKindBuilder().setName(KIND); QUERY = q.build(); - v1Beta3Options = V1Beta3Options.from(PROJECT_ID, QUERY, NAMESPACE); + V_1_OPTIONS = V1Options.from(PROJECT_ID, QUERY, NAMESPACE); } - private V1Beta3.Read initialRead; + private DatastoreV1.Read initialRead; @Mock Datastore mockDatastore; @Mock QuerySplitter mockQuerySplitter; @Mock - V1Beta3DatastoreFactory mockDatastoreFactory; + V1DatastoreFactory mockDatastoreFactory; @Rule public final ExpectedException thrown = ExpectedException.none(); @@ -134,7 +134,7 @@ public class V1Beta3Test { public void setUp() { MockitoAnnotations.initMocks(this); - initialRead = DatastoreIO.v1beta3().read() + initialRead = DatastoreIO.v1().read() .withProjectId(PROJECT_ID).withQuery(QUERY).withNamespace(NAMESPACE); when(mockDatastoreFactory.getDatastore(any(PipelineOptions.class), any(String.class))) @@ -145,7 +145,7 @@ public void setUp() { @Test public void testBuildRead() throws Exception { - V1Beta3.Read read = DatastoreIO.v1beta3().read() + DatastoreV1.Read read = DatastoreIO.v1().read() .withProjectId(PROJECT_ID).withQuery(QUERY).withNamespace(NAMESPACE); assertEquals(QUERY, read.getQuery()); assertEquals(PROJECT_ID, read.getProjectId()); @@ -157,7 +157,7 @@ public void testBuildRead() throws Exception { */ @Test public void testBuildReadAlt() throws Exception { - V1Beta3.Read read = DatastoreIO.v1beta3().read() + DatastoreV1.Read read = DatastoreIO.v1().read() .withProjectId(PROJECT_ID).withNamespace(NAMESPACE).withQuery(QUERY); assertEquals(QUERY, read.getQuery()); assertEquals(PROJECT_ID, read.getProjectId()); @@ -166,7 +166,7 @@ public void testBuildReadAlt() throws Exception { @Test public void testReadValidationFailsProject() throws Exception { - V1Beta3.Read read = DatastoreIO.v1beta3().read().withQuery(QUERY); + DatastoreV1.Read read = DatastoreIO.v1().read().withQuery(QUERY); thrown.expect(NullPointerException.class); thrown.expectMessage("project"); read.validate(null); @@ -174,7 +174,7 @@ public void testReadValidationFailsProject() throws Exception { @Test public void testReadValidationFailsQuery() throws Exception { - V1Beta3.Read read = DatastoreIO.v1beta3().read().withProjectId(PROJECT_ID); + DatastoreV1.Read read = DatastoreIO.v1().read().withProjectId(PROJECT_ID); thrown.expect(NullPointerException.class); thrown.expectMessage("query"); read.validate(null); @@ -186,7 +186,7 @@ public void testReadValidationFailsQueryLimitZero() throws Exception { thrown.expect(IllegalArgumentException.class); thrown.expectMessage("Invalid query limit 0: must be positive"); - DatastoreIO.v1beta3().read().withQuery(invalidLimit); + DatastoreIO.v1().read().withQuery(invalidLimit); } @Test @@ -195,22 +195,22 @@ public void testReadValidationFailsQueryLimitNegative() throws Exception { thrown.expect(IllegalArgumentException.class); thrown.expectMessage("Invalid query limit -5: must be positive"); - DatastoreIO.v1beta3().read().withQuery(invalidLimit); + DatastoreIO.v1().read().withQuery(invalidLimit); } @Test public void testReadValidationSucceedsNamespace() throws Exception { - V1Beta3.Read read = DatastoreIO.v1beta3().read().withProjectId(PROJECT_ID).withQuery(QUERY); + DatastoreV1.Read read = DatastoreIO.v1().read().withProjectId(PROJECT_ID).withQuery(QUERY); /* Should succeed, as a null namespace is fine. */ read.validate(null); } @Test public void testReadDisplayData() { - V1Beta3.Read read = DatastoreIO.v1beta3().read() - .withProjectId(PROJECT_ID) - .withQuery(QUERY) - .withNamespace(NAMESPACE); + DatastoreV1.Read read = DatastoreIO.v1().read() + .withProjectId(PROJECT_ID) + .withQuery(QUERY) + .withNamespace(NAMESPACE); DisplayData displayData = DisplayData.from(read); @@ -223,7 +223,7 @@ public void testReadDisplayData() { @Category(RunnableOnService.class) public void testSourcePrimitiveDisplayData() { DisplayDataEvaluator evaluator = DisplayDataEvaluator.create(); - PTransform read = DatastoreIO.v1beta3().read().withProjectId( + PTransform read = DatastoreIO.v1().read().withProjectId( "myProject").withQuery(Query.newBuilder().build()); Set displayData = evaluator.displayDataForPrimitiveSourceTransforms(read); @@ -236,12 +236,12 @@ public void testWriteDoesNotAllowNullProject() throws Exception { thrown.expect(NullPointerException.class); thrown.expectMessage("projectId"); - DatastoreIO.v1beta3().write().withProjectId(null); + DatastoreIO.v1().write().withProjectId(null); } @Test public void testWriteValidationFailsWithNoProject() throws Exception { - Write write = DatastoreIO.v1beta3().write(); + Write write = DatastoreIO.v1().write(); thrown.expect(NullPointerException.class); thrown.expectMessage("projectId"); @@ -251,13 +251,13 @@ public void testWriteValidationFailsWithNoProject() throws Exception { @Test public void testWriteValidationSucceedsWithProject() throws Exception { - Write write = DatastoreIO.v1beta3().write().withProjectId(PROJECT_ID); + Write write = DatastoreIO.v1().write().withProjectId(PROJECT_ID); write.validate(null); } @Test public void testWriteDisplayData() { - Write write = DatastoreIO.v1beta3().write().withProjectId(PROJECT_ID); + Write write = DatastoreIO.v1().write().withProjectId(PROJECT_ID); DisplayData displayData = DisplayData.from(write); @@ -269,12 +269,12 @@ public void testDeleteEntityDoesNotAllowNullProject() throws Exception { thrown.expect(NullPointerException.class); thrown.expectMessage("projectId"); - DatastoreIO.v1beta3().deleteEntity().withProjectId(null); + DatastoreIO.v1().deleteEntity().withProjectId(null); } @Test public void testDeleteEntityValidationFailsWithNoProject() throws Exception { - DeleteEntity deleteEntity = DatastoreIO.v1beta3().deleteEntity(); + DeleteEntity deleteEntity = DatastoreIO.v1().deleteEntity(); thrown.expect(NullPointerException.class); thrown.expectMessage("projectId"); @@ -284,13 +284,13 @@ public void testDeleteEntityValidationFailsWithNoProject() throws Exception { @Test public void testDeleteEntityValidationSucceedsWithProject() throws Exception { - DeleteEntity deleteEntity = DatastoreIO.v1beta3().deleteEntity().withProjectId(PROJECT_ID); + DeleteEntity deleteEntity = DatastoreIO.v1().deleteEntity().withProjectId(PROJECT_ID); deleteEntity.validate(null); } @Test public void testDeleteEntityDisplayData() { - DeleteEntity deleteEntity = DatastoreIO.v1beta3().deleteEntity().withProjectId(PROJECT_ID); + DeleteEntity deleteEntity = DatastoreIO.v1().deleteEntity().withProjectId(PROJECT_ID); DisplayData displayData = DisplayData.from(deleteEntity); @@ -302,12 +302,12 @@ public void testDeleteKeyDoesNotAllowNullProject() throws Exception { thrown.expect(NullPointerException.class); thrown.expectMessage("projectId"); - DatastoreIO.v1beta3().deleteKey().withProjectId(null); + DatastoreIO.v1().deleteKey().withProjectId(null); } @Test public void testDeleteKeyValidationFailsWithNoProject() throws Exception { - DeleteKey deleteKey = DatastoreIO.v1beta3().deleteKey(); + DeleteKey deleteKey = DatastoreIO.v1().deleteKey(); thrown.expect(NullPointerException.class); thrown.expectMessage("projectId"); @@ -317,13 +317,13 @@ public void testDeleteKeyValidationFailsWithNoProject() throws Exception { @Test public void testDeleteKeyValidationSucceedsWithProject() throws Exception { - DeleteKey deleteKey = DatastoreIO.v1beta3().deleteKey().withProjectId(PROJECT_ID); + DeleteKey deleteKey = DatastoreIO.v1().deleteKey().withProjectId(PROJECT_ID); deleteKey.validate(null); } @Test public void testDeleteKeyDisplayData() { - DeleteKey deleteKey = DatastoreIO.v1beta3().deleteKey().withProjectId(PROJECT_ID); + DeleteKey deleteKey = DatastoreIO.v1().deleteKey().withProjectId(PROJECT_ID); DisplayData displayData = DisplayData.from(deleteKey); @@ -335,7 +335,7 @@ public void testDeleteKeyDisplayData() { public void testWritePrimitiveDisplayData() { DisplayDataEvaluator evaluator = DisplayDataEvaluator.create(); PTransform, ?> write = - DatastoreIO.v1beta3().write().withProjectId("myProject"); + DatastoreIO.v1().write().withProjectId("myProject"); Set displayData = evaluator.displayDataForPrimitiveTransforms(write); assertThat("DatastoreIO write should include the project in its primitive display data", @@ -350,7 +350,7 @@ public void testWritePrimitiveDisplayData() { public void testDeleteEntityPrimitiveDisplayData() { DisplayDataEvaluator evaluator = DisplayDataEvaluator.create(); PTransform, ?> write = - DatastoreIO.v1beta3().deleteEntity().withProjectId("myProject"); + DatastoreIO.v1().deleteEntity().withProjectId("myProject"); Set displayData = evaluator.displayDataForPrimitiveTransforms(write); assertThat("DatastoreIO write should include the project in its primitive display data", @@ -365,7 +365,7 @@ public void testDeleteEntityPrimitiveDisplayData() { public void testDeleteKeyPrimitiveDisplayData() { DisplayDataEvaluator evaluator = DisplayDataEvaluator.create(); PTransform, ?> write = - DatastoreIO.v1beta3().deleteKey().withProjectId("myProject"); + DatastoreIO.v1().deleteKey().withProjectId("myProject"); Set displayData = evaluator.displayDataForPrimitiveTransforms(write); assertThat("DatastoreIO write should include the project in its primitive display data", @@ -380,7 +380,7 @@ public void testDeleteKeyPrimitiveDisplayData() { */ @Test public void testBuildWrite() throws Exception { - V1Beta3.Write write = DatastoreIO.v1beta3().write().withProjectId(PROJECT_ID); + DatastoreV1.Write write = DatastoreIO.v1().write().withProjectId(PROJECT_ID); assertEquals(PROJECT_ID, write.getProjectId()); } @@ -550,7 +550,7 @@ private void datastoreWriterFnTest(int numMutations) throws Exception { int end = Math.min(numMutations, start + DATASTORE_BATCH_UPDATE_LIMIT); CommitRequest.Builder commitRequest = CommitRequest.newBuilder(); commitRequest.setMode(CommitRequest.Mode.NON_TRANSACTIONAL); - commitRequest.addAllMutations(mutations.subList(start, end)); + commitRequest.addAllMutations(mutations.subList(start, end)); // Verify all the batch requests were made with the expected mutations. verify(mockDatastore, times(1)).commit(commitRequest.build()); start = end; @@ -558,7 +558,7 @@ private void datastoreWriterFnTest(int numMutations) throws Exception { } /** - * Tests {@link V1Beta3.Read#getEstimatedSizeBytes} to fetch and return estimated size for a + * Tests {@link DatastoreV1.Read#getEstimatedSizeBytes} to fetch and return estimated size for a * query. */ @Test @@ -585,7 +585,7 @@ public void testSplitQueryFnWithNumSplits() throws Exception { eq(QUERY), any(PartitionId.class), eq(numSplits), any(Datastore.class))) .thenReturn(splitQuery(QUERY, numSplits)); - SplitQueryFn splitQueryFn = new SplitQueryFn(v1Beta3Options, numSplits, mockDatastoreFactory); + SplitQueryFn splitQueryFn = new SplitQueryFn(V_1_OPTIONS, numSplits, mockDatastoreFactory); DoFnTester> doFnTester = DoFnTester.of(splitQueryFn); /** * Although Datastore client is marked transient in {@link SplitQueryFn}, when injected through @@ -623,7 +623,7 @@ public void testSplitQueryFnWithoutNumSplits() throws Exception { eq(QUERY), any(PartitionId.class), eq(expectedNumSplits), any(Datastore.class))) .thenReturn(splitQuery(QUERY, expectedNumSplits)); - SplitQueryFn splitQueryFn = new SplitQueryFn(v1Beta3Options, numSplits, mockDatastoreFactory); + SplitQueryFn splitQueryFn = new SplitQueryFn(V_1_OPTIONS, numSplits, mockDatastoreFactory); DoFnTester> doFnTester = DoFnTester.of(splitQueryFn); doFnTester.setCloningBehavior(CloningBehavior.DO_NOT_CLONE); List> queries = doFnTester.processBundle(QUERY); @@ -636,7 +636,7 @@ public void testSplitQueryFnWithoutNumSplits() throws Exception { } /** - * Tests {@link V1Beta3.Read.SplitQueryFn} when the query has a user specified limit. + * Tests {@link DatastoreV1.Read.SplitQueryFn} when the query has a user specified limit. */ @Test public void testSplitQueryFnWithQueryLimit() throws Exception { @@ -644,7 +644,7 @@ public void testSplitQueryFnWithQueryLimit() throws Exception { .setLimit(Int32Value.newBuilder().setValue(1)) .build(); - SplitQueryFn splitQueryFn = new SplitQueryFn(v1Beta3Options, 10, mockDatastoreFactory); + SplitQueryFn splitQueryFn = new SplitQueryFn(V_1_OPTIONS, 10, mockDatastoreFactory); DoFnTester> doFnTester = DoFnTester.of(splitQueryFn); doFnTester.setCloningBehavior(CloningBehavior.DO_NOT_CLONE); List> queries = doFnTester.processBundle(queryWithLimit); @@ -687,10 +687,10 @@ private void verifyUniqueKeys(List> queries) { /** * A helper function that creates mock {@link Entity} results in response to a query. Always * indicates that more results are available, unless the batch is limited to fewer than - * {@link V1Beta3.Read#QUERY_BATCH_LIMIT} results. + * {@link DatastoreV1.Read#QUERY_BATCH_LIMIT} results. */ private static RunQueryResponse mockResponseForQuery(Query q) { - // Every query V1Beta3 sends should have a limit. + // Every query DatastoreV1 sends should have a limit. assertTrue(q.hasLimit()); // The limit should be in the range [1, QUERY_BATCH_LIMIT] @@ -736,7 +736,7 @@ public RunQueryResponse answer(InvocationOnMock invocationOnMock) throws Throwab } }); - ReadFn readFn = new ReadFn(v1Beta3Options, mockDatastoreFactory); + ReadFn readFn = new ReadFn(V_1_OPTIONS, mockDatastoreFactory); DoFnTester doFnTester = DoFnTester.of(readFn); /** * Although Datastore client is marked transient in {@link ReadFn}, when injected through diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/V1Beta3ReadIT.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/V1ReadIT.java similarity index 69% rename from sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/V1Beta3ReadIT.java rename to sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/V1ReadIT.java index ddb6d81ae995e..8fedc774f661c 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/V1Beta3ReadIT.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/V1ReadIT.java @@ -18,24 +18,24 @@ package org.apache.beam.sdk.io.gcp.datastore; -import static org.apache.beam.sdk.io.gcp.datastore.V1Beta3TestUtil.deleteAllEntities; -import static org.apache.beam.sdk.io.gcp.datastore.V1Beta3TestUtil.getDatastore; -import static org.apache.beam.sdk.io.gcp.datastore.V1Beta3TestUtil.makeAncestorKey; -import static org.apache.beam.sdk.io.gcp.datastore.V1Beta3TestUtil.makeEntity; +import static org.apache.beam.sdk.io.gcp.datastore.V1TestUtil.deleteAllEntities; +import static org.apache.beam.sdk.io.gcp.datastore.V1TestUtil.getDatastore; +import static org.apache.beam.sdk.io.gcp.datastore.V1TestUtil.makeAncestorKey; +import static org.apache.beam.sdk.io.gcp.datastore.V1TestUtil.makeEntity; import org.apache.beam.sdk.Pipeline; -import org.apache.beam.sdk.io.gcp.datastore.V1Beta3TestUtil.UpsertMutationBuilder; -import org.apache.beam.sdk.io.gcp.datastore.V1Beta3TestUtil.V1Beta3TestWriter; +import org.apache.beam.sdk.io.gcp.datastore.V1TestUtil.UpsertMutationBuilder; +import org.apache.beam.sdk.io.gcp.datastore.V1TestUtil.V1TestWriter; import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.testing.PAssert; import org.apache.beam.sdk.testing.TestPipeline; import org.apache.beam.sdk.transforms.Count; import org.apache.beam.sdk.values.PCollection; -import com.google.datastore.v1beta3.Entity; -import com.google.datastore.v1beta3.Key; -import com.google.datastore.v1beta3.Query; -import com.google.datastore.v1beta3.client.Datastore; +import com.google.datastore.v1.Entity; +import com.google.datastore.v1.Key; +import com.google.datastore.v1.Query; +import com.google.datastore.v1.client.Datastore; import org.junit.After; import org.junit.Before; @@ -46,38 +46,38 @@ import java.util.UUID; /** - * End-to-end tests for Datastore V1Beta3.Read. + * End-to-end tests for Datastore DatastoreV1.Read. */ @RunWith(JUnit4.class) -public class V1Beta3ReadIT { - private V1Beta3TestOptions options; +public class V1ReadIT { + private V1TestOptions options; private String ancestor; private final long numEntities = 1000; @Before public void setup() { - PipelineOptionsFactory.register(V1Beta3TestOptions.class); - options = TestPipeline.testingPipelineOptions().as(V1Beta3TestOptions.class); + PipelineOptionsFactory.register(V1TestOptions.class); + options = TestPipeline.testingPipelineOptions().as(V1TestOptions.class); ancestor = UUID.randomUUID().toString(); } /** - * An end-to-end test for {@link V1Beta3.Read}. + * An end-to-end test for {@link DatastoreV1.Read}. * * Write some test entities to datastore and then run a dataflow pipeline that * reads and counts the total number of entities. Verify that the count matches the * number of entities written. */ @Test - public void testE2EV1Beta3Read() throws Exception { + public void testE2EV1Read() throws Exception { // Create entities and write them to datastore writeEntitiesToDatastore(options, ancestor, numEntities); // Read from datastore - Query query = V1Beta3TestUtil.makeAncestorKindQuery( + Query query = V1TestUtil.makeAncestorKindQuery( options.getKind(), options.getNamespace(), ancestor); - V1Beta3.Read read = DatastoreIO.v1beta3().read() + DatastoreV1.Read read = DatastoreIO.v1().read() .withProjectId(options.getProject()) .withQuery(query) .withNamespace(options.getNamespace()); @@ -93,11 +93,11 @@ public void testE2EV1Beta3Read() throws Exception { } // Creates entities and write them to datastore - private static void writeEntitiesToDatastore(V1Beta3TestOptions options, String ancestor, + private static void writeEntitiesToDatastore(V1TestOptions options, String ancestor, long numEntities) throws Exception { Datastore datastore = getDatastore(options, options.getProject()); // Write test entities to datastore - V1Beta3TestWriter writer = new V1Beta3TestWriter(datastore, new UpsertMutationBuilder()); + V1TestWriter writer = new V1TestWriter(datastore, new UpsertMutationBuilder()); Key ancestorKey = makeAncestorKey(options.getNamespace(), options.getKind(), ancestor); for (long i = 0; i < numEntities; i++) { diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/V1Beta3TestOptions.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/V1TestOptions.java similarity index 92% rename from sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/V1Beta3TestOptions.java rename to sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/V1TestOptions.java index 099ebe05c9b7e..360855fee0f2c 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/V1Beta3TestOptions.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/V1TestOptions.java @@ -25,9 +25,9 @@ import javax.annotation.Nullable; /** - * V1Beta3 Datastore related pipeline options. + * DatastoreV1 Datastore related pipeline options. */ -public interface V1Beta3TestOptions extends TestPipelineOptions { +public interface V1TestOptions extends TestPipelineOptions { @Description("Project ID to read from datastore") @Default.String("apache-beam-testing") String getProject(); diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/V1Beta3TestUtil.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/V1TestUtil.java similarity index 82% rename from sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/V1Beta3TestUtil.java rename to sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/V1TestUtil.java index 7eaf23e805689..1e323ec06a28c 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/V1Beta3TestUtil.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/V1TestUtil.java @@ -18,12 +18,12 @@ package org.apache.beam.sdk.io.gcp.datastore; -import static com.google.datastore.v1beta3.QueryResultBatch.MoreResultsType.NOT_FINISHED; -import static com.google.datastore.v1beta3.client.DatastoreHelper.makeDelete; -import static com.google.datastore.v1beta3.client.DatastoreHelper.makeFilter; -import static com.google.datastore.v1beta3.client.DatastoreHelper.makeKey; -import static com.google.datastore.v1beta3.client.DatastoreHelper.makeUpsert; -import static com.google.datastore.v1beta3.client.DatastoreHelper.makeValue; +import static com.google.datastore.v1.QueryResultBatch.MoreResultsType.NOT_FINISHED; +import static com.google.datastore.v1.client.DatastoreHelper.makeDelete; +import static com.google.datastore.v1.client.DatastoreHelper.makeFilter; +import static com.google.datastore.v1.client.DatastoreHelper.makeKey; +import static com.google.datastore.v1.client.DatastoreHelper.makeUpsert; +import static com.google.datastore.v1.client.DatastoreHelper.makeValue; import org.apache.beam.sdk.options.GcpOptions; import org.apache.beam.sdk.options.PipelineOptions; @@ -35,21 +35,21 @@ import com.google.api.client.util.BackOff; import com.google.api.client.util.BackOffUtils; import com.google.api.client.util.Sleeper; -import com.google.datastore.v1beta3.CommitRequest; -import com.google.datastore.v1beta3.Entity; -import com.google.datastore.v1beta3.EntityResult; -import com.google.datastore.v1beta3.Key; -import com.google.datastore.v1beta3.Key.PathElement; -import com.google.datastore.v1beta3.Mutation; -import com.google.datastore.v1beta3.PropertyFilter; -import com.google.datastore.v1beta3.Query; -import com.google.datastore.v1beta3.QueryResultBatch; -import com.google.datastore.v1beta3.RunQueryRequest; -import com.google.datastore.v1beta3.RunQueryResponse; -import com.google.datastore.v1beta3.client.Datastore; -import com.google.datastore.v1beta3.client.DatastoreException; -import com.google.datastore.v1beta3.client.DatastoreFactory; -import com.google.datastore.v1beta3.client.DatastoreOptions; +import com.google.datastore.v1.CommitRequest; +import com.google.datastore.v1.Entity; +import com.google.datastore.v1.EntityResult; +import com.google.datastore.v1.Key; +import com.google.datastore.v1.Key.PathElement; +import com.google.datastore.v1.Mutation; +import com.google.datastore.v1.PropertyFilter; +import com.google.datastore.v1.Query; +import com.google.datastore.v1.QueryResultBatch; +import com.google.datastore.v1.RunQueryRequest; +import com.google.datastore.v1.RunQueryResponse; +import com.google.datastore.v1.client.Datastore; +import com.google.datastore.v1.client.DatastoreException; +import com.google.datastore.v1.client.DatastoreFactory; +import com.google.datastore.v1.client.DatastoreOptions; import com.google.protobuf.Int32Value; import org.slf4j.Logger; @@ -62,8 +62,8 @@ import java.util.UUID; import javax.annotation.Nullable; -class V1Beta3TestUtil { - private static final Logger LOG = LoggerFactory.getLogger(V1Beta3TestUtil.class); +class V1TestUtil { + private static final Logger LOG = LoggerFactory.getLogger(V1TestUtil.class); /** * A helper function to create the ancestor key for all created and queried entities. @@ -161,13 +161,13 @@ private static RunQueryRequest makeRequest(Query query, @Nullable String namespa /** * Delete all entities with the given ancestor. */ - static void deleteAllEntities(V1Beta3TestOptions options, String ancestor) throws Exception { + static void deleteAllEntities(V1TestOptions options, String ancestor) throws Exception { Datastore datastore = getDatastore(options, options.getProject()); - Query query = V1Beta3TestUtil.makeAncestorKindQuery( + Query query = V1TestUtil.makeAncestorKindQuery( options.getKind(), options.getNamespace(), ancestor); - V1Beta3TestReader reader = new V1Beta3TestReader(datastore, query, options.getNamespace()); - V1Beta3TestWriter writer = new V1Beta3TestWriter(datastore, new DeleteMutationBuilder()); + V1TestReader reader = new V1TestReader(datastore, query, options.getNamespace()); + V1TestWriter writer = new V1TestWriter(datastore, new DeleteMutationBuilder()); long numEntities = 0; while (reader.advance()) { @@ -183,13 +183,13 @@ static void deleteAllEntities(V1Beta3TestOptions options, String ancestor) throw /** * Returns the total number of entities for the given datastore. */ - static long countEntities(V1Beta3TestOptions options, String ancestor) throws Exception { + static long countEntities(V1TestOptions options, String ancestor) throws Exception { // Read from datastore. - Datastore datastore = V1Beta3TestUtil.getDatastore(options, options.getProject()); - Query query = V1Beta3TestUtil.makeAncestorKindQuery( + Datastore datastore = V1TestUtil.getDatastore(options, options.getProject()); + Query query = V1TestUtil.makeAncestorKindQuery( options.getKind(), options.getNamespace(), ancestor); - V1Beta3TestReader reader = new V1Beta3TestReader(datastore, query, options.getNamespace()); + V1TestReader reader = new V1TestReader(datastore, query, options.getNamespace()); long numEntitiesRead = 0; while (reader.advance()) { @@ -228,8 +228,8 @@ public Mutation.Builder apply(Entity entity) { /** * A helper class to write entities to datastore. */ - static class V1Beta3TestWriter { - private static final Logger LOG = LoggerFactory.getLogger(V1Beta3TestWriter.class); + static class V1TestWriter { + private static final Logger LOG = LoggerFactory.getLogger(V1TestWriter.class); // Limits the number of entities updated per batch private static final int DATASTORE_BATCH_UPDATE_LIMIT = 500; // Number of times to retry on update failure @@ -252,7 +252,7 @@ static boolean isValidKey(Key key) { private final MutationBuilder mutationBuilder; private final List entities = new ArrayList<>(); - V1Beta3TestWriter(Datastore datastore, MutationBuilder mutationBuilder) { + V1TestWriter(Datastore datastore, MutationBuilder mutationBuilder) { this.datastore = datastore; this.mutationBuilder = mutationBuilder; } @@ -312,7 +312,7 @@ private void flushBatch() throws DatastoreException, IOException, InterruptedExc /** * A helper class to read entities from datastore. */ - static class V1Beta3TestReader { + static class V1TestReader { private static final int QUERY_BATCH_LIMIT = 500; private final Datastore datastore; private final Query query; @@ -324,7 +324,7 @@ static class V1Beta3TestReader { private QueryResultBatch currentBatch; private Entity currentEntity; - V1Beta3TestReader(Datastore datastore, Query query, @Nullable String namespace) { + V1TestReader(Datastore datastore, Query query, @Nullable String namespace) { this.datastore = datastore; this.query = query; this.namespace = namespace; @@ -368,7 +368,7 @@ private Iterator getIteratorAndMoveCursor() throws DatastoreExcept int numFetch = currentBatch.getEntityResultsCount(); // All indications from the API are that there are/may be more results. moreResults = ((numFetch == QUERY_BATCH_LIMIT) - || (currentBatch.getMoreResults() == NOT_FINISHED)); + || (currentBatch.getMoreResults() == NOT_FINISHED)); // May receive a batch of 0 results if the number of records is a multiple // of the request limit. diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/V1Beta3WriteIT.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/V1WriteIT.java similarity index 76% rename from sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/V1Beta3WriteIT.java rename to sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/V1WriteIT.java index 782065fab101c..b97c05c8c6c4c 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/V1Beta3WriteIT.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/V1WriteIT.java @@ -18,13 +18,13 @@ package org.apache.beam.sdk.io.gcp.datastore; -import static org.apache.beam.sdk.io.gcp.datastore.V1Beta3TestUtil.countEntities; -import static org.apache.beam.sdk.io.gcp.datastore.V1Beta3TestUtil.deleteAllEntities; +import static org.apache.beam.sdk.io.gcp.datastore.V1TestUtil.countEntities; +import static org.apache.beam.sdk.io.gcp.datastore.V1TestUtil.deleteAllEntities; import static org.junit.Assert.assertEquals; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.io.CountingInput; -import org.apache.beam.sdk.io.gcp.datastore.V1Beta3TestUtil.CreateEntityFn; +import org.apache.beam.sdk.io.gcp.datastore.V1TestUtil.CreateEntityFn; import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.testing.TestPipeline; import org.apache.beam.sdk.transforms.ParDo; @@ -38,37 +38,37 @@ import java.util.UUID; /** - * End-to-end tests for Datastore V1Beta3.Write. + * End-to-end tests for Datastore DatastoreV1.Write. */ @RunWith(JUnit4.class) -public class V1Beta3WriteIT { - private V1Beta3TestOptions options; +public class V1WriteIT { + private V1TestOptions options; private String ancestor; private final long numEntities = 1000; @Before public void setup() { - PipelineOptionsFactory.register(V1Beta3TestOptions.class); - options = TestPipeline.testingPipelineOptions().as(V1Beta3TestOptions.class); + PipelineOptionsFactory.register(V1TestOptions.class); + options = TestPipeline.testingPipelineOptions().as(V1TestOptions.class); ancestor = UUID.randomUUID().toString(); } /** - * An end-to-end test for {@link V1Beta3.Write}. + * An end-to-end test for {@link DatastoreV1.Write}. * * Write some test entities to datastore through a dataflow pipeline. * Read and count all the entities. Verify that the count matches the * number of entities written. */ @Test - public void testE2EV1Beta3Write() throws Exception { + public void testE2EV1Write() throws Exception { Pipeline p = Pipeline.create(options); // Write to datastore p.apply(CountingInput.upTo(numEntities)) .apply(ParDo.of(new CreateEntityFn( options.getKind(), options.getNamespace(), ancestor))) - .apply(DatastoreIO.v1beta3().write().withProjectId(options.getProject())); + .apply(DatastoreIO.v1().write().withProjectId(options.getProject())); p.run(); From c3c11b188b27f8b647ca9668a43fc94a3b85e98a Mon Sep 17 00:00:00 2001 From: Thomas Groh Date: Mon, 22 Aug 2016 09:23:57 -0700 Subject: [PATCH 046/346] Remove unused constant in ExecutorServiceParallelExecutor --- .../beam/runners/direct/ExecutorServiceParallelExecutor.java | 2 -- 1 file changed, 2 deletions(-) diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ExecutorServiceParallelExecutor.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ExecutorServiceParallelExecutor.java index 8c6c6eddb3b31..35b6239de81c7 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 @@ -338,8 +338,6 @@ public boolean isDone() { } private class MonitorRunnable implements Runnable { - // arbitrary termination condition to ensure progress in the presence of pushback - private final long maxTimeProcessingUpdatesNanos = TimeUnit.MILLISECONDS.toNanos(5L); private final String runnableName = String.format("%s$%s-monitor", evaluationContext.getPipelineOptions().getAppName(), ExecutorServiceParallelExecutor.class.getSimpleName()); From 64a2d51a00315da6ac859dc7187140ffda565483 Mon Sep 17 00:00:00 2001 From: Thomas Groh Date: Mon, 22 Aug 2016 10:05:35 -0700 Subject: [PATCH 047/346] Remove extra timer firings in WatermarkManager These timers should not be fired - the windows should be expired via the GC timer, and any elements should be emitted if neccessary. --- .../beam/runners/direct/WatermarkManager.java | 15 ++++----------- 1 file changed, 4 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 c8dfa8c914c45..a44fa501c434f 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 @@ -1139,17 +1139,10 @@ private Map, FiredTimers> extractFiredTimers() { inputWatermark.extractFiredEventTimeTimers(); Map, List> processingTimers; Map, List> synchronizedTimers; - if (inputWatermark.get().equals(BoundedWindow.TIMESTAMP_MAX_VALUE)) { - processingTimers = synchronizedProcessingInputWatermark.extractFiredDomainTimers( - TimeDomain.PROCESSING_TIME, BoundedWindow.TIMESTAMP_MAX_VALUE); - synchronizedTimers = synchronizedProcessingInputWatermark.extractFiredDomainTimers( - TimeDomain.PROCESSING_TIME, BoundedWindow.TIMESTAMP_MAX_VALUE); - } else { - processingTimers = synchronizedProcessingInputWatermark.extractFiredDomainTimers( - TimeDomain.PROCESSING_TIME, clock.now()); - synchronizedTimers = synchronizedProcessingInputWatermark.extractFiredDomainTimers( - TimeDomain.SYNCHRONIZED_PROCESSING_TIME, getSynchronizedProcessingInputTime()); - } + processingTimers = synchronizedProcessingInputWatermark.extractFiredDomainTimers( + TimeDomain.PROCESSING_TIME, clock.now()); + synchronizedTimers = synchronizedProcessingInputWatermark.extractFiredDomainTimers( + TimeDomain.SYNCHRONIZED_PROCESSING_TIME, getSynchronizedProcessingInputTime()); Map, Map>> groupedTimers = new HashMap<>(); groupFiredTimers(groupedTimers, eventTimeTimers, processingTimers, synchronizedTimers); From a60806a9de5eaa909d4f8d388390db4049ecc335 Mon Sep 17 00:00:00 2001 From: Dan Halperin Date: Sun, 14 Aug 2016 23:08:21 -0700 Subject: [PATCH 048/346] FileBasedSink: improve parallelism in GCS copy/remove --- .../org/apache/beam/sdk/util/GcsUtil.java | 241 ++++++++---------- .../org/apache/beam/sdk/util/GcsUtilTest.java | 69 +++++ 2 files changed, 178 insertions(+), 132 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 4e9ee6e62ec88..06685e577bf93 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 @@ -30,11 +30,9 @@ import com.google.api.client.googleapis.json.GoogleJsonError; import com.google.api.client.googleapis.json.GoogleJsonResponseException; import com.google.api.client.http.HttpHeaders; -import com.google.api.client.http.HttpRequestInitializer; import com.google.api.client.util.BackOff; import com.google.api.client.util.Sleeper; import com.google.api.services.storage.Storage; -import com.google.api.services.storage.StorageRequest; import com.google.api.services.storage.model.Objects; import com.google.api.services.storage.model.StorageObject; import com.google.cloud.hadoop.gcsio.GoogleCloudStorageReadChannel; @@ -47,6 +45,11 @@ import com.google.cloud.hadoop.util.RetryDeterminer; import com.google.common.annotations.VisibleForTesting; import com.google.common.collect.ImmutableList; +import com.google.common.collect.Lists; +import com.google.common.util.concurrent.Futures; +import com.google.common.util.concurrent.ListenableFuture; +import com.google.common.util.concurrent.ListeningExecutorService; +import com.google.common.util.concurrent.MoreExecutors; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -59,12 +62,16 @@ import java.util.Collections; import java.util.LinkedList; import java.util.List; +import java.util.concurrent.Callable; +import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutorService; +import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.ThreadPoolExecutor; +import java.util.concurrent.TimeUnit; import java.util.regex.Matcher; import java.util.regex.Pattern; import javax.annotation.Nullable; -import javax.annotation.concurrent.NotThreadSafe; /** * Provides operations on GCS. @@ -110,7 +117,11 @@ public GcsUtil create(PipelineOptions options) { /** * Maximum number of requests permitted in a GCS batch request. */ - private static final int MAX_REQUESTS_PER_BATCH = 1000; + private static final int MAX_REQUESTS_PER_BATCH = 100; + /** + * Maximum number of concurrent batches of requests executing on GCS. + */ + private static final int MAX_CONCURRENT_BATCHES = 256; ///////////////////////////////////////////////////////////////////////////// @@ -125,7 +136,6 @@ public GcsUtil create(PipelineOptions options) { // Exposed for testing. final ExecutorService executorService; - private final BatchHelper batchHelper; /** * Returns true if the given GCS pattern is supported otherwise fails with an * exception. @@ -145,8 +155,6 @@ private GcsUtil( this.storageClient = storageClient; this.uploadBufferSizeBytes = uploadBufferSizeBytes; this.executorService = executorService; - this.batchHelper = new BatchHelper( - storageClient.getRequestFactory().getInitializer(), storageClient, MAX_REQUESTS_PER_BATCH); } // Use this only for testing purposes. @@ -372,154 +380,123 @@ public boolean shouldRetry(IOException e) { } } + private static void executeBatches(List batches) throws IOException { + ListeningExecutorService executor = MoreExecutors.listeningDecorator( + MoreExecutors.getExitingExecutorService( + new ThreadPoolExecutor(MAX_CONCURRENT_BATCHES, MAX_CONCURRENT_BATCHES, + 0L, TimeUnit.MILLISECONDS, + new LinkedBlockingQueue()))); + + List> futures = new LinkedList<>(); + for (final BatchRequest batch : batches) { + futures.add(executor.submit(new Callable() { + public Void call() throws IOException { + batch.execute(); + return null; + } + })); + } + + try { + Futures.allAsList(futures).get(); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new IOException("Interrupted while executing batch GCS request", e); + } catch (ExecutionException e) { + throw new IOException("Error executing batch GCS request", e); + } finally { + executor.shutdown(); + } + } + public void copy(List srcFilenames, List destFilenames) throws IOException { + executeBatches(makeCopyBatches(srcFilenames, destFilenames)); + } + + List makeCopyBatches(List srcFilenames, List destFilenames) + throws IOException { checkArgument( srcFilenames.size() == destFilenames.size(), "Number of source files %s must equal number of destination files %s", srcFilenames.size(), destFilenames.size()); + + List batches = new LinkedList<>(); + BatchRequest batch = storageClient.batch(); for (int i = 0; i < srcFilenames.size(); i++) { final GcsPath sourcePath = GcsPath.fromUri(srcFilenames.get(i)); final GcsPath destPath = GcsPath.fromUri(destFilenames.get(i)); - LOG.debug("Copying {} to {}", sourcePath, destPath); - Storage.Objects.Copy copyObject = storageClient.objects().copy(sourcePath.getBucket(), - sourcePath.getObject(), destPath.getBucket(), destPath.getObject(), null); - batchHelper.queue(copyObject, new JsonBatchCallback() { - @Override - public void onSuccess(StorageObject obj, HttpHeaders responseHeaders) { - LOG.debug("Successfully copied {} to {}", sourcePath, destPath); - } - - @Override - public void onFailure(GoogleJsonError e, HttpHeaders responseHeaders) throws IOException { - // Do nothing on item not found. - if (!errorExtractor.itemNotFound(e)) { - throw new IOException(e.toString()); - } - LOG.debug("{} does not exist.", sourcePath); - } - }); + enqueueCopy(sourcePath, destPath, batch); + if (batch.size() >= MAX_REQUESTS_PER_BATCH) { + batches.add(batch); + batch = storageClient.batch(); + } } - batchHelper.flush(); - } - - public void remove(Collection filenames) throws IOException { - for (String filename : filenames) { - final GcsPath path = GcsPath.fromUri(filename); - LOG.debug("Removing: " + path); - Storage.Objects.Delete deleteObject = - storageClient.objects().delete(path.getBucket(), path.getObject()); - batchHelper.queue(deleteObject, new JsonBatchCallback() { - @Override - public void onSuccess(Void obj, HttpHeaders responseHeaders) throws IOException { - LOG.debug("Successfully removed {}", path); - } - - @Override - public void onFailure(GoogleJsonError e, HttpHeaders responseHeaders) throws IOException { - // Do nothing on item not found. - if (!errorExtractor.itemNotFound(e)) { - throw new IOException(e.toString()); - } - LOG.debug("{} does not exist.", path); - } - }); + if (batch.size() > 0) { + batches.add(batch); } - batchHelper.flush(); + return batches; } - /** - * BatchHelper abstracts out the logic for the maximum requests per batch for GCS. - * - *

    Copy of - * https://github.com/GoogleCloudPlatform/bigdata-interop/blob/master/gcs/src/main/java/com/google/cloud/hadoop/gcsio/BatchHelper.java - * - *

    Copied to prevent Dataflow from depending on the Hadoop-related dependencies that are not - * used in Dataflow. Hadoop-related dependencies will be removed from the Google Cloud Storage - * Connector (https://cloud.google.com/hadoop/google-cloud-storage-connector) so that this project - * and others may use the connector without introducing unnecessary dependencies. - * - *

    This class is not thread-safe; create a new BatchHelper instance per single-threaded logical - * grouping of requests. - */ - @NotThreadSafe - private static class BatchHelper { - /** - * Callback that causes a single StorageRequest to be added to the BatchRequest. - */ - protected static interface QueueRequestCallback { - void enqueue() throws IOException; + List makeRemoveBatches(Collection filenames) throws IOException { + List batches = new LinkedList<>(); + for (List filesToDelete : + Lists.partition(Lists.newArrayList(filenames), MAX_REQUESTS_PER_BATCH)) { + BatchRequest batch = storageClient.batch(); + for (String file : filesToDelete) { + enqueueDelete(GcsPath.fromUri(file), batch); + } + batches.add(batch); } + return batches; + } - private final List pendingBatchEntries; - private final BatchRequest batch; - - // Number of requests that can be queued into a single actual HTTP request - // before a sub-batch is sent. - private final long maxRequestsPerBatch; - - // Flag that indicates whether there is an in-progress flush. - private boolean flushing = false; + public void remove(Collection filenames) throws IOException { + executeBatches(makeRemoveBatches(filenames)); + } - /** - * Primary constructor, generally accessed only via the inner Factory class. - */ - public BatchHelper( - HttpRequestInitializer requestInitializer, Storage gcs, long maxRequestsPerBatch) { - this.pendingBatchEntries = new LinkedList<>(); - this.batch = gcs.batch(requestInitializer); - this.maxRequestsPerBatch = maxRequestsPerBatch; - } + private void enqueueCopy(final GcsPath from, final GcsPath to, BatchRequest batch) + throws IOException { + Storage.Objects.Copy copyRequest = storageClient.objects() + .copy(from.getBucket(), from.getObject(), to.getBucket(), to.getObject(), null); + copyRequest.queue(batch, new JsonBatchCallback() { + @Override + public void onSuccess(StorageObject obj, HttpHeaders responseHeaders) { + LOG.debug("Successfully copied {} to {}", from, to); + } - /** - * Adds an additional request to the batch, and possibly flushes the current contents of the - * batch if {@code maxRequestsPerBatch} has been reached. - */ - public void queue(final StorageRequest req, final JsonBatchCallback callback) - throws IOException { - QueueRequestCallback queueCallback = new QueueRequestCallback() { - @Override - public void enqueue() throws IOException { - req.queue(batch, callback); + @Override + public void onFailure(GoogleJsonError e, HttpHeaders responseHeaders) throws IOException { + if (errorExtractor.itemNotFound(e)) { + // Do nothing on item not found. + LOG.debug("{} does not exist, assuming this is a retry after deletion.", from); + return; } - }; - pendingBatchEntries.add(queueCallback); - - flushIfPossibleAndRequired(); - } - - // Flush our buffer if we have more pending entries than maxRequestsPerBatch - private void flushIfPossibleAndRequired() throws IOException { - if (pendingBatchEntries.size() > maxRequestsPerBatch) { - flushIfPossible(); + throw new IOException( + String.format("Error trying to copy %s to %s: %s", from, to, e)); } - } + }); + } - // Flush our buffer if we are not already in a flush operation and we have data to flush. - private void flushIfPossible() throws IOException { - if (!flushing && pendingBatchEntries.size() > 0) { - flushing = true; - try { - while (batch.size() < maxRequestsPerBatch && pendingBatchEntries.size() > 0) { - QueueRequestCallback head = pendingBatchEntries.remove(0); - head.enqueue(); - } + private void enqueueDelete(final GcsPath file, BatchRequest batch) throws IOException { + Storage.Objects.Delete deleteRequest = storageClient.objects() + .delete(file.getBucket(), file.getObject()); + deleteRequest.queue(batch, new JsonBatchCallback() { + @Override + public void onSuccess(Void obj, HttpHeaders responseHeaders) { + LOG.debug("Successfully deleted {}", file); + } - batch.execute(); - } finally { - flushing = false; + @Override + public void onFailure(GoogleJsonError e, HttpHeaders responseHeaders) throws IOException { + if (errorExtractor.itemNotFound(e)) { + // Do nothing on item not found. + LOG.debug("{} does not exist.", file); + return; } + throw new IOException(String.format("Error trying to delete %s: %s", file, e)); } - } - - - /** - * Sends any currently remaining requests in the batch; should be called at the end of any - * series of batched requests to ensure everything has been sent. - */ - public void flush() throws IOException { - flushIfPossible(); - } + }); } /** 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 49c7bc48ec4d6..997340a22fa2f 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 @@ -18,6 +18,8 @@ package org.apache.beam.sdk.util; import static org.hamcrest.Matchers.contains; +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.greaterThan; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertNotNull; @@ -32,6 +34,7 @@ import org.apache.beam.sdk.testing.FastNanoClockAndSleeper; import org.apache.beam.sdk.util.gcsfs.GcsPath; +import com.google.api.client.googleapis.batch.BatchRequest; import com.google.api.client.googleapis.json.GoogleJsonError.ErrorInfo; import com.google.api.client.googleapis.json.GoogleJsonResponseException; import com.google.api.client.http.HttpRequest; @@ -490,4 +493,70 @@ public LowLevelHttpRequest buildRequest(String method, String url) throws IOExce HttpResponse response = request.execute(); return GoogleJsonResponseException.from(jsonFactory, response); } + + private static List makeStrings(String s, int n) { + ImmutableList.Builder ret = ImmutableList.builder(); + for (int i = 0; i < n; ++i) { + ret.add(String.format("gs://bucket/%s%d", s, i)); + } + return ret.build(); + } + + private static int sumBatchSizes(List batches) { + int ret = 0; + for (BatchRequest b : batches) { + ret += b.size(); + assertThat(b.size(), greaterThan(0)); + } + return ret; + } + + @Test + public void testMakeCopyBatches() throws IOException { + GcsUtil gcsUtil = gcsOptionsWithTestCredential().getGcsUtil(); + + // Small number of files fits in 1 batch + List batches = gcsUtil.makeCopyBatches(makeStrings("s", 3), makeStrings("d", 3)); + assertThat(batches.size(), equalTo(1)); + assertThat(sumBatchSizes(batches), equalTo(3)); + + // 1 batch of files fits in 1 batch + batches = gcsUtil.makeCopyBatches(makeStrings("s", 100), makeStrings("d", 100)); + assertThat(batches.size(), equalTo(1)); + assertThat(sumBatchSizes(batches), equalTo(100)); + + // A little more than 5 batches of files fits in 6 batches + batches = gcsUtil.makeCopyBatches(makeStrings("s", 501), makeStrings("d", 501)); + assertThat(batches.size(), equalTo(6)); + assertThat(sumBatchSizes(batches), equalTo(501)); + } + + @Test + public void testInvalidCopyBatches() throws IOException { + GcsUtil gcsUtil = gcsOptionsWithTestCredential().getGcsUtil(); + thrown.expect(IllegalArgumentException.class); + thrown.expectMessage("Number of source files 3"); + + gcsUtil.makeCopyBatches(makeStrings("s", 3), makeStrings("d", 1)); + } + + @Test + public void testMakeRemoveBatches() throws IOException { + GcsUtil gcsUtil = gcsOptionsWithTestCredential().getGcsUtil(); + + // Small number of files fits in 1 batch + List batches = gcsUtil.makeRemoveBatches(makeStrings("s", 3)); + assertThat(batches.size(), equalTo(1)); + assertThat(sumBatchSizes(batches), equalTo(3)); + + // 1 batch of files fits in 1 batch + batches = gcsUtil.makeRemoveBatches(makeStrings("s", 100)); + assertThat(batches.size(), equalTo(1)); + assertThat(sumBatchSizes(batches), equalTo(100)); + + // A little more than 5 batches of files fits in 6 batches + batches = gcsUtil.makeRemoveBatches(makeStrings("s", 501)); + assertThat(batches.size(), equalTo(6)); + assertThat(sumBatchSizes(batches), equalTo(501)); + } } From 780ffcb24bd7cc430c9f70971cf376d52e3c6171 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Isma=C3=ABl=20Mej=C3=ADa?= Date: Mon, 22 Aug 2016 17:25:32 +0200 Subject: [PATCH 049/346] [BEAM-574] Remove log when new records have not been read yet (KafkaIO) --- .../src/main/java/org/apache/beam/sdk/io/kafka/KafkaIO.java | 1 - 1 file changed, 1 deletion(-) 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 8a0c7880e97db..6cc5d6a597d62 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 @@ -1076,7 +1076,6 @@ private void updateLatestOffsets() { @Override public Instant getWatermark() { if (curRecord == null) { - LOG.warn("{}: getWatermark() : no records have been read yet.", name); return initialWatermark; } From 730e7b05e30127c7f6c314708c038bedae8a23c3 Mon Sep 17 00:00:00 2001 From: Daniel Halperin Date: Tue, 23 Aug 2016 17:39:19 -0700 Subject: [PATCH 050/346] Write: Remove invalid import --- sdks/java/core/src/main/java/org/apache/beam/sdk/io/Write.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/Write.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/Write.java index fea65ef233984..a474528e04e6f 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 @@ -45,7 +45,7 @@ import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.PDone; -import com.google.api.client.util.Lists; +import com.google.common.collect.Lists; import org.slf4j.Logger; import org.slf4j.LoggerFactory; From 67efb17eec942f215f8bcd311e57520ca30dd7b3 Mon Sep 17 00:00:00 2001 From: Dan Halperin Date: Tue, 23 Aug 2016 19:32:34 -0700 Subject: [PATCH 051/346] JUnit tests: add @RunWith annotation --- .../apache/beam/runners/direct/AggregatorContainerTest.java | 5 +++++ .../apache/beam/runners/direct/DoFnLifecycleManagerTest.java | 3 +++ .../beam/runners/direct/WriteWithShardingFactoryTest.java | 3 +++ .../java/org/apache/beam/sdk/testing/WindowSupplierTest.java | 3 +++ .../apache/beam/sdk/transforms/reflect/DoFnInvokersTest.java | 3 +++ .../test/java/org/apache/beam/sdk/util/PubsubClientTest.java | 3 +++ .../java/org/apache/beam/sdk/util/PubsubGrpcClientTest.java | 4 ++++ .../java/org/apache/beam/sdk/util/PubsubJsonClientTest.java | 3 +++ .../java/org/apache/beam/sdk/util/PubsubTestClientTest.java | 3 +++ 9 files changed, 30 insertions(+) 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 035a1b03be070..b131b4c524c25 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,12 +24,16 @@ 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; import org.junit.rules.ExpectedException; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; import org.mockito.Mock; import org.mockito.MockitoAnnotations; + import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.TimeUnit; @@ -37,6 +41,7 @@ /** * Tests for {@link AggregatorContainer}. */ +@RunWith(JUnit4.class) public class AggregatorContainerTest { @Rule diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DoFnLifecycleManagerTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DoFnLifecycleManagerTest.java index 77b32968f8b4f..9da46f265b7da 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DoFnLifecycleManagerTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DoFnLifecycleManagerTest.java @@ -29,6 +29,8 @@ import org.hamcrest.Matchers; import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; import java.util.ArrayList; import java.util.List; @@ -42,6 +44,7 @@ /** * Tests for {@link DoFnLifecycleManager}. */ +@RunWith(JUnit4.class) public class DoFnLifecycleManagerTest { private TestFn fn = new TestFn(); private DoFnLifecycleManager mgr = DoFnLifecycleManager.of(fn); 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 a53bc64ecd7fb..20b2776890e91 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 @@ -53,6 +53,8 @@ import org.junit.Rule; import org.junit.Test; import org.junit.rules.TemporaryFolder; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; import java.io.File; import java.io.FileReader; @@ -68,6 +70,7 @@ /** * Tests for {@link WriteWithShardingFactory}. */ +@RunWith(JUnit4.class) public class WriteWithShardingFactoryTest { public static final int INPUT_SIZE = 10000; @Rule public TemporaryFolder tmp = new TemporaryFolder(); diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/WindowSupplierTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/WindowSupplierTest.java index 178c67ca7956b..1a2fd1d560416 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/WindowSupplierTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/WindowSupplierTest.java @@ -32,6 +32,8 @@ import org.junit.Rule; import org.junit.Test; import org.junit.rules.ExpectedException; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; import java.io.IOException; import java.io.InputStream; @@ -41,6 +43,7 @@ /** * Tests for {@link WindowSupplier}. */ +@RunWith(JUnit4.class) public class WindowSupplierTest { private final IntervalWindow window = new IntervalWindow(new Instant(0L), new Instant(100L)); private final IntervalWindow otherWindow = 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 7e756e245f917..28f0143d69c14 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 @@ -29,10 +29,13 @@ 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 DoFnInvokers}. */ +@RunWith(JUnit4.class) public class DoFnInvokersTest { /** A convenience struct holding flags that indicate whether a particular method was invoked. */ public static class Invocations { diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/PubsubClientTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/PubsubClientTest.java index 22508572e5a0c..07031240d16ce 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/PubsubClientTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/PubsubClientTest.java @@ -30,12 +30,15 @@ import org.junit.Rule; import org.junit.Test; import org.junit.rules.ExpectedException; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; import java.util.Map; /** * Tests for helper classes and methods in PubsubClient. */ +@RunWith(JUnit4.class) public class PubsubClientTest { @Rule public ExpectedException thrown = ExpectedException.none(); diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/PubsubGrpcClientTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/PubsubGrpcClientTest.java index 71ee27c86aae4..4d0ec29ca2822 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/PubsubGrpcClientTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/PubsubGrpcClientTest.java @@ -40,9 +40,12 @@ import com.google.pubsub.v1.SubscriberGrpc; import io.grpc.ManagedChannel; + import org.junit.After; import org.junit.Before; import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; import org.mockito.Mockito; import java.io.IOException; @@ -51,6 +54,7 @@ /** * Tests for PubsubGrpcClient. */ +@RunWith(JUnit4.class) public class PubsubGrpcClientTest { private ManagedChannel mockChannel; private GoogleCredentials mockCredentials; diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/PubsubJsonClientTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/PubsubJsonClientTest.java index dfdc46ecea680..21689358197d9 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/PubsubJsonClientTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/PubsubJsonClientTest.java @@ -38,6 +38,8 @@ import org.junit.After; import org.junit.Before; import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; import org.mockito.Mockito; import java.io.IOException; @@ -46,6 +48,7 @@ /** * Tests for PubsubJsonClient. */ +@RunWith(JUnit4.class) public class PubsubJsonClientTest { private Pubsub mockPubsub; private PubsubClient client; diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/PubsubTestClientTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/PubsubTestClientTest.java index d788f1070cecc..d86f5e7ca89d8 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/PubsubTestClientTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/PubsubTestClientTest.java @@ -32,6 +32,8 @@ import com.google.common.collect.Sets; import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; import java.io.IOException; import java.util.List; @@ -40,6 +42,7 @@ /** * Tests for PubsubTestClient. */ +@RunWith(JUnit4.class) public class PubsubTestClientTest { private static final TopicPath TOPIC = PubsubClient.topicPathFromName("testProject", "testTopic"); private static final SubscriptionPath SUBSCRIPTION = From 438d8bdd617af54aa3ca40c4bd86d5c70ca0d55e Mon Sep 17 00:00:00 2001 From: Thomas Groh Date: Tue, 23 Aug 2016 14:44:53 -0700 Subject: [PATCH 052/346] Remove ParDoTest Suppression in Google Cloud Dataflow This reenables the lifecycle tests now that they are properly supported. Update the container image. --- runners/google-cloud-dataflow-java/pom.xml | 11 ----------- .../apache/beam/runners/dataflow/DataflowRunner.java | 4 ++-- 2 files changed, 2 insertions(+), 13 deletions(-) diff --git a/runners/google-cloud-dataflow-java/pom.xml b/runners/google-cloud-dataflow-java/pom.xml index 0044823068c04..bf66f388c9440 100644 --- a/runners/google-cloud-dataflow-java/pom.xml +++ b/runners/google-cloud-dataflow-java/pom.xml @@ -60,17 +60,6 @@ true - - - runnable-on-service-tests - - - org/apache/beam/sdk/transforms/ParDoLifecycleTest.java - org/apache/beam/sdk/transforms/ParDoTest.java - - - - - - + From 00441f8994cb9b204df82064ab737af7ecaa151d Mon Sep 17 00:00:00 2001 From: bchambers Date: Wed, 24 Aug 2016 13:01:03 -0700 Subject: [PATCH 066/346] Optimize imports --- .../beam/examples/DebuggingWordCount.java | 8 +- .../beam/examples/WindowedWordCount.java | 17 ++- .../org/apache/beam/examples/WordCount.java | 8 +- .../common/ExampleBigQueryTableOptions.java | 3 +- .../beam/examples/common/ExampleOptions.java | 7 +- .../beam/examples/common/ExampleUtils.java | 14 +-- .../examples/common/PubsubFileInjector.java | 14 +-- .../beam/examples/complete/AutoComplete.java | 33 +++--- .../complete/StreamingWordExtract.java | 12 +- .../apache/beam/examples/complete/TfIdf.java | 14 +-- .../complete/TopWikipediaSessions.java | 7 +- .../examples/complete/TrafficMaxLaneFlow.java | 19 ++- .../beam/examples/complete/TrafficRoutes.java | 29 ++--- .../examples/cookbook/BigQueryTornadoes.java | 12 +- .../cookbook/CombinePerKeyExamples.java | 12 +- .../examples/cookbook/DatastoreWordCount.java | 18 ++- .../examples/cookbook/FilterExamples.java | 14 +-- .../beam/examples/cookbook/JoinExamples.java | 3 +- .../examples/cookbook/MaxPerKeyExamples.java | 12 +- .../examples/cookbook/TriggerExample.java | 17 ++- .../beam/examples/DebuggingWordCountTest.java | 6 +- .../org/apache/beam/examples/WordCountIT.java | 4 +- .../apache/beam/examples/WordCountTest.java | 6 +- .../examples/complete/AutoCompleteTest.java | 12 +- .../beam/examples/complete/TfIdfTest.java | 6 +- .../complete/TopWikipediaSessionsTest.java | 7 +- .../cookbook/BigQueryTornadoesIT.java | 1 - .../cookbook/BigQueryTornadoesTest.java | 7 +- .../cookbook/CombinePerKeyExamplesTest.java | 7 +- .../examples/cookbook/DeDupExampleTest.java | 6 +- .../examples/cookbook/FilterExamplesTest.java | 9 +- .../examples/cookbook/JoinExamplesTest.java | 9 +- .../cookbook/MaxPerKeyExamplesTest.java | 9 +- .../examples/cookbook/TriggerExampleTest.java | 17 ++- .../beam/examples/MinimalWordCountJava8.java | 3 +- .../examples/complete/game/GameStats.java | 8 +- .../complete/game/HourlyTeamScore.java | 8 +- .../examples/complete/game/LeaderBoard.java | 8 +- .../examples/complete/game/UserScore.java | 8 +- .../complete/game/injector/Injector.java | 8 +- .../complete/game/injector/InjectorUtils.java | 1 - .../injector/RetryHttpInitializerWrapper.java | 1 - .../complete/game/utils/WriteToBigQuery.java | 18 ++- .../game/utils/WriteWindowedToBigQuery.java | 6 +- .../examples/MinimalWordCountJava8Test.java | 21 ++-- .../examples/complete/game/GameStatsTest.java | 8 +- .../complete/game/HourlyTeamScoreTest.java | 8 +- .../examples/complete/game/UserScoreTest.java | 8 +- .../beam/runners/core/SideInputHandler.java | 17 ++- .../core/UnboundedReadFromBoundedSource.java | 37 +++--- .../beam/sdk/util/AssignWindowsDoFn.java | 7 +- .../beam/sdk/util/BatchTimerInternals.java | 8 +- .../apache/beam/sdk/util/DoFnRunnerBase.java | 22 ++-- .../org/apache/beam/sdk/util/DoFnRunners.java | 3 +- ...GroupAlsoByWindowsViaOutputBufferDoFn.java | 7 +- .../sdk/util/GroupByKeyViaGroupByKeyOnly.java | 9 +- .../sdk/util/LateDataDroppingDoFnRunner.java | 10 +- .../apache/beam/sdk/util/PaneInfoTracker.java | 6 +- .../sdk/util/PushbackSideInputDoFnRunner.java | 6 +- .../org/apache/beam/sdk/util/ReduceFn.java | 4 +- .../beam/sdk/util/ReduceFnContextFactory.java | 12 +- .../apache/beam/sdk/util/ReduceFnRunner.java | 26 ++-- .../beam/sdk/util/SimpleDoFnRunner.java | 2 +- .../apache/beam/sdk/util/SystemReduceFn.java | 3 +- .../apache/beam/sdk/util/TriggerRunner.java | 16 +-- .../apache/beam/sdk/util/WatermarkHold.java | 12 +- .../runners/core/SideInputHandlerTest.java | 4 +- .../UnboundedReadFromBoundedSourceTest.java | 27 ++--- .../sdk/util/BatchTimerInternalsTest.java | 1 - .../util/GroupAlsoByWindowsProperties.java | 19 ++- ...pAlsoByWindowsViaOutputBufferDoFnTest.java | 1 - .../util/LateDataDroppingDoFnRunnerTest.java | 9 +- .../util/PushbackSideInputDoFnRunnerTest.java | 9 +- .../beam/sdk/util/ReduceFnRunnerTest.java | 13 +- .../apache/beam/sdk/util/ReduceFnTester.java | 45 ++++--- .../beam/sdk/util/SimpleDoFnRunnerTest.java | 6 +- .../runners/direct/AggregatorContainer.java | 8 +- .../direct/BoundedReadEvaluatorFactory.java | 14 +-- .../beam/runners/direct/CommittedResult.java | 9 +- .../ConsumerTrackingPipelineVisitor.java | 13 +- .../beam/runners/direct/DirectRegistrar.java | 5 +- .../beam/runners/direct/DirectRunner.java | 25 ++-- .../runners/direct/DirectTimerInternals.java | 4 +- .../runners/direct/DoFnLifecycleManager.java | 15 +-- ...ycleManagerRemovingTransformEvaluator.java | 1 - .../runners/direct/EvaluationContext.java | 26 ++-- .../ExecutorServiceParallelExecutor.java | 34 +++--- .../GroupAlsoByWindowEvaluatorFactory.java | 6 +- .../GroupByKeyOnlyEvaluatorFactory.java | 13 +- .../ImmutabilityCheckingBundleFactory.java | 6 +- .../ImmutabilityEnforcementFactory.java | 5 +- .../direct/ImmutableListBundleFactory.java | 6 +- .../direct/KeyedPValueTrackingVisitor.java | 5 +- .../beam/runners/direct/NanosOffsetClock.java | 3 +- .../beam/runners/direct/ParDoEvaluator.java | 14 +-- .../direct/ParDoMultiEvaluatorFactory.java | 11 +- .../direct/ParDoSingleEvaluatorFactory.java | 13 +- .../beam/runners/direct/PipelineExecutor.java | 3 +- .../runners/direct/SideInputContainer.java | 19 ++- .../beam/runners/direct/StepAndKey.java | 4 +- .../runners/direct/StepTransformResult.java | 16 +-- .../direct/TestStreamEvaluatorFactory.java | 14 +-- .../direct/TransformEvaluatorFactory.java | 3 +- .../direct/TransformEvaluatorRegistry.java | 16 +-- .../runners/direct/TransformExecutor.java | 8 +- .../direct/TransformExecutorServices.java | 1 - .../beam/runners/direct/TransformResult.java | 7 +- .../direct/UnboundedReadDeduplicator.java | 11 +- .../direct/UnboundedReadEvaluatorFactory.java | 16 +-- .../UncommittedBundleOutputManager.java | 3 +- .../runners/direct/ViewEvaluatorFactory.java | 5 +- .../direct/WatermarkCallbackExecutor.java | 11 +- .../beam/runners/direct/WatermarkManager.java | 28 ++--- .../direct/WindowEvaluatorFactory.java | 10 +- .../direct/WriteWithShardingFactory.java | 7 +- .../direct/AggregatorContainerTest.java | 8 +- .../BoundedReadEvaluatorFactoryTest.java | 13 +- .../runners/direct/CommittedResultTest.java | 13 +- .../ConsumerTrackingPipelineVisitorTest.java | 6 +- .../runners/direct/DirectRegistrarTest.java | 9 +- .../beam/runners/direct/DirectRunnerTest.java | 14 +-- .../direct/DirectTimerInternalsTest.java | 1 - ...ManagerRemovingTransformEvaluatorTest.java | 6 +- .../direct/DoFnLifecycleManagerTest.java | 12 +- .../direct/DoFnLifecycleManagersTest.java | 9 +- .../EncodabilityEnforcementFactoryTest.java | 10 +- .../runners/direct/EvaluationContextTest.java | 17 ++- .../direct/FlattenEvaluatorFactoryTest.java | 1 - .../direct/ForwardingPTransformTest.java | 1 - .../GroupByKeyEvaluatorFactoryTest.java | 8 +- .../GroupByKeyOnlyEvaluatorFactoryTest.java | 8 +- ...ImmutabilityCheckingBundleFactoryTest.java | 1 - .../ImmutabilityEnforcementFactoryTest.java | 6 +- .../ImmutableListBundleFactoryTest.java | 11 +- .../KeyedPValueTrackingVisitorTest.java | 9 +- .../runners/direct/ParDoEvaluatorTest.java | 14 +-- .../ParDoMultiEvaluatorFactoryTest.java | 2 +- .../ParDoSingleEvaluatorFactoryTest.java | 2 +- .../direct/SideInputContainerTest.java | 17 ++- .../direct/StepTransformResultTest.java | 1 - .../runners/direct/StructuralKeyTest.java | 1 - .../direct/TransformExecutorServicesTest.java | 4 +- .../runners/direct/TransformExecutorTest.java | 23 ++-- .../direct/UnboundedReadDeduplicatorTest.java | 12 +- .../UnboundedReadEvaluatorFactoryTest.java | 26 ++-- .../direct/ViewEvaluatorFactoryTest.java | 4 +- .../direct/WatermarkCallbackExecutorTest.java | 8 +- .../runners/direct/WatermarkManagerTest.java | 21 ++-- .../direct/WindowEvaluatorFactoryTest.java | 14 +-- .../direct/WriteWithShardingFactoryTest.java | 27 ++--- .../beam/runners/flink/examples/TFIDF.java | 14 +-- .../runners/flink/examples/WordCount.java | 1 - .../examples/streaming/AutoComplete.java | 6 +- .../examples/streaming/JoinExamples.java | 1 - .../examples/streaming/KafkaIOExamples.java | 12 +- .../KafkaWindowedWordCountExample.java | 4 +- .../examples/streaming/WindowedWordCount.java | 4 +- .../flink/DefaultParallelismFactory.java | 1 - .../FlinkPipelineExecutionEnvironment.java | 4 +- .../runners/flink/FlinkPipelineOptions.java | 6 +- .../beam/runners/flink/FlinkRunner.java | 23 ++-- .../runners/flink/FlinkRunnerRegistrar.java | 5 +- .../beam/runners/flink/FlinkRunnerResult.java | 10 +- .../beam/runners/flink/TestFlinkRunner.java | 2 - .../FlinkBatchPipelineTranslator.java | 1 - .../FlinkBatchTransformTranslators.java | 19 ++- .../FlinkBatchTranslationContext.java | 6 +- .../FlinkStreamingPipelineTranslator.java | 1 - .../FlinkStreamingTransformTranslators.java | 25 ++-- .../FlinkStreamingTranslationContext.java | 6 +- .../functions/FlinkAssignContext.java | 6 +- .../functions/FlinkAssignWindows.java | 4 +- .../functions/FlinkDoFnFunction.java | 4 +- .../FlinkMergingNonShuffleReduceFunction.java | 17 ++- .../FlinkMergingPartialReduceFunction.java | 17 ++- .../functions/FlinkMergingReduceFunction.java | 21 ++-- .../FlinkMultiOutputDoFnFunction.java | 4 +- .../FlinkMultiOutputProcessContext.java | 6 +- .../FlinkMultiOutputPruningFunction.java | 1 - .../FlinkNoElementAssignContext.java | 1 - .../functions/FlinkPartialReduceFunction.java | 17 ++- .../functions/FlinkProcessContext.java | 15 +-- .../functions/FlinkReduceFunction.java | 21 ++-- .../functions/SideInputInitializer.java | 10 +- .../types/CoderTypeInformation.java | 1 - .../types/CoderTypeSerializer.java | 8 +- .../types/EncodedValueComparator.java | 6 +- .../types/EncodedValueSerializer.java | 4 +- .../types/EncodedValueTypeInformation.java | 4 - .../flink/translation/types/FlinkCoder.java | 11 +- .../translation/types/KvKeySelector.java | 1 - .../utils/SerializedPipelineOptions.java | 4 +- .../wrappers/DataInputViewWrapper.java | 3 +- .../wrappers/DataOutputViewWrapper.java | 3 +- .../SerializableFnAggregatorWrapper.java | 9 +- .../wrappers/SourceInputFormat.java | 6 +- .../wrappers/SourceInputSplit.java | 1 - .../wrappers/streaming/DoFnOperator.java | 22 ++-- .../streaming/FlinkStateInternals.java | 13 +- .../streaming/SingletonKeyedWorkItem.java | 3 +- .../SingletonKeyedWorkItemCoder.java | 17 ++- .../streaming/WindowDoFnOperator.java | 34 +++--- .../streaming/WorkItemKeySelector.java | 5 +- .../streaming/io/BoundedSourceWrapper.java | 9 +- .../streaming/io/UnboundedFlinkSink.java | 14 +-- .../streaming/io/UnboundedFlinkSource.java | 7 +- .../streaming/io/UnboundedSocketSource.java | 17 ++- .../streaming/io/UnboundedSourceWrapper.java | 15 +-- .../flink/EncodedValueComparatorTest.java | 1 - .../flink/FlinkRunnerRegistrarTest.java | 1 - .../runners/flink/PipelineOptionsTest.java | 6 +- .../beam/runners/flink/ReadSourceITCase.java | 9 +- .../flink/ReadSourceStreamingITCase.java | 4 +- .../beam/runners/flink/WriteSinkITCase.java | 13 +- .../flink/streaming/DoFnOperatorTest.java | 11 +- .../streaming/FlinkStateInternalsTest.java | 6 +- .../flink/streaming/GroupByNullKeyTest.java | 11 +- .../flink/streaming/TestCountingSource.java | 12 +- .../streaming/TopWikipediaSessionsITCase.java | 11 +- .../streaming/UnboundedSourceWrapperTest.java | 8 +- .../dataflow/BlockingDataflowRunner.java | 7 +- .../dataflow/DataflowJobException.java | 1 - .../runners/dataflow/DataflowPipelineJob.java | 30 ++--- .../dataflow/DataflowPipelineRegistrar.java | 5 +- .../dataflow/DataflowPipelineTranslator.java | 48 ++++---- .../beam/runners/dataflow/DataflowRunner.java | 107 ++++++++--------- .../runners/dataflow/DataflowRunnerHooks.java | 3 +- .../dataflow/internal/CustomSources.java | 18 ++- .../DataflowAggregatorTransforms.java | 10 +- .../DataflowMetricUpdateExtractor.java | 10 +- ...ataflowUnboundedReadFromBoundedSource.java | 37 +++--- .../runners/dataflow/internal/IsmFormat.java | 35 +++--- .../dataflow/internal/ReadTranslator.java | 8 +- .../options/CloudDebuggerOptions.java | 6 +- .../options/DataflowPipelineDebugOptions.java | 11 +- .../options/DataflowPipelineOptions.java | 7 +- .../DataflowPipelineWorkerPoolOptions.java | 9 +- .../options/DataflowProfilingOptions.java | 3 +- .../options/DataflowWorkerLoggingOptions.java | 8 +- .../dataflow/testing/TestDataflowRunner.java | 27 ++--- .../dataflow/util/DataflowTransport.java | 6 +- .../beam/runners/dataflow/util/DoFnInfo.java | 3 +- .../beam/runners/dataflow/util/GcsStager.java | 6 +- .../runners/dataflow/util/MonitoringUtil.java | 14 +-- .../runners/dataflow/util/PackageUtil.java | 18 ++- .../dataflow/util/RandomAccessData.java | 16 +-- .../beam/runners/dataflow/util/Stager.java | 1 - .../beam/runners/dataflow/util/TimeUtil.java | 8 +- .../dataflow/BlockingDataflowRunnerTest.java | 1 - .../dataflow/DataflowPipelineJobTest.java | 33 +++--- .../DataflowPipelineRegistrarTest.java | 9 +- .../DataflowPipelineTranslatorTest.java | 38 +++--- .../runners/dataflow/DataflowRunnerTest.java | 48 ++++---- .../dataflow/RecordingPipelineVisitor.java | 5 +- .../DataflowPipelineDebugOptionsTest.java | 1 - .../options/DataflowPipelineOptionsTest.java | 1 - .../options/DataflowProfilingOptionsTest.java | 4 +- .../DataflowWorkerLoggingOptionsTest.java | 8 +- .../testing/TestDataflowRunnerTest.java | 35 +++--- .../transforms/DataflowGroupByKeyTest.java | 6 +- .../dataflow/transforms/DataflowViewTest.java | 1 - .../dataflow/util/MonitoringUtilTest.java | 17 ++- .../dataflow/util/PackageUtilTest.java | 41 +++---- .../dataflow/util/RandomAccessDataTest.java | 13 +- .../runners/dataflow/util/TimeUtilTest.java | 1 - .../beam/runners/spark/SparkRunner.java | 1 - .../runners/spark/SparkRunnerRegistrar.java | 5 +- .../spark/aggregators/NamedAggregators.java | 10 +- .../runners/spark/coders/CoderHelpers.java | 12 +- .../spark/coders/NullWritableCoder.java | 6 +- .../runners/spark/coders/WritableCoder.java | 14 +-- .../apache/beam/runners/spark/io/KafkaIO.java | 8 +- .../runners/spark/io/hadoop/HadoopIO.java | 6 +- .../spark/io/hadoop/ShardNameBuilder.java | 3 +- .../io/hadoop/ShardNameTemplateHelper.java | 3 +- .../hadoop/TemplatedAvroKeyOutputFormat.java | 5 +- .../TemplatedSequenceFileOutputFormat.java | 3 +- .../io/hadoop/TemplatedTextOutputFormat.java | 3 +- .../spark/translation/DoFnFunction.java | 10 +- .../spark/translation/EvaluationContext.java | 19 ++- .../spark/translation/MultiDoFnFunction.java | 16 +-- .../translation/SparkProcessContext.java | 15 +-- .../translation/SparkRuntimeContext.java | 19 ++- .../spark/translation/TransformEvaluator.java | 3 +- .../translation/TransformTranslator.java | 30 ++--- .../streaming/StreamingEvaluationContext.java | 14 +-- .../StreamingTransformTranslator.java | 26 ++-- .../runners/spark/util/BroadcastHelper.java | 8 +- .../beam/runners/spark/util/ByteArray.java | 1 - .../apache/beam/runners/spark/DeDupTest.java | 11 +- .../beam/runners/spark/EmptyInputTest.java | 9 +- .../runners/spark/SimpleWordCountTest.java | 15 +-- .../spark/SparkRunnerRegistrarTest.java | 9 +- .../apache/beam/runners/spark/TfIdfTest.java | 6 +- .../spark/coders/WritableCoderTest.java | 1 - .../runners/spark/io/AvroPipelineTest.java | 25 ++-- .../beam/runners/spark/io/NumShardsTest.java | 21 ++-- .../hadoop/HadoopFileFormatPipelineTest.java | 6 +- .../spark/io/hadoop/ShardNameBuilderTest.java | 1 - .../translation/CombineGloballyTest.java | 9 +- .../spark/translation/CombinePerKeyTest.java | 11 +- .../spark/translation/DoFnOutputTest.java | 4 +- .../translation/MultiOutputWordCountTest.java | 9 +- .../spark/translation/SerializationTest.java | 23 ++-- .../spark/translation/SideEffectsTest.java | 6 +- .../translation/SparkPipelineOptionsTest.java | 1 - .../translation/TransformTranslatorTest.java | 21 ++-- .../translation/WindowedWordCountTest.java | 9 +- .../streaming/FlattenStreamingTest.java | 8 +- .../streaming/KafkaStreamingTest.java | 16 +-- .../SimpleStreamingWordCountTest.java | 10 +- .../streaming/utils/EmbeddedKafkaCluster.java | 12 +- .../streaming/utils/PAssertStreaming.java | 4 +- .../beam/sdk/AggregatorPipelineExtractor.java | 12 +- .../org/apache/beam/sdk/AggregatorValues.java | 5 +- .../java/org/apache/beam/sdk/Pipeline.java | 21 ++-- .../org/apache/beam/sdk/PipelineResult.java | 4 +- .../org/apache/beam/sdk/coders/AvroCoder.java | 35 +++--- .../beam/sdk/coders/BigDecimalCoder.java | 1 - .../sdk/coders/BigEndianIntegerCoder.java | 1 - .../beam/sdk/coders/BigEndianLongCoder.java | 1 - .../beam/sdk/coders/BigIntegerCoder.java | 1 - .../beam/sdk/coders/ByteArrayCoder.java | 11 +- .../org/apache/beam/sdk/coders/ByteCoder.java | 1 - .../beam/sdk/coders/ByteStringCoder.java | 7 +- .../org/apache/beam/sdk/coders/Coder.java | 11 +- .../beam/sdk/coders/CoderFactories.java | 3 +- .../beam/sdk/coders/CoderProviders.java | 6 +- .../apache/beam/sdk/coders/CoderRegistry.java | 24 ++-- .../beam/sdk/coders/CollectionCoder.java | 4 +- .../apache/beam/sdk/coders/CustomCoder.java | 16 +-- .../apache/beam/sdk/coders/DefaultCoder.java | 3 +- .../apache/beam/sdk/coders/DelegateCoder.java | 1 - .../apache/beam/sdk/coders/DoubleCoder.java | 1 - .../apache/beam/sdk/coders/DurationCoder.java | 8 +- .../apache/beam/sdk/coders/InstantCoder.java | 12 +- .../apache/beam/sdk/coders/IterableCoder.java | 9 +- .../beam/sdk/coders/IterableLikeCoder.java | 9 +- .../org/apache/beam/sdk/coders/JAXBCoder.java | 12 +- .../org/apache/beam/sdk/coders/KvCoder.java | 13 +- .../org/apache/beam/sdk/coders/ListCoder.java | 4 +- .../org/apache/beam/sdk/coders/MapCoder.java | 9 +- .../apache/beam/sdk/coders/NullableCoder.java | 12 +- .../beam/sdk/coders/SerializableCoder.java | 6 +- .../org/apache/beam/sdk/coders/SetCoder.java | 4 +- .../apache/beam/sdk/coders/StandardCoder.java | 11 +- .../beam/sdk/coders/StringDelegateCoder.java | 5 +- .../beam/sdk/coders/StringUtf8Coder.java | 11 +- .../beam/sdk/coders/TableRowJsonCoder.java | 4 +- .../beam/sdk/coders/TextualIntegerCoder.java | 1 - .../apache/beam/sdk/coders/VarIntCoder.java | 4 +- .../apache/beam/sdk/coders/VarLongCoder.java | 4 +- .../org/apache/beam/sdk/coders/VoidCoder.java | 1 - .../beam/sdk/coders/protobuf/ProtoCoder.java | 28 ++--- .../sdk/coders/protobuf/ProtobufUtil.java | 4 +- .../java/org/apache/beam/sdk/io/AvroIO.java | 24 ++-- .../org/apache/beam/sdk/io/AvroSource.java | 45 ++++--- .../apache/beam/sdk/io/BlockBasedSource.java | 6 +- .../io/BoundedReadFromUnboundedSource.java | 17 ++- .../org/apache/beam/sdk/io/BoundedSource.java | 11 +- .../apache/beam/sdk/io/CompressedSource.java | 16 +-- .../org/apache/beam/sdk/io/CountingInput.java | 4 +- .../apache/beam/sdk/io/CountingSource.java | 11 +- .../org/apache/beam/sdk/io/FileBasedSink.java | 29 ++--- .../apache/beam/sdk/io/FileBasedSource.java | 17 ++- .../apache/beam/sdk/io/OffsetBasedSource.java | 10 +- .../java/org/apache/beam/sdk/io/PubsubIO.java | 20 ++-- .../beam/sdk/io/PubsubUnboundedSink.java | 23 ++-- .../beam/sdk/io/PubsubUnboundedSource.java | 47 ++++---- .../java/org/apache/beam/sdk/io/Read.java | 4 +- .../java/org/apache/beam/sdk/io/Sink.java | 3 +- .../java/org/apache/beam/sdk/io/Source.java | 8 +- .../java/org/apache/beam/sdk/io/TextIO.java | 29 ++--- .../apache/beam/sdk/io/UnboundedSource.java | 11 +- .../java/org/apache/beam/sdk/io/Write.java | 11 +- .../java/org/apache/beam/sdk/io/XmlSink.java | 14 +-- .../org/apache/beam/sdk/io/XmlSource.java | 15 +-- .../org/apache/beam/sdk/io/range/ByteKey.java | 1 - .../beam/sdk/io/range/ByteKeyRange.java | 6 +- .../sdk/io/range/ByteKeyRangeTracker.java | 4 +- .../beam/sdk/io/range/OffsetRangeTracker.java | 4 +- .../apache/beam/sdk/options/GcpOptions.java | 18 ++- .../apache/beam/sdk/options/GcsOptions.java | 15 +-- .../sdk/options/GoogleApiDebugOptions.java | 1 - .../beam/sdk/options/PipelineOptionSpec.java | 4 +- .../beam/sdk/options/PipelineOptions.java | 19 ++- .../sdk/options/PipelineOptionsFactory.java | 26 ++-- .../sdk/options/PipelineOptionsReflector.java | 4 +- .../sdk/options/PipelineOptionsRegistrar.java | 1 - .../sdk/options/PipelineOptionsValidator.java | 6 +- .../sdk/options/ProxyInvocationHandler.java | 38 +++--- .../sdk/runners/PipelineRunnerRegistrar.java | 1 - .../beam/sdk/runners/TransformHierarchy.java | 9 +- .../beam/sdk/runners/TransformTreeNode.java | 14 +-- .../beam/sdk/testing/CoderProperties.java | 23 ++-- .../beam/sdk/testing/FileChecksumMatcher.java | 17 ++- .../beam/sdk/testing/MatcherDeserializer.java | 7 +- .../beam/sdk/testing/MatcherSerializer.java | 7 +- .../org/apache/beam/sdk/testing/PAssert.java | 24 ++-- .../beam/sdk/testing/PaneExtractors.java | 5 +- .../beam/sdk/testing/SerializableMatcher.java | 3 +- .../sdk/testing/SerializableMatchers.java | 18 ++- .../beam/sdk/testing/SourceTestUtils.java | 27 ++--- .../beam/sdk/testing/StaticWindows.java | 10 +- .../apache/beam/sdk/testing/TestPipeline.java | 32 +++-- .../beam/sdk/testing/TestPipelineOptions.java | 1 - .../apache/beam/sdk/testing/TestStream.java | 24 ++-- .../beam/sdk/testing/WindowFnTestUtils.java | 19 ++- .../beam/sdk/testing/WindowSupplier.java | 10 +- .../sdk/transforms/AppliedPTransform.java | 5 +- .../sdk/transforms/ApproximateQuantiles.java | 29 ++--- .../sdk/transforms/ApproximateUnique.java | 23 ++-- .../apache/beam/sdk/transforms/Combine.java | 27 ++--- .../beam/sdk/transforms/CombineFnBase.java | 10 +- .../beam/sdk/transforms/CombineFns.java | 39 +++--- .../org/apache/beam/sdk/transforms/Count.java | 13 +- .../apache/beam/sdk/transforms/Create.java | 32 +++-- .../org/apache/beam/sdk/transforms/DoFn.java | 18 ++- .../beam/sdk/transforms/DoFnAdapters.java | 4 +- .../beam/sdk/transforms/DoFnTester.java | 26 ++-- .../beam/sdk/transforms/FlatMapElements.java | 3 +- .../IntraBundleParallelization.java | 11 +- .../org/apache/beam/sdk/transforms/Max.java | 5 +- .../org/apache/beam/sdk/transforms/Mean.java | 12 +- .../org/apache/beam/sdk/transforms/Min.java | 5 +- .../apache/beam/sdk/transforms/OldDoFn.java | 19 ++- .../beam/sdk/transforms/PTransform.java | 7 +- .../org/apache/beam/sdk/transforms/ParDo.java | 10 +- .../apache/beam/sdk/transforms/Partition.java | 3 +- .../apache/beam/sdk/transforms/Sample.java | 7 +- .../org/apache/beam/sdk/transforms/Top.java | 20 ++-- .../org/apache/beam/sdk/transforms/View.java | 5 +- .../apache/beam/sdk/transforms/ViewFn.java | 3 +- .../beam/sdk/transforms/WithTimestamps.java | 1 - .../sdk/transforms/display/DisplayData.java | 21 ++-- .../beam/sdk/transforms/join/CoGbkResult.java | 30 ++--- .../transforms/join/CoGbkResultSchema.java | 10 +- .../sdk/transforms/join/CoGroupByKey.java | 5 +- .../join/KeyedPCollectionTuple.java | 7 +- .../beam/sdk/transforms/join/UnionCoder.java | 14 +-- .../sdk/transforms/reflect/DoFnInvokers.java | 33 +++--- .../sdk/transforms/reflect/DoFnSignature.java | 5 +- .../transforms/reflect/DoFnSignatures.java | 8 +- .../sdk/transforms/windowing/AfterAll.java | 9 +- .../windowing/AfterDelayFromFirstElement.java | 16 +-- .../sdk/transforms/windowing/AfterEach.java | 9 +- .../sdk/transforms/windowing/AfterFirst.java | 9 +- .../sdk/transforms/windowing/AfterPane.java | 9 +- .../windowing/AfterProcessingTime.java | 9 +- .../AfterSynchronizedProcessingTime.java | 10 +- .../transforms/windowing/AfterWatermark.java | 12 +- .../transforms/windowing/BoundedWindow.java | 3 +- .../transforms/windowing/CalendarWindows.java | 1 - .../transforms/windowing/DefaultTrigger.java | 4 +- .../transforms/windowing/FixedWindows.java | 4 +- .../transforms/windowing/GlobalWindow.java | 6 +- .../transforms/windowing/GlobalWindows.java | 6 +- .../transforms/windowing/IntervalWindow.java | 10 +- .../transforms/windowing/InvalidWindows.java | 4 +- .../beam/sdk/transforms/windowing/Never.java | 4 +- .../windowing/OrFinallyTrigger.java | 7 +- .../transforms/windowing/OutputTimeFn.java | 7 +- .../transforms/windowing/OutputTimeFns.java | 7 +- .../sdk/transforms/windowing/PaneInfo.java | 14 +-- .../windowing/PartitioningWindowFn.java | 3 +- .../sdk/transforms/windowing/Repeatedly.java | 6 +- .../sdk/transforms/windowing/Sessions.java | 8 +- .../transforms/windowing/SlidingWindows.java | 10 +- .../sdk/transforms/windowing/Trigger.java | 16 +-- .../beam/sdk/transforms/windowing/Window.java | 4 +- .../sdk/transforms/windowing/WindowFn.java | 6 +- .../apache/beam/sdk/util/ActiveWindowSet.java | 6 +- .../org/apache/beam/sdk/util/ApiSurface.java | 6 +- .../beam/sdk/util/AppliedCombineFn.java | 6 +- ...temptAndTimeBoundedExponentialBackOff.java | 4 +- .../org/apache/beam/sdk/util/AvroUtils.java | 7 +- .../beam/sdk/util/BaseExecutionContext.java | 9 +- .../org/apache/beam/sdk/util/BitSetCoder.java | 7 +- .../beam/sdk/util/BucketingFunction.java | 3 +- .../BufferedElementCountingOutputStream.java | 4 +- .../apache/beam/sdk/util/CloudKnownType.java | 1 - .../org/apache/beam/sdk/util/CloudObject.java | 2 - .../org/apache/beam/sdk/util/CoderUtils.java | 17 ++- .../apache/beam/sdk/util/CombineFnUtil.java | 7 +- .../beam/sdk/util/CredentialFactory.java | 1 - .../org/apache/beam/sdk/util/Credentials.java | 9 +- .../beam/sdk/util/DirectSideInputReader.java | 5 +- .../beam/sdk/util/ExecutableTrigger.java | 7 +- .../beam/sdk/util/ExecutionContext.java | 5 +- .../sdk/util/ExposedByteArrayInputStream.java | 3 +- .../util/ExposedByteArrayOutputStream.java | 3 +- .../beam/sdk/util/FileIOChannelFactory.java | 6 +- .../beam/sdk/util/FinishedTriggersSet.java | 1 - .../beam/sdk/util/GcpCredentialFactory.java | 6 +- .../beam/sdk/util/GcsIOChannelFactory.java | 5 +- .../beam/sdk/util/GcsPathValidator.java | 3 +- .../org/apache/beam/sdk/util/GcsUtil.java | 16 +-- .../apache/beam/sdk/util/IOChannelUtils.java | 5 +- .../beam/sdk/util/IdentityWindowFn.java | 6 +- .../apache/beam/sdk/util/InstanceBuilder.java | 5 +- .../beam/sdk/util/KeyedWorkItemCoder.java | 17 ++- .../apache/beam/sdk/util/KeyedWorkItems.java | 4 +- .../beam/sdk/util/MergingActiveWindowSet.java | 21 ++-- .../apache/beam/sdk/util/MovingFunction.java | 3 +- .../beam/sdk/util/MutationDetectors.java | 5 +- .../sdk/util/NonMergingActiveWindowSet.java | 6 +- .../beam/sdk/util/NoopCredentialFactory.java | 4 +- .../beam/sdk/util/NullSideInputReader.java | 6 +- .../beam/sdk/util/PCollectionViewWindow.java | 3 +- .../beam/sdk/util/PCollectionViews.java | 25 ++-- .../java/org/apache/beam/sdk/util/PTuple.java | 3 +- .../beam/sdk/util/PerKeyCombineFnRunner.java | 5 +- .../beam/sdk/util/PerKeyCombineFnRunners.java | 6 +- .../apache/beam/sdk/util/PubsubClient.java | 5 +- .../beam/sdk/util/PubsubGrpcClient.java | 8 +- .../beam/sdk/util/PubsubJsonClient.java | 5 +- .../beam/sdk/util/PubsubTestClient.java | 5 +- .../org/apache/beam/sdk/util/ReleaseInfo.java | 6 +- .../org/apache/beam/sdk/util/Reshuffle.java | 1 - .../beam/sdk/util/ReshuffleTrigger.java | 4 +- .../sdk/util/RetryHttpRequestInitializer.java | 7 +- .../beam/sdk/util/SerializableUtils.java | 13 +- .../org/apache/beam/sdk/util/Serializer.java | 2 - .../apache/beam/sdk/util/SideInputReader.java | 3 +- .../org/apache/beam/sdk/util/StringUtils.java | 4 +- .../org/apache/beam/sdk/util/Structs.java | 2 - .../beam/sdk/util/SystemDoFnInternal.java | 3 +- .../apache/beam/sdk/util/TestCredential.java | 1 - .../apache/beam/sdk/util/TimerInternals.java | 26 ++-- .../java/org/apache/beam/sdk/util/Timers.java | 4 +- .../org/apache/beam/sdk/util/Transport.java | 8 +- .../beam/sdk/util/TriggerContextFactory.java | 18 ++- .../beam/sdk/util/UnownedInputStream.java | 1 - .../beam/sdk/util/UnownedOutputStream.java | 1 - .../sdk/util/UploadIdResponseInterceptor.java | 4 +- .../beam/sdk/util/ValueWithRecordId.java | 13 +- .../java/org/apache/beam/sdk/util/Values.java | 1 - .../apache/beam/sdk/util/WindowedValue.java | 32 +++-- .../beam/sdk/util/WindowingInternals.java | 6 +- .../beam/sdk/util/WindowingStrategy.java | 11 +- .../org/apache/beam/sdk/util/ZipFiles.java | 4 +- .../beam/sdk/util/common/ReflectHelpers.java | 3 - .../apache/beam/sdk/util/gcsfs/GcsPath.java | 2 - .../CopyOnAccessInMemoryStateInternals.java | 16 +-- .../util/state/InMemoryStateInternals.java | 13 +- .../sdk/util/state/MergingStateAccessor.java | 3 +- .../beam/sdk/util/state/StateContexts.java | 3 +- .../sdk/util/state/StateInternalsFactory.java | 3 +- .../beam/sdk/util/state/StateMerging.java | 9 +- .../beam/sdk/util/state/StateNamespaces.java | 10 +- .../beam/sdk/util/state/StateTable.java | 4 +- .../apache/beam/sdk/util/state/StateTag.java | 5 +- .../apache/beam/sdk/util/state/StateTags.java | 10 +- .../sdk/util/state/WatermarkHoldState.java | 1 - .../java/org/apache/beam/sdk/values/KV.java | 10 +- .../org/apache/beam/sdk/values/PBegin.java | 5 +- .../beam/sdk/values/PCollectionList.java | 12 +- .../beam/sdk/values/PCollectionTuple.java | 12 +- .../beam/sdk/values/PCollectionView.java | 3 +- .../org/apache/beam/sdk/values/PDone.java | 5 +- .../org/apache/beam/sdk/values/PInput.java | 3 +- .../org/apache/beam/sdk/values/POutput.java | 3 +- .../apache/beam/sdk/values/PValueBase.java | 5 +- .../beam/sdk/values/TimestampedValue.java | 12 +- .../org/apache/beam/sdk/values/TupleTag.java | 13 +- .../apache/beam/sdk/values/TupleTagList.java | 4 +- .../beam/sdk/values/TypeDescriptor.java | 2 - .../beam/sdk/values/TypeDescriptors.java | 1 - .../dataflow/util/GcsPathValidatorTest.java | 1 - .../sdk/AggregatorPipelineExtractorTest.java | 15 +-- .../org/apache/beam/sdk/DataflowMatchers.java | 4 +- .../org/apache/beam/sdk/PipelineTest.java | 4 +- .../java/org/apache/beam/sdk/TestUtils.java | 10 +- .../org/apache/beam/sdk/WindowMatchers.java | 9 +- .../apache/beam/sdk/WindowMatchersTest.java | 4 +- .../apache/beam/sdk/coders/AvroCoderTest.java | 60 +++++----- .../beam/sdk/coders/BigDecimalCoderTest.java | 6 +- .../sdk/coders/BigEndianIntegerCoderTest.java | 6 +- .../sdk/coders/BigEndianLongCoderTest.java | 6 +- .../beam/sdk/coders/BigIntegerCoderTest.java | 6 +- .../beam/sdk/coders/ByteArrayCoderTest.java | 10 +- .../apache/beam/sdk/coders/ByteCoderTest.java | 6 +- .../beam/sdk/coders/ByteStringCoderTest.java | 11 +- .../beam/sdk/coders/CoderFactoriesTest.java | 5 +- .../beam/sdk/coders/CoderProvidersTest.java | 4 +- .../beam/sdk/coders/CoderRegistryTest.java | 31 +++-- .../org/apache/beam/sdk/coders/CoderTest.java | 4 +- .../beam/sdk/coders/CollectionCoderTest.java | 14 +-- .../beam/sdk/coders/CustomCoderTest.java | 12 +- .../beam/sdk/coders/DefaultCoderTest.java | 7 +- .../beam/sdk/coders/DelegateCoderTest.java | 11 +- .../beam/sdk/coders/DoubleCoderTest.java | 6 +- .../beam/sdk/coders/DurationCoderTest.java | 9 +- .../beam/sdk/coders/InstantCoderTest.java | 15 +-- .../beam/sdk/coders/IterableCoderTest.java | 10 +- .../apache/beam/sdk/coders/JAXBCoderTest.java | 16 +-- .../apache/beam/sdk/coders/KvCoderTest.java | 13 +- .../apache/beam/sdk/coders/ListCoderTest.java | 10 +- .../apache/beam/sdk/coders/MapCoderTest.java | 17 ++- .../beam/sdk/coders/NullableCoderTest.java | 13 +- .../beam/sdk/coders/PrintBase64Encodings.java | 4 +- .../sdk/coders/SerializableCoderTest.java | 14 +-- .../apache/beam/sdk/coders/SetCoderTest.java | 12 +- .../beam/sdk/coders/StandardCoderTest.java | 16 +-- .../sdk/coders/StringDelegateCoderTest.java | 8 +- .../beam/sdk/coders/StringUtf8CoderTest.java | 6 +- .../sdk/coders/TableRowJsonCoderTest.java | 9 +- .../sdk/coders/TextualIntegerCoderTest.java | 6 +- .../beam/sdk/coders/VarIntCoderTest.java | 6 +- .../beam/sdk/coders/VarLongCoderTest.java | 6 +- .../sdk/coders/protobuf/ProtoCoderTest.java | 14 +-- .../sdk/coders/protobuf/ProtobufUtilTest.java | 10 +- .../beam/sdk/io/AvroIOGeneratedClassTest.java | 26 ++-- .../org/apache/beam/sdk/io/AvroIOTest.java | 30 ++--- .../apache/beam/sdk/io/AvroSourceTest.java | 46 ++++---- .../BoundedReadFromUnboundedSourceTest.java | 11 +- .../beam/sdk/io/CompressedSourceTest.java | 41 +++---- .../apache/beam/sdk/io/CountingInputTest.java | 2 - .../beam/sdk/io/CountingSourceTest.java | 6 +- .../apache/beam/sdk/io/FileBasedSinkTest.java | 22 ++-- .../beam/sdk/io/FileBasedSourceTest.java | 32 +++-- .../beam/sdk/io/OffsetBasedSourceTest.java | 11 +- .../org/apache/beam/sdk/io/PubsubIOTest.java | 5 +- .../beam/sdk/io/PubsubUnboundedSinkTest.java | 13 +- .../sdk/io/PubsubUnboundedSourceTest.java | 19 ++- .../java/org/apache/beam/sdk/io/ReadTest.java | 12 +- .../org/apache/beam/sdk/io/TextIOTest.java | 47 ++++---- .../org/apache/beam/sdk/io/WriteTest.java | 29 ++--- .../org/apache/beam/sdk/io/XmlSinkTest.java | 27 ++--- .../org/apache/beam/sdk/io/XmlSourceTest.java | 29 ++--- .../ByteKeyRangeEstimateFractionTest.java | 1 - .../range/ByteKeyRangeInterpolateKeyTest.java | 1 - .../beam/sdk/io/range/ByteKeyRangeTest.java | 6 +- .../apache/beam/sdk/io/range/ByteKeyTest.java | 3 +- .../beam/sdk/options/GcpOptionsTest.java | 16 +-- .../options/GoogleApiDebugOptionsTest.java | 8 +- .../options/PipelineOptionsFactoryTest.java | 27 ++--- .../options/PipelineOptionsReflectorTest.java | 6 +- .../beam/sdk/options/PipelineOptionsTest.java | 12 +- .../options/PipelineOptionsValidatorTest.java | 1 - .../options/ProxyInvocationHandlerTest.java | 28 ++--- .../beam/sdk/runners/PipelineRunnerTest.java | 1 - .../beam/sdk/runners/TransformTreeTest.java | 8 +- .../runners/dataflow/TestCountingSource.java | 13 +- .../dataflow/TestCountingSourceTest.java | 4 +- .../beam/sdk/testing/CoderPropertiesTest.java | 11 +- .../beam/sdk/testing/CrashingRunnerTest.java | 1 - .../apache/beam/sdk/testing/ExpectedLogs.java | 12 +- .../beam/sdk/testing/ExpectedLogsTest.java | 13 +- .../sdk/testing/FastNanoClockAndSleeper.java | 1 - .../testing/FastNanoClockAndSleeperTest.java | 3 +- .../sdk/testing/FileChecksumMatcherTest.java | 9 +- .../apache/beam/sdk/testing/PAssertTest.java | 17 ++- .../sdk/testing/PCollectionViewTesting.java | 15 +-- .../beam/sdk/testing/PaneExtractorsTest.java | 4 +- .../sdk/testing/RestoreSystemProperties.java | 5 +- .../sdk/testing/SerializableMatchersTest.java | 12 +- .../beam/sdk/testing/SourceTestUtilsTest.java | 9 +- .../beam/sdk/testing/StaticWindowsTest.java | 4 +- .../sdk/testing/SystemNanoTimeSleeper.java | 1 - .../testing/SystemNanoTimeSleeperTest.java | 1 - .../beam/sdk/testing/TestPipelineTest.java | 12 +- .../beam/sdk/testing/TestStreamTest.java | 4 +- .../beam/sdk/testing/WindowSupplierTest.java | 13 +- .../transforms/ApproximateQuantilesTest.java | 13 +- .../sdk/transforms/ApproximateUniqueTest.java | 18 ++- .../beam/sdk/transforms/CombineFnsTest.java | 18 ++- .../beam/sdk/transforms/CombineTest.java | 38 +++--- .../apache/beam/sdk/transforms/CountTest.java | 7 +- .../beam/sdk/transforms/CreateTest.java | 26 ++-- .../DoFnDelegatingAggregatorTest.java | 1 - .../apache/beam/sdk/transforms/DoFnTest.java | 4 +- .../beam/sdk/transforms/DoFnTesterTest.java | 4 +- .../beam/sdk/transforms/FilterTest.java | 5 +- .../sdk/transforms/FlatMapElementsTest.java | 16 +-- .../beam/sdk/transforms/FlattenTest.java | 17 ++- .../beam/sdk/transforms/GroupByKeyTest.java | 30 ++--- .../IntraBundleParallelizationTest.java | 7 +- .../apache/beam/sdk/transforms/KeysTest.java | 4 +- .../beam/sdk/transforms/KvSwapTest.java | 4 +- .../beam/sdk/transforms/MapElementsTest.java | 7 +- .../apache/beam/sdk/transforms/MaxTest.java | 5 +- .../apache/beam/sdk/transforms/MeanTest.java | 10 +- .../apache/beam/sdk/transforms/MinTest.java | 5 +- .../beam/sdk/transforms/NoOpOldDoFn.java | 1 - .../beam/sdk/transforms/OldDoFnTest.java | 9 +- .../beam/sdk/transforms/PTransformTest.java | 1 - .../sdk/transforms/ParDoLifecycleTest.java | 6 +- .../apache/beam/sdk/transforms/ParDoTest.java | 23 ++-- .../beam/sdk/transforms/PartitionTest.java | 9 +- .../sdk/transforms/RemoveDuplicatesTest.java | 10 +- .../beam/sdk/transforms/SampleTest.java | 21 ++-- .../sdk/transforms/SimpleStatsFnsTest.java | 7 +- .../apache/beam/sdk/transforms/SumTest.java | 5 +- .../apache/beam/sdk/transforms/TopTest.java | 13 +- .../beam/sdk/transforms/ValuesTest.java | 4 +- .../apache/beam/sdk/transforms/ViewTest.java | 31 ++--- .../beam/sdk/transforms/WithKeysTest.java | 6 +- .../sdk/transforms/WithTimestampsTest.java | 4 +- .../display/DisplayDataEvaluator.java | 8 +- .../display/DisplayDataEvaluatorTest.java | 7 +- .../display/DisplayDataMatchers.java | 7 +- .../display/DisplayDataMatchersTest.java | 2 - .../transforms/display/DisplayDataTest.java | 26 ++-- .../transforms/join/CoGbkResultCoderTest.java | 4 +- .../sdk/transforms/join/CoGbkResultTest.java | 6 +- .../sdk/transforms/join/CoGroupByKeyTest.java | 15 +-- .../sdk/transforms/join/UnionCoderTest.java | 4 +- .../transforms/reflect/DoFnInvokersTest.java | 1 - .../reflect/DoFnSignaturesTest.java | 9 +- .../transforms/windowing/AfterAllTest.java | 1 - .../transforms/windowing/AfterEachTest.java | 1 - .../transforms/windowing/AfterFirstTest.java | 1 - .../transforms/windowing/AfterPaneTest.java | 1 - .../windowing/AfterProcessingTimeTest.java | 1 - .../AfterSynchronizedProcessingTimeTest.java | 1 - .../windowing/AfterWatermarkTest.java | 1 - .../windowing/CalendarWindowsTest.java | 13 +- .../windowing/DefaultTriggerTest.java | 1 - .../windowing/FixedWindowsTest.java | 11 +- .../windowing/IntervalWindowTest.java | 7 +- .../sdk/transforms/windowing/NeverTest.java | 1 - .../windowing/OrFinallyTriggerTest.java | 1 - .../transforms/windowing/PaneInfoTest.java | 1 - .../transforms/windowing/RepeatedlyTest.java | 1 - .../transforms/windowing/SessionsTest.java | 16 +-- .../windowing/SlidingWindowsTest.java | 11 +- .../sdk/transforms/windowing/StubTrigger.java | 4 +- .../sdk/transforms/windowing/TriggerTest.java | 5 +- .../sdk/transforms/windowing/WindowTest.java | 5 +- .../transforms/windowing/WindowingTest.java | 10 +- .../apache/beam/sdk/util/ApiSurfaceTest.java | 10 +- ...tAndTimeBoundedExponentialBackOffTest.java | 4 +- .../AttemptBoundedExponentialBackOffTest.java | 1 - .../apache/beam/sdk/util/AvroUtilsTest.java | 20 ++-- .../beam/sdk/util/BucketingFunctionTest.java | 1 - ...fferedElementCountingOutputStreamTest.java | 19 ++- .../apache/beam/sdk/util/CoderUtilsTest.java | 7 +- .../beam/sdk/util/CombineFnUtilTest.java | 13 +- .../beam/sdk/util/ExecutableTriggerTest.java | 6 +- .../util/ExposedByteArrayInputStreamTest.java | 5 +- .../ExposedByteArrayOutputStreamTest.java | 5 +- .../sdk/util/FileIOChannelFactoryTest.java | 16 ++- .../sdk/util/FinishedTriggersSetTest.java | 3 +- .../beam/sdk/util/GatherAllPanesTest.java | 7 +- .../sdk/util/GcsIOChannelFactoryTest.java | 1 - .../org/apache/beam/sdk/util/GcsUtilTest.java | 23 ++-- .../beam/sdk/util/IOChannelUtilsTest.java | 6 +- .../sdk/util/IdentitySideInputWindowFn.java | 5 +- .../beam/sdk/util/InstanceBuilderTest.java | 1 - .../beam/sdk/util/KeyedWorkItemCoderTest.java | 4 +- .../sdk/util/MergingActiveWindowSetTest.java | 13 +- .../beam/sdk/util/MovingFunctionTest.java | 1 - .../beam/sdk/util/MutationDetectorsTest.java | 17 ++- .../org/apache/beam/sdk/util/PTupleTest.java | 1 - .../beam/sdk/util/PubsubClientTest.java | 7 +- .../beam/sdk/util/PubsubGrpcClientTest.java | 16 +-- .../beam/sdk/util/PubsubJsonClientTest.java | 15 +-- .../beam/sdk/util/PubsubTestClientTest.java | 17 ++- .../apache/beam/sdk/util/ReshuffleTest.java | 7 +- .../beam/sdk/util/ReshuffleTriggerTest.java | 1 - .../util/RetryHttpRequestInitializerTest.java | 12 +- .../beam/sdk/util/SerializableUtilsTest.java | 19 ++- .../apache/beam/sdk/util/StreamUtilsTest.java | 9 +- .../apache/beam/sdk/util/StringUtilsTest.java | 1 - .../org/apache/beam/sdk/util/StructsTest.java | 11 +- .../beam/sdk/util/TimerInternalsTest.java | 1 - .../apache/beam/sdk/util/TriggerTester.java | 31 ++--- .../beam/sdk/util/UnownedInputStreamTest.java | 3 +- .../sdk/util/UnownedOutputStreamTest.java | 3 +- .../util/UploadIdResponseInterceptorTest.java | 7 +- .../beam/sdk/util/UserCodeExceptionTest.java | 3 +- .../org/apache/beam/sdk/util/VarIntTest.java | 9 +- .../beam/sdk/util/WindowedValueTest.java | 9 +- .../apache/beam/sdk/util/ZipFilesTest.java | 14 +-- .../sdk/util/common/ReflectHelpersTest.java | 6 +- .../beam/sdk/util/gcsfs/GcsPathTest.java | 11 +- ...opyOnAccessInMemoryStateInternalsTest.java | 1 - .../state/InMemoryStateInternalsTest.java | 4 +- .../sdk/util/state/StateNamespacesTest.java | 1 - .../beam/sdk/util/state/StateTagTest.java | 1 - .../org/apache/beam/sdk/values/KVTest.java | 4 +- .../beam/sdk/values/PCollectionListTest.java | 3 +- .../beam/sdk/values/PCollectionTupleTest.java | 8 +- .../org/apache/beam/sdk/values/PDoneTest.java | 4 +- .../beam/sdk/values/TypeDescriptorTest.java | 10 +- .../beam/sdk/values/TypeDescriptorsTest.java | 6 +- .../beam/sdk/values/TypedPValueTest.java | 1 - .../extensions/joinlibrary/InnerJoinTest.java | 6 +- .../joinlibrary/OuterLeftJoinTest.java | 6 +- .../joinlibrary/OuterRightJoinTest.java | 6 +- .../io/gcp/bigquery/BigQueryAvroUtils.java | 7 +- .../beam/sdk/io/gcp/bigquery/BigQueryIO.java | 111 +++++++++--------- .../sdk/io/gcp/bigquery/BigQueryServices.java | 5 +- .../io/gcp/bigquery/BigQueryServicesImpl.java | 20 ++-- .../gcp/bigquery/BigQueryTableInserter.java | 14 +-- .../bigquery/BigQueryTableRowIterator.java | 12 +- .../beam/sdk/io/gcp/bigtable/BigtableIO.java | 43 ++++--- .../sdk/io/gcp/bigtable/BigtableService.java | 6 +- .../io/gcp/bigtable/BigtableServiceImpl.java | 12 +- .../sdk/io/gcp/datastore/DatastoreV1.java | 47 ++++---- .../gcp/bigquery/BigQueryAvroUtilsTest.java | 11 +- .../sdk/io/gcp/bigquery/BigQueryIOTest.java | 73 ++++++------ .../bigquery/BigQueryServicesImplTest.java | 29 ++--- .../bigquery/BigQueryTableInserterTest.java | 20 ++-- .../BigQueryTableRowIteratorTest.java | 10 +- .../sdk/io/gcp/bigquery/BigQueryUtilTest.java | 19 ++- .../sdk/io/gcp/bigtable/BigtableIOTest.java | 53 ++++----- .../sdk/io/gcp/bigtable/BigtableReadIT.java | 6 +- .../sdk/io/gcp/bigtable/BigtableWriteIT.java | 29 ++--- .../sdk/io/gcp/datastore/DatastoreV1Test.java | 53 ++++----- .../beam/sdk/io/gcp/datastore/V1ReadIT.java | 13 +- .../sdk/io/gcp/datastore/V1TestOptions.java | 3 +- .../beam/sdk/io/gcp/datastore/V1TestUtil.java | 17 ++- .../beam/sdk/io/gcp/datastore/V1WriteIT.java | 4 +- .../beam/sdk/io/hdfs/AvroHDFSFileSource.java | 23 ++-- .../beam/sdk/io/hdfs/AvroWrapperCoder.java | 16 ++- .../apache/beam/sdk/io/hdfs/HDFSFileSink.java | 17 ++- .../beam/sdk/io/hdfs/HDFSFileSource.java | 30 ++--- .../beam/sdk/io/hdfs/WritableCoder.java | 12 +- .../SimpleAuthAvroHDFSFileSource.java | 12 +- .../simpleauth/SimpleAuthHDFSFileSink.java | 3 +- .../simpleauth/SimpleAuthHDFSFileSource.java | 12 +- .../sdk/io/hdfs/AvroWrapperCoderTest.java | 5 +- .../beam/sdk/io/hdfs/HDFSFileSourceTest.java | 12 +- .../beam/sdk/io/hdfs/WritableCoderTest.java | 1 - .../beam/sdk/io/jms/JmsCheckpointMark.java | 9 +- .../org/apache/beam/sdk/io/jms/JmsIO.java | 36 +++--- .../org/apache/beam/sdk/io/jms/JmsRecord.java | 1 - .../org/apache/beam/sdk/io/jms/JmsIOTest.java | 25 ++-- .../sdk/io/kafka/KafkaCheckpointMark.java | 8 +- .../org/apache/beam/sdk/io/kafka/KafkaIO.java | 59 +++++----- .../apache/beam/sdk/io/kafka/KafkaRecord.java | 3 +- .../beam/sdk/io/kafka/KafkaRecordCoder.java | 14 +-- .../apache/beam/sdk/io/kafka/KafkaIOTest.java | 33 +++--- .../beam/sdk/transforms/CombineJava8Test.java | 7 +- .../beam/sdk/transforms/FilterJava8Test.java | 4 +- .../transforms/FlatMapElementsJava8Test.java | 9 +- .../sdk/transforms/MapElementsJava8Test.java | 4 +- .../sdk/transforms/PartitionJava8Test.java | 4 +- .../transforms/RemoveDuplicatesJava8Test.java | 11 +- .../sdk/transforms/WithKeysJava8Test.java | 1 - .../transforms/WithTimestampsJava8Test.java | 4 +- .../src/main/java/DebuggingWordCount.java | 9 +- .../src/main/java/WindowedWordCount.java | 19 ++- .../java/common/DataflowExampleUtils.java | 25 ++-- .../common/ExampleBigQueryTableOptions.java | 3 +- .../main/java/common/PubsubFileInjector.java | 14 +-- .../src/test/java/DebuggingWordCountTest.java | 6 +- .../src/test/java/WordCountTest.java | 5 +- .../src/main/java/StarterPipeline.java | 1 - .../src/main/java/it/pkg/StarterPipeline.java | 1 - .../coders/AvroCoderBenchmark.java | 6 +- .../coders/ByteArrayCoderBenchmark.java | 6 +- .../coders/CoderBenchmarking.java | 3 +- .../coders/StringUtf8CoderBenchmark.java | 6 +- 856 files changed, 3599 insertions(+), 5533 deletions(-) diff --git a/examples/java/src/main/java/org/apache/beam/examples/DebuggingWordCount.java b/examples/java/src/main/java/org/apache/beam/examples/DebuggingWordCount.java index c1b273ca9bec2..4a9aba9da5e8c 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/DebuggingWordCount.java +++ b/examples/java/src/main/java/org/apache/beam/examples/DebuggingWordCount.java @@ -17,6 +17,9 @@ */ package org.apache.beam.examples; +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; @@ -29,14 +32,9 @@ 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; -import java.util.Arrays; -import java.util.List; -import java.util.regex.Pattern; - /** * An example that verifies word counts in Shakespeare and includes Dataflow best practices. 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 842cb54b40fd2..6d69f1488526d 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,6 +17,13 @@ */ 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 org.apache.beam.examples.common.ExampleBigQueryTableOptions; import org.apache.beam.examples.common.ExampleOptions; import org.apache.beam.examples.common.ExampleUtils; @@ -33,19 +40,9 @@ import org.apache.beam.sdk.transforms.windowing.Window; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; - -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 org.joda.time.Duration; import org.joda.time.Instant; -import java.io.IOException; -import java.util.ArrayList; -import java.util.List; - /** * An example that counts words in text, and can run over either unbounded or bounded input 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 42d30bb5fa94a..d42d6214973d3 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/WordCount.java +++ b/examples/java/src/main/java/org/apache/beam/examples/WordCount.java @@ -17,6 +17,9 @@ */ package org.apache.beam.examples; +import com.google.common.base.Strings; +import com.google.common.io.Resources; +import java.io.IOException; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.io.TextIO; import org.apache.beam.sdk.options.Default; @@ -37,11 +40,6 @@ import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; -import com.google.common.base.Strings; -import com.google.common.io.Resources; - -import java.io.IOException; - /** * An example that counts words in Shakespeare and includes Beam best practices. * diff --git a/examples/java/src/main/java/org/apache/beam/examples/common/ExampleBigQueryTableOptions.java b/examples/java/src/main/java/org/apache/beam/examples/common/ExampleBigQueryTableOptions.java index 54cc99ea32fc6..2eef525faed0b 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/common/ExampleBigQueryTableOptions.java +++ b/examples/java/src/main/java/org/apache/beam/examples/common/ExampleBigQueryTableOptions.java @@ -17,14 +17,13 @@ */ package org.apache.beam.examples.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; -import com.google.api.services.bigquery.model.TableSchema; - /** * Options that can be used to configure BigQuery tables in Beam examples. * The project defaults to the project being used to run the example. diff --git a/examples/java/src/main/java/org/apache/beam/examples/common/ExampleOptions.java b/examples/java/src/main/java/org/apache/beam/examples/common/ExampleOptions.java index 43afeb4c61128..a7dcc7c175176 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/common/ExampleOptions.java +++ b/examples/java/src/main/java/org/apache/beam/examples/common/ExampleOptions.java @@ -17,21 +17,18 @@ */ package org.apache.beam.examples.common; +import com.google.common.base.MoreObjects; +import java.util.concurrent.ThreadLocalRandom; import org.apache.beam.sdk.options.ApplicationNameOptions; 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 com.google.common.base.MoreObjects; - import org.joda.time.DateTimeUtils; import org.joda.time.DateTimeZone; import org.joda.time.format.DateTimeFormat; import org.joda.time.format.DateTimeFormatter; -import java.util.concurrent.ThreadLocalRandom; - /** * Options that can be used to configure the Beam examples. */ diff --git a/examples/java/src/main/java/org/apache/beam/examples/common/ExampleUtils.java b/examples/java/src/main/java/org/apache/beam/examples/common/ExampleUtils.java index 7f03fc0d1c6f6..eadb580a257c4 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/common/ExampleUtils.java +++ b/examples/java/src/main/java/org/apache/beam/examples/common/ExampleUtils.java @@ -17,13 +17,6 @@ */ package org.apache.beam.examples.common; -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.AttemptBoundedExponentialBackOff; -import org.apache.beam.sdk.util.Transport; - import com.google.api.client.googleapis.json.GoogleJsonResponseException; import com.google.api.client.googleapis.services.AbstractGoogleClientRequest; import com.google.api.client.util.BackOff; @@ -43,12 +36,17 @@ 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.AttemptBoundedExponentialBackOff; +import org.apache.beam.sdk.util.Transport; /** * The utility class that sets up and tears down external resources, diff --git a/examples/java/src/main/java/org/apache/beam/examples/common/PubsubFileInjector.java b/examples/java/src/main/java/org/apache/beam/examples/common/PubsubFileInjector.java index 0a93521821d84..e6a1495e545d5 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/common/PubsubFileInjector.java +++ b/examples/java/src/main/java/org/apache/beam/examples/common/PubsubFileInjector.java @@ -17,6 +17,12 @@ */ package org.apache.beam.examples.common; +import com.google.api.services.pubsub.Pubsub; +import com.google.api.services.pubsub.model.PublishRequest; +import com.google.api.services.pubsub.model.PubsubMessage; +import com.google.common.collect.ImmutableMap; +import java.io.IOException; +import java.util.Arrays; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.io.TextIO; import org.apache.beam.sdk.options.Description; @@ -28,14 +34,6 @@ import org.apache.beam.sdk.transforms.OldDoFn; import org.apache.beam.sdk.util.Transport; -import com.google.api.services.pubsub.Pubsub; -import com.google.api.services.pubsub.model.PublishRequest; -import com.google.api.services.pubsub.model.PubsubMessage; -import com.google.common.collect.ImmutableMap; - -import java.io.IOException; -import java.util.Arrays; - /** * A batch Dataflow pipeline for injecting a set of GCS files into * a PubSub topic line by line. Empty lines are skipped. 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 120c64fae54ac..56c7855e45e09 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 @@ -21,6 +21,21 @@ import static com.google.datastore.v1.client.DatastoreHelper.makeKey; import static com.google.datastore.v1.client.DatastoreHelper.makeValue; +import com.google.api.services.bigquery.model.TableFieldSchema; +import com.google.api.services.bigquery.model.TableReference; +import com.google.api.services.bigquery.model.TableRow; +import com.google.api.services.bigquery.model.TableSchema; +import com.google.common.base.MoreObjects; +import com.google.datastore.v1.Entity; +import com.google.datastore.v1.Key; +import com.google.datastore.v1.Value; +import java.io.IOException; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.regex.Matcher; +import java.util.regex.Pattern; import org.apache.beam.examples.common.ExampleBigQueryTableOptions; import org.apache.beam.examples.common.ExampleOptions; import org.apache.beam.examples.common.ExampleUtils; @@ -53,26 +68,8 @@ import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionList; - -import com.google.api.services.bigquery.model.TableFieldSchema; -import com.google.api.services.bigquery.model.TableReference; -import com.google.api.services.bigquery.model.TableRow; -import com.google.api.services.bigquery.model.TableSchema; -import com.google.common.base.MoreObjects; -import com.google.datastore.v1.Entity; -import com.google.datastore.v1.Key; -import com.google.datastore.v1.Value; - import org.joda.time.Duration; -import java.io.IOException; -import java.util.ArrayList; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.regex.Matcher; -import java.util.regex.Pattern; - /** * An example that computes the most popular hash tags * for every prefix, which can be used for auto-completion. diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/StreamingWordExtract.java b/examples/java/src/main/java/org/apache/beam/examples/complete/StreamingWordExtract.java index 3f30f2150504a..c4ad35bcc8189 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/complete/StreamingWordExtract.java +++ b/examples/java/src/main/java/org/apache/beam/examples/complete/StreamingWordExtract.java @@ -17,6 +17,11 @@ */ package org.apache.beam.examples.complete; +import com.google.api.services.bigquery.model.TableFieldSchema; +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 org.apache.beam.examples.common.ExampleBigQueryTableOptions; import org.apache.beam.examples.common.ExampleOptions; import org.apache.beam.examples.common.ExampleUtils; @@ -31,13 +36,6 @@ import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.ParDo; -import com.google.api.services.bigquery.model.TableFieldSchema; -import com.google.api.services.bigquery.model.TableRow; -import com.google.api.services.bigquery.model.TableSchema; - -import java.io.IOException; -import java.util.ArrayList; - /** * A streaming Dataflow Example using BigQuery output. * 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 76b6b6a060085..a5a939263ee4e 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 @@ -17,6 +17,12 @@ */ package org.apache.beam.examples.complete; +import java.io.File; +import java.io.IOException; +import java.net.URI; +import java.net.URISyntaxException; +import java.util.HashSet; +import java.util.Set; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.KvCoder; @@ -51,17 +57,9 @@ import org.apache.beam.sdk.values.PDone; import org.apache.beam.sdk.values.PInput; import org.apache.beam.sdk.values.TupleTag; - import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.io.File; -import java.io.IOException; -import java.net.URI; -import java.net.URISyntaxException; -import java.util.HashSet; -import java.util.Set; - /** * An example that computes a basic TF-IDF search table for a directory or GCS prefix. * diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/TopWikipediaSessions.java b/examples/java/src/main/java/org/apache/beam/examples/complete/TopWikipediaSessions.java index aff41ccb5a4ec..4c5472bf6f687 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 @@ -17,6 +17,8 @@ */ package org.apache.beam.examples.complete; +import com.google.api.services.bigquery.model.TableRow; +import java.util.List; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.coders.TableRowJsonCoder; import org.apache.beam.sdk.io.TextIO; @@ -38,14 +40,9 @@ import org.apache.beam.sdk.transforms.windowing.Window; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; - -import com.google.api.services.bigquery.model.TableRow; - import org.joda.time.Duration; import org.joda.time.Instant; -import java.util.List; - /** * 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. 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 394b4327025c0..f7ff035dbf50e 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 @@ -17,6 +17,14 @@ */ package org.apache.beam.examples.complete; +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 org.apache.avro.reflect.Nullable; import org.apache.beam.examples.common.ExampleBigQueryTableOptions; import org.apache.beam.examples.common.ExampleOptions; import org.apache.beam.examples.common.ExampleUtils; @@ -39,22 +47,11 @@ import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PBegin; import org.apache.beam.sdk.values.PCollection; - -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 org.apache.avro.reflect.Nullable; import org.joda.time.Duration; import org.joda.time.Instant; import org.joda.time.format.DateTimeFormat; import org.joda.time.format.DateTimeFormatter; -import java.io.IOException; -import java.util.ArrayList; -import java.util.List; - /** * A Dataflow Example that runs in both batch and streaming modes with traffic sensor data. * You can configure the running mode by setting {@literal --streaming} to true or false. 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 ef716e9dd832d..12eaeaa972579 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 @@ -17,6 +17,19 @@ */ package org.apache.beam.examples.complete; +import com.google.api.services.bigquery.model.TableFieldSchema; +import com.google.api.services.bigquery.model.TableReference; +import com.google.api.services.bigquery.model.TableRow; +import com.google.api.services.bigquery.model.TableSchema; +import com.google.common.collect.Lists; +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.Hashtable; +import java.util.List; +import java.util.Map; +import org.apache.avro.reflect.Nullable; import org.apache.beam.examples.common.ExampleBigQueryTableOptions; import org.apache.beam.examples.common.ExampleOptions; import org.apache.beam.examples.common.ExampleUtils; @@ -38,27 +51,11 @@ import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PBegin; import org.apache.beam.sdk.values.PCollection; - -import com.google.api.services.bigquery.model.TableFieldSchema; -import com.google.api.services.bigquery.model.TableReference; -import com.google.api.services.bigquery.model.TableRow; -import com.google.api.services.bigquery.model.TableSchema; -import com.google.common.collect.Lists; - -import org.apache.avro.reflect.Nullable; import org.joda.time.Duration; import org.joda.time.Instant; import org.joda.time.format.DateTimeFormat; import org.joda.time.format.DateTimeFormatter; -import java.io.IOException; -import java.util.ArrayList; -import java.util.Collections; -import java.util.HashMap; -import java.util.Hashtable; -import java.util.List; -import java.util.Map; - /** * A Dataflow Example that runs in both batch and streaming modes with traffic sensor data. * You can configure the running mode by setting {@literal --streaming} to true or false. 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 09d9c29734e90..439cf020aaa9f 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 @@ -17,6 +17,11 @@ */ package org.apache.beam.examples.cookbook; +import com.google.api.services.bigquery.model.TableFieldSchema; +import com.google.api.services.bigquery.model.TableRow; +import com.google.api.services.bigquery.model.TableSchema; +import java.util.ArrayList; +import java.util.List; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO; import org.apache.beam.sdk.options.Default; @@ -31,13 +36,6 @@ import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; -import com.google.api.services.bigquery.model.TableFieldSchema; -import com.google.api.services.bigquery.model.TableRow; -import com.google.api.services.bigquery.model.TableSchema; - -import java.util.ArrayList; -import java.util.List; - /** * An example that reads the public samples of weather data from BigQuery, counts the number of * tornadoes that occur in each month, and writes the results to BigQuery. 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 67918a3f74284..1d280a6e154b5 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 @@ -17,6 +17,11 @@ */ package org.apache.beam.examples.cookbook; +import com.google.api.services.bigquery.model.TableFieldSchema; +import com.google.api.services.bigquery.model.TableRow; +import com.google.api.services.bigquery.model.TableSchema; +import java.util.ArrayList; +import java.util.List; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO; import org.apache.beam.sdk.options.Default; @@ -34,13 +39,6 @@ import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; -import com.google.api.services.bigquery.model.TableFieldSchema; -import com.google.api.services.bigquery.model.TableRow; -import com.google.api.services.bigquery.model.TableSchema; - -import java.util.ArrayList; -import java.util.List; - /** * An example that reads the public 'Shakespeare' data, and for each word in * the dataset that is over a given length, generates a string containing the diff --git a/examples/java/src/main/java/org/apache/beam/examples/cookbook/DatastoreWordCount.java b/examples/java/src/main/java/org/apache/beam/examples/cookbook/DatastoreWordCount.java index 215e2ffc78981..9a9e79968670e 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/cookbook/DatastoreWordCount.java +++ b/examples/java/src/main/java/org/apache/beam/examples/cookbook/DatastoreWordCount.java @@ -22,6 +22,14 @@ import static com.google.datastore.v1.client.DatastoreHelper.makeKey; import static com.google.datastore.v1.client.DatastoreHelper.makeValue; +import com.google.datastore.v1.Entity; +import com.google.datastore.v1.Key; +import com.google.datastore.v1.PropertyFilter; +import com.google.datastore.v1.Query; +import com.google.datastore.v1.Value; +import java.util.Map; +import java.util.UUID; +import javax.annotation.Nullable; import org.apache.beam.examples.WordCount; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.io.TextIO; @@ -36,16 +44,6 @@ import org.apache.beam.sdk.transforms.MapElements; import org.apache.beam.sdk.transforms.ParDo; -import com.google.datastore.v1.Entity; -import com.google.datastore.v1.Key; -import com.google.datastore.v1.PropertyFilter; -import com.google.datastore.v1.Query; -import com.google.datastore.v1.Value; - -import java.util.Map; -import java.util.UUID; -import javax.annotation.Nullable; - /** * A WordCount example using DatastoreIO. * 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 9a0f7a2a54938..6c42520ff7c6e 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 @@ -17,6 +17,12 @@ */ package org.apache.beam.examples.cookbook; +import com.google.api.services.bigquery.model.TableFieldSchema; +import com.google.api.services.bigquery.model.TableRow; +import com.google.api.services.bigquery.model.TableSchema; +import java.util.ArrayList; +import java.util.List; +import java.util.logging.Logger; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO; import org.apache.beam.sdk.options.Default; @@ -32,14 +38,6 @@ import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionView; -import com.google.api.services.bigquery.model.TableFieldSchema; -import com.google.api.services.bigquery.model.TableRow; -import com.google.api.services.bigquery.model.TableSchema; - -import java.util.ArrayList; -import java.util.List; -import java.util.logging.Logger; - /** * This is an example that demonstrates several approaches to filtering, and use of the Mean * transform. It shows how to dynamically set parameters by defining and using new pipeline options, diff --git a/examples/java/src/main/java/org/apache/beam/examples/cookbook/JoinExamples.java b/examples/java/src/main/java/org/apache/beam/examples/cookbook/JoinExamples.java index 5ff2ce22a0d9e..1b91bf1e0615b 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/cookbook/JoinExamples.java +++ b/examples/java/src/main/java/org/apache/beam/examples/cookbook/JoinExamples.java @@ -17,6 +17,7 @@ */ package org.apache.beam.examples.cookbook; +import com.google.api.services.bigquery.model.TableRow; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.io.TextIO; import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO; @@ -33,8 +34,6 @@ import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.TupleTag; -import com.google.api.services.bigquery.model.TableRow; - /** * This example shows how to do a join on two collections. * It uses a sample of the GDELT 'world event' data (http://goo.gl/OB6oin), joining the event 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 4f266d30ac968..3772a7bc5b86a 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 @@ -17,6 +17,11 @@ */ package org.apache.beam.examples.cookbook; +import com.google.api.services.bigquery.model.TableFieldSchema; +import com.google.api.services.bigquery.model.TableRow; +import com.google.api.services.bigquery.model.TableSchema; +import java.util.ArrayList; +import java.util.List; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO; import org.apache.beam.sdk.options.Default; @@ -31,13 +36,6 @@ import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; -import com.google.api.services.bigquery.model.TableFieldSchema; -import com.google.api.services.bigquery.model.TableRow; -import com.google.api.services.bigquery.model.TableSchema; - -import java.util.ArrayList; -import java.util.List; - /** * An example that reads the public samples of weather data from BigQuery, and finds * the maximum temperature ('mean_temp') for each month. 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 04ac2c363ed64..db59435555458 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 @@ -17,6 +17,13 @@ */ package org.apache.beam.examples.cookbook; +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.util.ArrayList; +import java.util.List; +import java.util.concurrent.TimeUnit; import org.apache.beam.examples.common.ExampleBigQueryTableOptions; import org.apache.beam.examples.common.ExampleOptions; import org.apache.beam.examples.common.ExampleUtils; @@ -42,19 +49,9 @@ import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionList; - -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 org.joda.time.Duration; import org.joda.time.Instant; -import java.util.ArrayList; -import java.util.List; -import java.util.concurrent.TimeUnit; - /** * This example illustrates the basic concepts behind triggering. It shows how to use different * trigger definitions to produce partial (speculative) results before all the data is processed and diff --git a/examples/java/src/test/java/org/apache/beam/examples/DebuggingWordCountTest.java b/examples/java/src/test/java/org/apache/beam/examples/DebuggingWordCountTest.java index f463b1e651dbd..c1bd5d45e38bf 100644 --- a/examples/java/src/test/java/org/apache/beam/examples/DebuggingWordCountTest.java +++ b/examples/java/src/test/java/org/apache/beam/examples/DebuggingWordCountTest.java @@ -18,16 +18,14 @@ package org.apache.beam.examples; import com.google.common.io.Files; - +import java.io.File; +import java.nio.charset.StandardCharsets; import org.junit.Rule; import org.junit.Test; import org.junit.rules.TemporaryFolder; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.io.File; -import java.nio.charset.StandardCharsets; - /** * Tests for {@link DebuggingWordCount}. */ 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 f93dc2b2f7792..ca0c9d6bda09d 100644 --- a/examples/java/src/test/java/org/apache/beam/examples/WordCountIT.java +++ b/examples/java/src/test/java/org/apache/beam/examples/WordCountIT.java @@ -18,6 +18,7 @@ package org.apache.beam.examples; +import java.util.Date; import org.apache.beam.examples.WordCount.WordCountOptions; import org.apache.beam.sdk.options.Default; import org.apache.beam.sdk.options.PipelineOptionsFactory; @@ -25,13 +26,10 @@ import org.apache.beam.sdk.testing.TestPipeline; import org.apache.beam.sdk.testing.TestPipelineOptions; import org.apache.beam.sdk.util.IOChannelUtils; - import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.util.Date; - /** * End-to-end tests of WordCount. */ 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 9d36a3e3e9703..98c5b17621be1 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 @@ -17,6 +17,8 @@ */ package org.apache.beam.examples; +import java.util.Arrays; +import java.util.List; import org.apache.beam.examples.WordCount.CountWords; import org.apache.beam.examples.WordCount.ExtractWordsFn; import org.apache.beam.examples.WordCount.FormatAsTextFn; @@ -30,7 +32,6 @@ 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; @@ -38,9 +39,6 @@ import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.util.Arrays; -import java.util.List; - /** * Tests of WordCount. */ 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 6f28dec0ecdb2..b6751c528bc87 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 @@ -17,6 +17,11 @@ */ package org.apache.beam.examples.complete; +import java.io.Serializable; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +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; @@ -33,19 +38,12 @@ import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.TimestampedValue; - import org.joda.time.Duration; import org.joda.time.Instant; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.Parameterized; -import java.io.Serializable; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collection; -import java.util.List; - /** * Tests of AutoComplete. */ 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 c7ce67e0c6958..c2d654ec18d99 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 @@ -17,6 +17,8 @@ */ package org.apache.beam.examples.complete; +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; @@ -27,15 +29,11 @@ import org.apache.beam.sdk.transforms.RemoveDuplicates; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; - import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.net.URI; -import java.util.Arrays; - /** * Tests of {@link TfIdf}. */ 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 d19998ee3896d..42fb06a031b34 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 @@ -17,22 +17,19 @@ */ package org.apache.beam.examples.complete; +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 com.google.api.services.bigquery.model.TableRow; - import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.util.Arrays; - /** Unit tests for {@link TopWikipediaSessions}. */ @RunWith(JUnit4.class) public class TopWikipediaSessionsTest { diff --git a/examples/java/src/test/java/org/apache/beam/examples/cookbook/BigQueryTornadoesIT.java b/examples/java/src/test/java/org/apache/beam/examples/cookbook/BigQueryTornadoesIT.java index fbd775cf50c8f..8bcab4a705221 100644 --- a/examples/java/src/test/java/org/apache/beam/examples/cookbook/BigQueryTornadoesIT.java +++ b/examples/java/src/test/java/org/apache/beam/examples/cookbook/BigQueryTornadoesIT.java @@ -21,7 +21,6 @@ import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.testing.TestPipeline; import org.apache.beam.sdk.testing.TestPipelineOptions; - import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; diff --git a/examples/java/src/test/java/org/apache/beam/examples/cookbook/BigQueryTornadoesTest.java b/examples/java/src/test/java/org/apache/beam/examples/cookbook/BigQueryTornadoesTest.java index b986c0bdfa3b8..87e1614ee0ca2 100644 --- a/examples/java/src/test/java/org/apache/beam/examples/cookbook/BigQueryTornadoesTest.java +++ b/examples/java/src/test/java/org/apache/beam/examples/cookbook/BigQueryTornadoesTest.java @@ -17,21 +17,18 @@ */ package org.apache.beam.examples.cookbook; +import com.google.api.services.bigquery.model.TableRow; +import java.util.List; import org.apache.beam.examples.cookbook.BigQueryTornadoes.ExtractTornadoesFn; import org.apache.beam.examples.cookbook.BigQueryTornadoes.FormatCountsFn; import org.apache.beam.sdk.transforms.DoFnTester; import org.apache.beam.sdk.values.KV; - -import com.google.api.services.bigquery.model.TableRow; - import org.hamcrest.CoreMatchers; import org.junit.Assert; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.util.List; - /** * Test case for {@link BigQueryTornadoes}. */ diff --git a/examples/java/src/test/java/org/apache/beam/examples/cookbook/CombinePerKeyExamplesTest.java b/examples/java/src/test/java/org/apache/beam/examples/cookbook/CombinePerKeyExamplesTest.java index 6d0b16793865f..34e06799e6408 100644 --- a/examples/java/src/test/java/org/apache/beam/examples/cookbook/CombinePerKeyExamplesTest.java +++ b/examples/java/src/test/java/org/apache/beam/examples/cookbook/CombinePerKeyExamplesTest.java @@ -17,21 +17,18 @@ */ package org.apache.beam.examples.cookbook; +import com.google.api.services.bigquery.model.TableRow; +import java.util.List; import org.apache.beam.examples.cookbook.CombinePerKeyExamples.ExtractLargeWordsFn; import org.apache.beam.examples.cookbook.CombinePerKeyExamples.FormatShakespeareOutputFn; import org.apache.beam.sdk.transforms.DoFnTester; import org.apache.beam.sdk.values.KV; - -import com.google.api.services.bigquery.model.TableRow; - import org.hamcrest.CoreMatchers; import org.junit.Assert; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.util.List; - /** Unit tests for {@link CombinePerKeyExamples}. */ @RunWith(JUnit4.class) public class CombinePerKeyExamplesTest { diff --git a/examples/java/src/test/java/org/apache/beam/examples/cookbook/DeDupExampleTest.java b/examples/java/src/test/java/org/apache/beam/examples/cookbook/DeDupExampleTest.java index 20e247062c499..c725e4f6bf02e 100644 --- a/examples/java/src/test/java/org/apache/beam/examples/cookbook/DeDupExampleTest.java +++ b/examples/java/src/test/java/org/apache/beam/examples/cookbook/DeDupExampleTest.java @@ -17,6 +17,8 @@ */ package org.apache.beam.examples.cookbook; +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; @@ -25,15 +27,11 @@ import org.apache.beam.sdk.transforms.Create; import org.apache.beam.sdk.transforms.RemoveDuplicates; import org.apache.beam.sdk.values.PCollection; - import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.util.Arrays; -import java.util.List; - /** Unit tests for {@link DeDupExample}. */ @RunWith(JUnit4.class) public class DeDupExampleTest { diff --git a/examples/java/src/test/java/org/apache/beam/examples/cookbook/FilterExamplesTest.java b/examples/java/src/test/java/org/apache/beam/examples/cookbook/FilterExamplesTest.java index 2598a971dd2ff..279478c50a243 100644 --- a/examples/java/src/test/java/org/apache/beam/examples/cookbook/FilterExamplesTest.java +++ b/examples/java/src/test/java/org/apache/beam/examples/cookbook/FilterExamplesTest.java @@ -17,21 +17,18 @@ */ package org.apache.beam.examples.cookbook; +import com.google.api.services.bigquery.model.TableRow; +import java.util.Arrays; +import java.util.List; import org.apache.beam.examples.cookbook.FilterExamples.FilterSingleMonthDataFn; import org.apache.beam.examples.cookbook.FilterExamples.ProjectionFn; import org.apache.beam.sdk.transforms.DoFnTester; - -import com.google.api.services.bigquery.model.TableRow; - import org.hamcrest.CoreMatchers; import org.junit.Assert; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.util.Arrays; -import java.util.List; - /** Unit tests for {@link FilterExamples}. */ @RunWith(JUnit4.class) public class FilterExamplesTest { 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 9b04667407552..60f71a2120f53 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 @@ -17,6 +17,9 @@ */ package org.apache.beam.examples.cookbook; +import com.google.api.services.bigquery.model.TableRow; +import java.util.Arrays; +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; @@ -27,9 +30,6 @@ import org.apache.beam.sdk.transforms.DoFnTester; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; - -import com.google.api.services.bigquery.model.TableRow; - import org.hamcrest.CoreMatchers; import org.junit.Assert; import org.junit.Test; @@ -37,9 +37,6 @@ import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.util.Arrays; -import java.util.List; - /** Unit tests for {@link JoinExamples}. */ @RunWith(JUnit4.class) public class JoinExamplesTest { diff --git a/examples/java/src/test/java/org/apache/beam/examples/cookbook/MaxPerKeyExamplesTest.java b/examples/java/src/test/java/org/apache/beam/examples/cookbook/MaxPerKeyExamplesTest.java index 1d5bcf473c1cb..b5ea0fc4bf0e2 100644 --- a/examples/java/src/test/java/org/apache/beam/examples/cookbook/MaxPerKeyExamplesTest.java +++ b/examples/java/src/test/java/org/apache/beam/examples/cookbook/MaxPerKeyExamplesTest.java @@ -17,22 +17,19 @@ */ package org.apache.beam.examples.cookbook; +import com.google.api.services.bigquery.model.TableRow; +import com.google.common.collect.ImmutableList; +import java.util.List; import org.apache.beam.examples.cookbook.MaxPerKeyExamples.ExtractTempFn; import org.apache.beam.examples.cookbook.MaxPerKeyExamples.FormatMaxesFn; import org.apache.beam.sdk.transforms.DoFnTester; import org.apache.beam.sdk.values.KV; - -import com.google.api.services.bigquery.model.TableRow; -import com.google.common.collect.ImmutableList; - import org.hamcrest.CoreMatchers; import org.junit.Assert; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.util.List; - /** Unit tests for {@link MaxPerKeyExamples}. */ @RunWith(JUnit4.class) public class MaxPerKeyExamplesTest { 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 fee3c141358d6..3848ca1135e82 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 @@ -17,6 +17,13 @@ */ package org.apache.beam.examples.cookbook; +import com.google.api.services.bigquery.model.TableRow; +import com.google.common.base.Joiner; +import com.google.common.collect.Lists; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +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; @@ -32,11 +39,6 @@ import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.TimestampedValue; - -import com.google.api.services.bigquery.model.TableRow; -import com.google.common.base.Joiner; -import com.google.common.collect.Lists; - import org.joda.time.Duration; import org.joda.time.Instant; import org.junit.Assert; @@ -45,11 +47,6 @@ import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.util.Arrays; -import java.util.Collections; -import java.util.List; -import java.util.Map; - /** * Unit Tests for {@link TriggerExample}. * The results generated by triggers are by definition non-deterministic and hence hard to test. diff --git a/examples/java8/src/main/java/org/apache/beam/examples/MinimalWordCountJava8.java b/examples/java8/src/main/java/org/apache/beam/examples/MinimalWordCountJava8.java index ff8ca552b57c7..a49da7bdfbb65 100644 --- a/examples/java8/src/main/java/org/apache/beam/examples/MinimalWordCountJava8.java +++ b/examples/java8/src/main/java/org/apache/beam/examples/MinimalWordCountJava8.java @@ -17,6 +17,7 @@ */ package org.apache.beam.examples; +import java.util.Arrays; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.io.TextIO; import org.apache.beam.sdk.options.PipelineOptions; @@ -28,8 +29,6 @@ import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.TypeDescriptors; -import java.util.Arrays; - /** * An example that counts words in Shakespeare, using Java 8 language features. * 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 01ffb1de6883e..f9957ebc0e3b9 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 @@ -17,6 +17,9 @@ */ package org.apache.beam.examples.complete.game; +import java.util.HashMap; +import java.util.Map; +import java.util.TimeZone; import org.apache.beam.examples.common.ExampleUtils; import org.apache.beam.examples.complete.game.utils.WriteWindowedToBigQuery; import org.apache.beam.sdk.Pipeline; @@ -45,7 +48,6 @@ import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TypeDescriptors; - import org.joda.time.DateTimeZone; import org.joda.time.Duration; import org.joda.time.Instant; @@ -54,10 +56,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.util.HashMap; -import java.util.Map; -import java.util.TimeZone; - /** * This class is the fourth in a series of four pipelines that tell a story in a 'gaming' * domain, following {@link UserScore}, {@link HourlyTeamScore}, and {@link LeaderBoard}. diff --git a/examples/java8/src/main/java/org/apache/beam/examples/complete/game/HourlyTeamScore.java b/examples/java8/src/main/java/org/apache/beam/examples/complete/game/HourlyTeamScore.java index e489607dee4d6..d408e2132dabf 100644 --- a/examples/java8/src/main/java/org/apache/beam/examples/complete/game/HourlyTeamScore.java +++ b/examples/java8/src/main/java/org/apache/beam/examples/complete/game/HourlyTeamScore.java @@ -17,6 +17,9 @@ */ package org.apache.beam.examples.complete.game; +import java.util.HashMap; +import java.util.Map; +import java.util.TimeZone; import org.apache.beam.examples.complete.game.utils.WriteWindowedToBigQuery; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.io.TextIO; @@ -30,17 +33,12 @@ 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.joda.time.DateTimeZone; import org.joda.time.Duration; import org.joda.time.Instant; import org.joda.time.format.DateTimeFormat; import org.joda.time.format.DateTimeFormatter; -import java.util.HashMap; -import java.util.Map; -import java.util.TimeZone; - /** * This class is the second in a series of four pipelines that tell a story in a 'gaming' * domain, following {@link UserScore}. In addition to the concepts introduced in {@link UserScore}, 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 bd223059e3051..8dd4e39bee01f 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 @@ -17,6 +17,9 @@ */ package org.apache.beam.examples.complete.game; +import java.util.HashMap; +import java.util.Map; +import java.util.TimeZone; import org.apache.beam.examples.common.ExampleOptions; import org.apache.beam.examples.common.ExampleUtils; import org.apache.beam.examples.complete.game.utils.WriteToBigQuery; @@ -39,17 +42,12 @@ 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.DateTimeZone; import org.joda.time.Duration; import org.joda.time.Instant; import org.joda.time.format.DateTimeFormat; import org.joda.time.format.DateTimeFormatter; -import java.util.HashMap; -import java.util.Map; -import java.util.TimeZone; - /** * This class is the third in a series of four pipelines that tell a story in a 'gaming' domain, * following {@link UserScore} and {@link HourlyTeamScore}. Concepts include: processing unbounded 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 c97eb4152991e..65036cee6b922 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 @@ -17,6 +17,9 @@ */ package org.apache.beam.examples.complete.game; +import java.util.HashMap; +import java.util.Map; +import org.apache.avro.reflect.Nullable; import org.apache.beam.examples.complete.game.utils.WriteToBigQuery; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.coders.AvroCoder; @@ -36,14 +39,9 @@ import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.TypeDescriptors; - -import org.apache.avro.reflect.Nullable; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.util.HashMap; -import java.util.Map; - /** * This class is the first in a series of four pipelines that tell a story in a 'gaming' domain. * Concepts: batch processing; reading input from Google Cloud Storage and writing output to diff --git a/examples/java8/src/main/java/org/apache/beam/examples/complete/game/injector/Injector.java b/examples/java8/src/main/java/org/apache/beam/examples/complete/game/injector/Injector.java index 034a186462927..8f8bd9febc31c 100644 --- a/examples/java8/src/main/java/org/apache/beam/examples/complete/game/injector/Injector.java +++ b/examples/java8/src/main/java/org/apache/beam/examples/complete/game/injector/Injector.java @@ -21,11 +21,6 @@ import com.google.api.services.pubsub.model.PublishRequest; import com.google.api.services.pubsub.model.PubsubMessage; import com.google.common.collect.ImmutableMap; - -import org.joda.time.DateTimeZone; -import org.joda.time.format.DateTimeFormat; -import org.joda.time.format.DateTimeFormatter; - import java.io.BufferedOutputStream; import java.io.FileOutputStream; import java.io.IOException; @@ -36,6 +31,9 @@ import java.util.List; import java.util.Random; import java.util.TimeZone; +import org.joda.time.DateTimeZone; +import org.joda.time.format.DateTimeFormat; +import org.joda.time.format.DateTimeFormatter; /** diff --git a/examples/java8/src/main/java/org/apache/beam/examples/complete/game/injector/InjectorUtils.java b/examples/java8/src/main/java/org/apache/beam/examples/complete/game/injector/InjectorUtils.java index 53e644d67fe75..8cba6c2d05f0f 100644 --- a/examples/java8/src/main/java/org/apache/beam/examples/complete/game/injector/InjectorUtils.java +++ b/examples/java8/src/main/java/org/apache/beam/examples/complete/game/injector/InjectorUtils.java @@ -29,7 +29,6 @@ import com.google.api.services.pubsub.Pubsub; import com.google.api.services.pubsub.PubsubScopes; import com.google.api.services.pubsub.model.Topic; - import java.io.IOException; class InjectorUtils { diff --git a/examples/java8/src/main/java/org/apache/beam/examples/complete/game/injector/RetryHttpInitializerWrapper.java b/examples/java8/src/main/java/org/apache/beam/examples/complete/game/injector/RetryHttpInitializerWrapper.java index 45be28791fc76..059999cae7d23 100644 --- a/examples/java8/src/main/java/org/apache/beam/examples/complete/game/injector/RetryHttpInitializerWrapper.java +++ b/examples/java8/src/main/java/org/apache/beam/examples/complete/game/injector/RetryHttpInitializerWrapper.java @@ -28,7 +28,6 @@ import com.google.api.client.http.HttpUnsuccessfulResponseHandler; import com.google.api.client.util.ExponentialBackOff; import com.google.api.client.util.Sleeper; - import java.io.IOException; import java.util.logging.Logger; 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 6af6e15a53217..40c4286f3afeb 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 @@ -17,6 +17,14 @@ */ package org.apache.beam.examples.complete.game.utils; +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.Serializable; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; import org.apache.beam.examples.complete.game.UserScore; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO; @@ -31,16 +39,6 @@ import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PDone; -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.Serializable; -import java.util.ArrayList; -import java.util.List; -import java.util.Map; - /** * Generate, format, and write BigQuery table row information. Use provided information about * the field names and types, as well as lambda functions that describe how to generate their 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 c59fd61f68a92..09f3b6cf90d97 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 @@ -17,6 +17,8 @@ */ package org.apache.beam.examples.complete.game.utils; +import com.google.api.services.bigquery.model.TableRow; +import java.util.Map; import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO; import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.Write.CreateDisposition; import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.Write.WriteDisposition; @@ -27,10 +29,6 @@ import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PDone; -import com.google.api.services.bigquery.model.TableRow; - -import java.util.Map; - /** * Generate, format, and write BigQuery table row information. Subclasses {@link WriteToBigQuery} * to require windowing; so this subclass may be used for writes that require access to the 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 4dfa474c7cf51..85841a7801474 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 @@ -17,6 +17,15 @@ */ package org.apache.beam.examples; +import com.google.common.collect.ImmutableList; +import java.io.IOException; +import java.io.Serializable; +import java.nio.channels.FileChannel; +import java.nio.channels.SeekableByteChannel; +import java.nio.file.Files; +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; @@ -29,9 +38,6 @@ import org.apache.beam.sdk.util.gcsfs.GcsPath; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.TypeDescriptors; - -import com.google.common.collect.ImmutableList; - import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; @@ -39,15 +45,6 @@ import org.mockito.invocation.InvocationOnMock; import org.mockito.stubbing.Answer; -import java.io.IOException; -import java.io.Serializable; -import java.nio.channels.FileChannel; -import java.nio.channels.SeekableByteChannel; -import java.nio.file.Files; -import java.nio.file.StandardOpenOption; -import java.util.Arrays; -import java.util.List; - /** * To keep {@link MinimalWordCountJava8} simple, it is not factored or testable. This test * file should be maintained with a copy of its code for a basic smoke test. 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 df8800ddb5f6a..7cd03f365b345 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 @@ -17,6 +17,9 @@ */ package org.apache.beam.examples.complete.game; +import java.io.Serializable; +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.testing.PAssert; @@ -25,16 +28,11 @@ import org.apache.beam.sdk.transforms.Create; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; - import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.io.Serializable; -import java.util.Arrays; -import java.util.List; - /** * Tests of GameStats. * Because the pipeline was designed for easy readability and explanations, it lacks good 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 b917b4cf535f9..f9fefb61f35c6 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 @@ -17,6 +17,9 @@ */ package org.apache.beam.examples.complete.game; +import java.io.Serializable; +import java.util.Arrays; +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; @@ -31,17 +34,12 @@ import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.TypeDescriptors; - import org.joda.time.Instant; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.io.Serializable; -import java.util.Arrays; -import java.util.List; - /** * Tests of HourlyTeamScore. * Because the pipeline was designed for easy readability and explanations, it lacks good 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 75d371a8caa49..7c86adf3754fa 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 @@ -17,6 +17,9 @@ */ package org.apache.beam.examples.complete.game; +import java.io.Serializable; +import java.util.Arrays; +import java.util.List; 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; @@ -32,17 +35,12 @@ import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.TypeDescriptors; - import org.junit.Assert; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.io.Serializable; -import java.util.Arrays; -import java.util.List; - /** * Tests of UserScore. */ diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/SideInputHandler.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/SideInputHandler.java index 65502517219cd..a97d3f306d0ee 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/SideInputHandler.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/SideInputHandler.java @@ -17,6 +17,14 @@ */ package org.apache.beam.runners.core; +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.coders.Coder; import org.apache.beam.sdk.coders.SetCoder; import org.apache.beam.sdk.transforms.Combine; @@ -31,15 +39,6 @@ import org.apache.beam.sdk.util.state.ValueState; import org.apache.beam.sdk.values.PCollectionView; -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; - /** * Generic side input handler that uses {@link StateInternals} to store all data. Both the actual * side-input data and data about the windows for which we have side inputs available are stored 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 3ce0c060c061e..73688d45592e9 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 @@ -17,11 +17,25 @@ */ package org.apache.beam.runners.core; -import static org.apache.beam.sdk.util.StringUtils.approximateSimpleName; - import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.base.Preconditions.checkNotNull; +import static org.apache.beam.sdk.util.StringUtils.approximateSimpleName; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Function; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.Lists; +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.util.Arrays; +import java.util.Collections; +import java.util.Iterator; +import java.util.List; +import java.util.NoSuchElementException; +import javax.annotation.Nullable; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.CoderException; import org.apache.beam.sdk.coders.ListCoder; @@ -40,29 +54,10 @@ import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PInput; import org.apache.beam.sdk.values.TimestampedValue; - -import com.google.common.annotations.VisibleForTesting; -import com.google.common.base.Function; -import com.google.common.collect.ImmutableList; -import com.google.common.collect.Lists; - -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonProperty; import org.joda.time.Instant; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.io.IOException; -import java.io.InputStream; -import java.io.OutputStream; -import java.util.Arrays; -import java.util.Collections; -import java.util.Iterator; -import java.util.List; -import java.util.NoSuchElementException; - -import javax.annotation.Nullable; - /** * {@link PTransform} that converts a {@link BoundedSource} as an {@link UnboundedSource}. * diff --git a/runners/core-java/src/main/java/org/apache/beam/sdk/util/AssignWindowsDoFn.java b/runners/core-java/src/main/java/org/apache/beam/sdk/util/AssignWindowsDoFn.java index 739db45e667a6..7e26253a829ad 100644 --- a/runners/core-java/src/main/java/org/apache/beam/sdk/util/AssignWindowsDoFn.java +++ b/runners/core-java/src/main/java/org/apache/beam/sdk/util/AssignWindowsDoFn.java @@ -19,18 +19,15 @@ import static com.google.common.base.Preconditions.checkNotNull; +import com.google.common.collect.Iterables; +import java.util.Collection; import org.apache.beam.sdk.transforms.OldDoFn; import org.apache.beam.sdk.transforms.OldDoFn.RequiresWindowAccess; 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 com.google.common.collect.Iterables; - import org.joda.time.Instant; -import java.util.Collection; - /** * {@link OldDoFn} that tags elements of a {@link PCollection} with windows, according to the * provided {@link WindowFn}. diff --git a/runners/core-java/src/main/java/org/apache/beam/sdk/util/BatchTimerInternals.java b/runners/core-java/src/main/java/org/apache/beam/sdk/util/BatchTimerInternals.java index 0dd03ba085e58..f3e84a6861cb7 100644 --- a/runners/core-java/src/main/java/org/apache/beam/sdk/util/BatchTimerInternals.java +++ b/runners/core-java/src/main/java/org/apache/beam/sdk/util/BatchTimerInternals.java @@ -19,17 +19,13 @@ import static com.google.common.base.Preconditions.checkState; -import org.apache.beam.sdk.transforms.windowing.BoundedWindow; - import com.google.common.base.MoreObjects; - -import org.joda.time.Instant; - 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.joda.time.Instant; /** * TimerInternals that uses priority queues to manage the timers that are ready to fire. diff --git a/runners/core-java/src/main/java/org/apache/beam/sdk/util/DoFnRunnerBase.java b/runners/core-java/src/main/java/org/apache/beam/sdk/util/DoFnRunnerBase.java index ed9ec10f0c688..8a0f6bf868d91 100644 --- a/runners/core-java/src/main/java/org/apache/beam/sdk/util/DoFnRunnerBase.java +++ b/runners/core-java/src/main/java/org/apache/beam/sdk/util/DoFnRunnerBase.java @@ -19,6 +19,17 @@ import static com.google.common.base.Preconditions.checkNotNull; +import com.google.common.collect.Iterables; +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; +import com.google.common.collect.Sets; +import java.io.IOException; +import java.util.Collection; +import java.util.Collections; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Set; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.IterableCoder; import org.apache.beam.sdk.options.PipelineOptions; @@ -37,19 +48,8 @@ import org.apache.beam.sdk.util.state.StateInternals; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TupleTag; -import com.google.common.collect.Iterables; -import com.google.common.collect.Lists; -import com.google.common.collect.Maps; -import com.google.common.collect.Sets; import org.joda.time.Instant; import org.joda.time.format.PeriodFormat; -import java.io.IOException; -import java.util.Collection; -import java.util.Collections; -import java.util.Iterator; -import java.util.List; -import java.util.Map; -import java.util.Set; /** * A base implementation of {@link DoFnRunner}. diff --git a/runners/core-java/src/main/java/org/apache/beam/sdk/util/DoFnRunners.java b/runners/core-java/src/main/java/org/apache/beam/sdk/util/DoFnRunners.java index 60892281c7b78..c4df7b222036f 100644 --- a/runners/core-java/src/main/java/org/apache/beam/sdk/util/DoFnRunners.java +++ b/runners/core-java/src/main/java/org/apache/beam/sdk/util/DoFnRunners.java @@ -17,6 +17,7 @@ */ package org.apache.beam.sdk.util; +import java.util.List; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.Aggregator.AggregatorFactory; import org.apache.beam.sdk.transforms.OldDoFn; @@ -26,8 +27,6 @@ import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.TupleTag; -import java.util.List; - /** * Static utility methods that provide {@link DoFnRunner} implementations. */ diff --git a/runners/core-java/src/main/java/org/apache/beam/sdk/util/GroupAlsoByWindowsViaOutputBufferDoFn.java b/runners/core-java/src/main/java/org/apache/beam/sdk/util/GroupAlsoByWindowsViaOutputBufferDoFn.java index f872ffcf1bb09..cc418daec7e97 100644 --- a/runners/core-java/src/main/java/org/apache/beam/sdk/util/GroupAlsoByWindowsViaOutputBufferDoFn.java +++ b/runners/core-java/src/main/java/org/apache/beam/sdk/util/GroupAlsoByWindowsViaOutputBufferDoFn.java @@ -17,18 +17,15 @@ */ package org.apache.beam.sdk.util; +import com.google.common.collect.Iterables; +import java.util.List; import org.apache.beam.sdk.transforms.OldDoFn; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.util.state.StateInternals; import org.apache.beam.sdk.util.state.StateInternalsFactory; import org.apache.beam.sdk.values.KV; - -import com.google.common.collect.Iterables; - import org.joda.time.Instant; -import java.util.List; - /** * The default batch {@link GroupAlsoByWindowsDoFn} implementation, if no specialized "fast path" * implementation is applicable. diff --git a/runners/core-java/src/main/java/org/apache/beam/sdk/util/GroupByKeyViaGroupByKeyOnly.java b/runners/core-java/src/main/java/org/apache/beam/sdk/util/GroupByKeyViaGroupByKeyOnly.java index f0f900744e73a..fdad17a8bb0c7 100644 --- a/runners/core-java/src/main/java/org/apache/beam/sdk/util/GroupByKeyViaGroupByKeyOnly.java +++ b/runners/core-java/src/main/java/org/apache/beam/sdk/util/GroupByKeyViaGroupByKeyOnly.java @@ -19,6 +19,10 @@ import static com.google.common.base.Preconditions.checkArgument; +import java.util.ArrayList; +import java.util.Collections; +import java.util.Comparator; +import java.util.List; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.IterableCoder; import org.apache.beam.sdk.coders.KvCoder; @@ -32,11 +36,6 @@ import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; -import java.util.ArrayList; -import java.util.Collections; -import java.util.Comparator; -import java.util.List; - /** * An implementation of {@link GroupByKey} built on top of a lower-level {@link GroupByKeyOnly} * primitive. diff --git a/runners/core-java/src/main/java/org/apache/beam/sdk/util/LateDataDroppingDoFnRunner.java b/runners/core-java/src/main/java/org/apache/beam/sdk/util/LateDataDroppingDoFnRunner.java index 8b3ba24bb21cd..08c670e4753f5 100644 --- a/runners/core-java/src/main/java/org/apache/beam/sdk/util/LateDataDroppingDoFnRunner.java +++ b/runners/core-java/src/main/java/org/apache/beam/sdk/util/LateDataDroppingDoFnRunner.java @@ -17,16 +17,14 @@ */ package org.apache.beam.sdk.util; -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.values.KV; - import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Function; import com.google.common.base.Predicate; import com.google.common.collect.Iterables; - +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.values.KV; import org.joda.time.Instant; /** diff --git a/runners/core-java/src/main/java/org/apache/beam/sdk/util/PaneInfoTracker.java b/runners/core-java/src/main/java/org/apache/beam/sdk/util/PaneInfoTracker.java index 0c5849e99cdc5..90c10b5c7449e 100644 --- a/runners/core-java/src/main/java/org/apache/beam/sdk/util/PaneInfoTracker.java +++ b/runners/core-java/src/main/java/org/apache/beam/sdk/util/PaneInfoTracker.java @@ -19,6 +19,8 @@ import static com.google.common.base.Preconditions.checkState; +import com.google.common.annotations.VisibleForTesting; +import edu.umd.cs.findbugs.annotations.SuppressFBWarnings; import org.apache.beam.sdk.transforms.windowing.AfterWatermark; import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.transforms.windowing.PaneInfo.PaneInfoCoder; @@ -28,11 +30,7 @@ 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 com.google.common.annotations.VisibleForTesting; - import org.joda.time.Instant; -import edu.umd.cs.findbugs.annotations.SuppressFBWarnings; /** * Determine the timing and other properties of a new pane for a given computation, key and window. diff --git a/runners/core-java/src/main/java/org/apache/beam/sdk/util/PushbackSideInputDoFnRunner.java b/runners/core-java/src/main/java/org/apache/beam/sdk/util/PushbackSideInputDoFnRunner.java index b1442dd1e24a5..d9f1fbffccb38 100644 --- a/runners/core-java/src/main/java/org/apache/beam/sdk/util/PushbackSideInputDoFnRunner.java +++ b/runners/core-java/src/main/java/org/apache/beam/sdk/util/PushbackSideInputDoFnRunner.java @@ -17,16 +17,14 @@ */ package org.apache.beam.sdk.util; -import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.values.PCollectionView; - import com.google.common.collect.ImmutableList; import com.google.common.collect.Iterables; - import java.util.Collection; import java.util.Collections; import java.util.HashSet; import java.util.Set; +import org.apache.beam.sdk.transforms.windowing.BoundedWindow; +import org.apache.beam.sdk.values.PCollectionView; /** * A {@link DoFnRunner} that can refuse to process elements that are not ready, instead returning diff --git a/runners/core-java/src/main/java/org/apache/beam/sdk/util/ReduceFn.java b/runners/core-java/src/main/java/org/apache/beam/sdk/util/ReduceFn.java index c5ee1e1f940d0..8135a5beca530 100644 --- a/runners/core-java/src/main/java/org/apache/beam/sdk/util/ReduceFn.java +++ b/runners/core-java/src/main/java/org/apache/beam/sdk/util/ReduceFn.java @@ -17,16 +17,14 @@ */ package org.apache.beam.sdk.util; +import java.io.Serializable; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.util.state.MergingStateAccessor; import org.apache.beam.sdk.util.state.ReadableState; import org.apache.beam.sdk.util.state.StateAccessor; - import org.joda.time.Instant; -import java.io.Serializable; - /** * Specification for processing to happen after elements have been grouped by key. * diff --git a/runners/core-java/src/main/java/org/apache/beam/sdk/util/ReduceFnContextFactory.java b/runners/core-java/src/main/java/org/apache/beam/sdk/util/ReduceFnContextFactory.java index 2d865086f74d4..5d27d51ad2f43 100644 --- a/runners/core-java/src/main/java/org/apache/beam/sdk/util/ReduceFnContextFactory.java +++ b/runners/core-java/src/main/java/org/apache/beam/sdk/util/ReduceFnContextFactory.java @@ -20,6 +20,10 @@ import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.base.Preconditions.checkNotNull; +import com.google.common.collect.ImmutableMap; +import java.util.Collection; +import java.util.Map; +import javax.annotation.Nullable; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; @@ -36,16 +40,8 @@ import org.apache.beam.sdk.util.state.StateNamespaces; import org.apache.beam.sdk.util.state.StateNamespaces.WindowNamespace; import org.apache.beam.sdk.util.state.StateTag; - -import com.google.common.collect.ImmutableMap; - import org.joda.time.Instant; -import java.util.Collection; -import java.util.Map; - -import javax.annotation.Nullable; - /** * Factory for creating instances of the various {@link ReduceFn} contexts. */ diff --git a/runners/core-java/src/main/java/org/apache/beam/sdk/util/ReduceFnRunner.java b/runners/core-java/src/main/java/org/apache/beam/sdk/util/ReduceFnRunner.java index 1fa083097aada..61e5b21ebfd93 100644 --- a/runners/core-java/src/main/java/org/apache/beam/sdk/util/ReduceFnRunner.java +++ b/runners/core-java/src/main/java/org/apache/beam/sdk/util/ReduceFnRunner.java @@ -20,6 +20,17 @@ import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.base.Preconditions.checkState; +import com.google.common.annotations.VisibleForTesting; +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.List; +import java.util.Map; +import java.util.Set; +import javax.annotation.Nullable; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.Aggregator; import org.apache.beam.sdk.transforms.OldDoFn; @@ -41,24 +52,9 @@ import org.apache.beam.sdk.util.state.StateNamespaces.WindowNamespace; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; - -import com.google.common.annotations.VisibleForTesting; -import com.google.common.collect.ImmutableMap; - import org.joda.time.Duration; import org.joda.time.Instant; -import java.util.ArrayList; -import java.util.Collection; -import java.util.Collections; -import java.util.HashMap; -import java.util.HashSet; -import java.util.List; -import java.util.Map; -import java.util.Set; - -import javax.annotation.Nullable; - /** * Manages the execution of a {@link ReduceFn} after a {@link GroupByKeyOnly} has partitioned the * {@link PCollection} by key. diff --git a/runners/core-java/src/main/java/org/apache/beam/sdk/util/SimpleDoFnRunner.java b/runners/core-java/src/main/java/org/apache/beam/sdk/util/SimpleDoFnRunner.java index a0cdb40f779f3..6c1cf451d61a6 100644 --- a/runners/core-java/src/main/java/org/apache/beam/sdk/util/SimpleDoFnRunner.java +++ b/runners/core-java/src/main/java/org/apache/beam/sdk/util/SimpleDoFnRunner.java @@ -17,13 +17,13 @@ */ package org.apache.beam.sdk.util; +import java.util.List; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.Aggregator.AggregatorFactory; import org.apache.beam.sdk.transforms.OldDoFn; import org.apache.beam.sdk.util.DoFnRunners.OutputManager; import org.apache.beam.sdk.util.ExecutionContext.StepContext; import org.apache.beam.sdk.values.TupleTag; -import java.util.List; /** * Runs a {@link OldDoFn} by constructing the appropriate contexts and passing them in. diff --git a/runners/core-java/src/main/java/org/apache/beam/sdk/util/SystemReduceFn.java b/runners/core-java/src/main/java/org/apache/beam/sdk/util/SystemReduceFn.java index f7dca9433ec8d..28177a811cd3a 100644 --- a/runners/core-java/src/main/java/org/apache/beam/sdk/util/SystemReduceFn.java +++ b/runners/core-java/src/main/java/org/apache/beam/sdk/util/SystemReduceFn.java @@ -18,6 +18,7 @@ package org.apache.beam.sdk.util; +import edu.umd.cs.findbugs.annotations.SuppressFBWarnings; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.transforms.Combine.CombineFn; import org.apache.beam.sdk.transforms.Combine.KeyedCombineFn; @@ -34,8 +35,6 @@ import org.apache.beam.sdk.util.state.StateTag; import org.apache.beam.sdk.util.state.StateTags; -import edu.umd.cs.findbugs.annotations.SuppressFBWarnings; - /** * {@link ReduceFn} implementing the default reduction behaviors of {@link GroupByKey}. * diff --git a/runners/core-java/src/main/java/org/apache/beam/sdk/util/TriggerRunner.java b/runners/core-java/src/main/java/org/apache/beam/sdk/util/TriggerRunner.java index e00b843e53417..a53fb8c175135 100644 --- a/runners/core-java/src/main/java/org/apache/beam/sdk/util/TriggerRunner.java +++ b/runners/core-java/src/main/java/org/apache/beam/sdk/util/TriggerRunner.java @@ -19,6 +19,12 @@ import static com.google.common.base.Preconditions.checkState; +import com.google.common.annotations.VisibleForTesting; +import com.google.common.collect.ImmutableMap; +import edu.umd.cs.findbugs.annotations.SuppressFBWarnings; +import java.util.BitSet; +import java.util.Collection; +import java.util.Map; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.DefaultTrigger; import org.apache.beam.sdk.transforms.windowing.Trigger; @@ -27,18 +33,8 @@ 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 com.google.common.annotations.VisibleForTesting; -import com.google.common.collect.ImmutableMap; - import org.joda.time.Instant; -import java.util.BitSet; -import java.util.Collection; -import java.util.Map; - -import edu.umd.cs.findbugs.annotations.SuppressFBWarnings; - /** * Executes a trigger while managing persistence of information about which subtriggers are * finished. Subtriggers include all recursive trigger expressions as well as the entire trigger. diff --git a/runners/core-java/src/main/java/org/apache/beam/sdk/util/WatermarkHold.java b/runners/core-java/src/main/java/org/apache/beam/sdk/util/WatermarkHold.java index 5c1700992c296..7404e1bc8ba3a 100644 --- a/runners/core-java/src/main/java/org/apache/beam/sdk/util/WatermarkHold.java +++ b/runners/core-java/src/main/java/org/apache/beam/sdk/util/WatermarkHold.java @@ -19,6 +19,10 @@ import static com.google.common.base.Preconditions.checkState; +import com.google.common.annotations.VisibleForTesting; +import edu.umd.cs.findbugs.annotations.SuppressFBWarnings; +import java.io.Serializable; +import javax.annotation.Nullable; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.OutputTimeFn; import org.apache.beam.sdk.transforms.windowing.OutputTimeFns; @@ -30,17 +34,9 @@ import org.apache.beam.sdk.util.state.StateTag; import org.apache.beam.sdk.util.state.StateTags; import org.apache.beam.sdk.util.state.WatermarkHoldState; - -import com.google.common.annotations.VisibleForTesting; - import org.joda.time.Duration; import org.joda.time.Instant; -import java.io.Serializable; -import javax.annotation.Nullable; - -import edu.umd.cs.findbugs.annotations.SuppressFBWarnings; - /** * Implements the logic to hold the output watermark for a computation back * until it has seen all the elements it needs based on the input watermark for the diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/SideInputHandlerTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/SideInputHandlerTest.java index 641e25e5e0f5c..0bf5e90d0d6a2 100644 --- a/runners/core-java/src/test/java/org/apache/beam/runners/core/SideInputHandlerTest.java +++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/SideInputHandlerTest.java @@ -21,6 +21,7 @@ import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; +import com.google.common.collect.ImmutableList; import org.apache.beam.sdk.coders.StringUtf8Coder; import org.apache.beam.sdk.testing.PCollectionViewTesting; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; @@ -32,9 +33,6 @@ import org.apache.beam.sdk.util.state.InMemoryStateInternals; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TupleTag; - -import com.google.common.collect.ImmutableList; - import org.joda.time.Duration; import org.joda.time.Instant; import org.junit.Assert; 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 45c1414e2a578..efc446e008fa9 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 @@ -21,6 +21,18 @@ import static org.junit.Assert.assertNull; import static org.junit.Assert.assertTrue; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.Lists; +import com.google.common.collect.Sets; +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.List; +import java.util.NoSuchElementException; +import java.util.Random; 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; @@ -45,11 +57,6 @@ import org.apache.beam.sdk.util.CoderUtils; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.TimestampedValue; - -import com.google.common.collect.ImmutableList; -import com.google.common.collect.Lists; -import com.google.common.collect.Sets; - import org.joda.time.Instant; import org.junit.Rule; import org.junit.Test; @@ -59,16 +66,6 @@ import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -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.List; -import java.util.NoSuchElementException; -import java.util.Random; - /** * Unit tests for {@link UnboundedReadFromBoundedSource}. */ diff --git a/runners/core-java/src/test/java/org/apache/beam/sdk/util/BatchTimerInternalsTest.java b/runners/core-java/src/test/java/org/apache/beam/sdk/util/BatchTimerInternalsTest.java index 3e1528f3ac503..20a9852b09580 100644 --- a/runners/core-java/src/test/java/org/apache/beam/sdk/util/BatchTimerInternalsTest.java +++ b/runners/core-java/src/test/java/org/apache/beam/sdk/util/BatchTimerInternalsTest.java @@ -20,7 +20,6 @@ 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.Before; import org.junit.Test; diff --git a/runners/core-java/src/test/java/org/apache/beam/sdk/util/GroupAlsoByWindowsProperties.java b/runners/core-java/src/test/java/org/apache/beam/sdk/util/GroupAlsoByWindowsProperties.java index 43c287e434155..215cd4c38cb31 100644 --- a/runners/core-java/src/test/java/org/apache/beam/sdk/util/GroupAlsoByWindowsProperties.java +++ b/runners/core-java/src/test/java/org/apache/beam/sdk/util/GroupAlsoByWindowsProperties.java @@ -23,6 +23,14 @@ import static org.hamcrest.Matchers.hasSize; import static org.junit.Assert.assertThat; +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.Arrays; +import java.util.Collection; +import java.util.Collections; import org.apache.beam.sdk.transforms.Combine.CombineFn; import org.apache.beam.sdk.transforms.DoFnTester; import org.apache.beam.sdk.transforms.DoFnTester.CloningBehavior; @@ -38,20 +46,9 @@ import org.apache.beam.sdk.util.state.StateInternalsFactory; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.TimestampedValue; - -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 org.joda.time.Duration; import org.joda.time.Instant; -import java.util.Arrays; -import java.util.Collection; -import java.util.Collections; - /** * Properties of {@link GroupAlsoByWindowsDoFn}. * diff --git a/runners/core-java/src/test/java/org/apache/beam/sdk/util/GroupAlsoByWindowsViaOutputBufferDoFnTest.java b/runners/core-java/src/test/java/org/apache/beam/sdk/util/GroupAlsoByWindowsViaOutputBufferDoFnTest.java index 1f02a8f997b99..a1586c8bf2d9d 100644 --- a/runners/core-java/src/test/java/org/apache/beam/sdk/util/GroupAlsoByWindowsViaOutputBufferDoFnTest.java +++ b/runners/core-java/src/test/java/org/apache/beam/sdk/util/GroupAlsoByWindowsViaOutputBufferDoFnTest.java @@ -22,7 +22,6 @@ import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.util.GroupAlsoByWindowsProperties.GroupAlsoByWindowsDoFnFactory; import org.apache.beam.sdk.util.state.StateInternalsFactory; - import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; diff --git a/runners/core-java/src/test/java/org/apache/beam/sdk/util/LateDataDroppingDoFnRunnerTest.java b/runners/core-java/src/test/java/org/apache/beam/sdk/util/LateDataDroppingDoFnRunnerTest.java index d929d39430116..c63e43ef55b03 100644 --- a/runners/core-java/src/test/java/org/apache/beam/sdk/util/LateDataDroppingDoFnRunnerTest.java +++ b/runners/core-java/src/test/java/org/apache/beam/sdk/util/LateDataDroppingDoFnRunnerTest.java @@ -22,16 +22,15 @@ import static org.junit.Assert.assertThat; import static org.mockito.Mockito.when; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.Iterables; +import java.util.Arrays; import org.apache.beam.sdk.transforms.Aggregator; import org.apache.beam.sdk.transforms.Combine.CombineFn; import org.apache.beam.sdk.transforms.Sum; import org.apache.beam.sdk.transforms.windowing.FixedWindows; import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.util.LateDataDroppingDoFnRunner.LateDataFilter; - -import com.google.common.collect.ImmutableList; -import com.google.common.collect.Iterables; - import org.joda.time.Duration; import org.joda.time.Instant; import org.junit.Before; @@ -41,8 +40,6 @@ import org.mockito.Mock; import org.mockito.MockitoAnnotations; -import java.util.Arrays; - /** * Unit tests for {@link LateDataDroppingDoFnRunner}. */ diff --git a/runners/core-java/src/test/java/org/apache/beam/sdk/util/PushbackSideInputDoFnRunnerTest.java b/runners/core-java/src/test/java/org/apache/beam/sdk/util/PushbackSideInputDoFnRunnerTest.java index 88851187bb65d..f8ad291d9bc95 100644 --- a/runners/core-java/src/test/java/org/apache/beam/sdk/util/PushbackSideInputDoFnRunnerTest.java +++ b/runners/core-java/src/test/java/org/apache/beam/sdk/util/PushbackSideInputDoFnRunnerTest.java @@ -24,6 +24,9 @@ import static org.junit.Assert.assertThat; import static org.mockito.Mockito.when; +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; @@ -34,9 +37,6 @@ import org.apache.beam.sdk.transforms.windowing.Window; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionView; - -import com.google.common.collect.ImmutableList; - import org.hamcrest.Matchers; import org.joda.time.Instant; import org.junit.Before; @@ -47,9 +47,6 @@ import org.mockito.Mockito; import org.mockito.MockitoAnnotations; -import java.util.ArrayList; -import java.util.List; - /** * Tests for {@link PushbackSideInputDoFnRunner}. */ diff --git a/runners/core-java/src/test/java/org/apache/beam/sdk/util/ReduceFnRunnerTest.java b/runners/core-java/src/test/java/org/apache/beam/sdk/util/ReduceFnRunnerTest.java index 8d604cb2b9e03..647495cc344a3 100644 --- a/runners/core-java/src/test/java/org/apache/beam/sdk/util/ReduceFnRunnerTest.java +++ b/runners/core-java/src/test/java/org/apache/beam/sdk/util/ReduceFnRunnerTest.java @@ -17,11 +17,9 @@ */ package org.apache.beam.sdk.util; +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 com.google.common.base.Preconditions.checkArgument; - import static org.hamcrest.Matchers.contains; import static org.hamcrest.Matchers.containsInAnyOrder; import static org.hamcrest.Matchers.emptyIterable; @@ -37,6 +35,9 @@ import static org.mockito.Mockito.when; import static org.mockito.Mockito.withSettings; +import com.google.common.collect.Iterables; +import java.util.Iterator; +import java.util.List; import org.apache.beam.sdk.WindowMatchers; import org.apache.beam.sdk.coders.VarIntCoder; import org.apache.beam.sdk.options.PipelineOptions; @@ -67,9 +68,6 @@ import org.apache.beam.sdk.util.WindowingStrategy.AccumulationMode; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TimestampedValue; - -import com.google.common.collect.Iterables; - import org.joda.time.Duration; import org.joda.time.Instant; import org.junit.Before; @@ -83,9 +81,6 @@ import org.mockito.invocation.InvocationOnMock; import org.mockito.stubbing.Answer; -import java.util.Iterator; -import java.util.List; - /** * Tests for {@link ReduceFnRunner}. These tests instantiate a full "stack" of * {@link ReduceFnRunner} with enclosed {@link ReduceFn}, down to the installed {@link Trigger} diff --git a/runners/core-java/src/test/java/org/apache/beam/sdk/util/ReduceFnTester.java b/runners/core-java/src/test/java/org/apache/beam/sdk/util/ReduceFnTester.java index feba191267e4f..24e33ddf310ce 100644 --- a/runners/core-java/src/test/java/org/apache/beam/sdk/util/ReduceFnTester.java +++ b/runners/core-java/src/test/java/org/apache/beam/sdk/util/ReduceFnTester.java @@ -19,11 +19,30 @@ import static com.google.common.base.Preconditions.checkNotNull; import static com.google.common.base.Preconditions.checkState; - import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; +import com.google.common.base.Function; +import com.google.common.base.MoreObjects; +import com.google.common.collect.FluentIterable; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableSet; +import com.google.common.collect.Iterables; +import com.google.common.collect.Sets; +import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; +import java.util.PriorityQueue; +import java.util.Set; +import javax.annotation.Nullable; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.CoderRegistry; import org.apache.beam.sdk.coders.IterableCoder; @@ -57,33 +76,9 @@ import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TimestampedValue; import org.apache.beam.sdk.values.TupleTag; - -import com.google.common.base.Function; -import com.google.common.base.MoreObjects; -import com.google.common.collect.FluentIterable; -import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableSet; -import com.google.common.collect.Iterables; -import com.google.common.collect.Sets; - import org.joda.time.Duration; import org.joda.time.Instant; -import java.io.IOException; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collection; -import java.util.Collections; -import java.util.HashMap; -import java.util.HashSet; -import java.util.List; -import java.util.Map; -import java.util.Map.Entry; -import java.util.PriorityQueue; -import java.util.Set; - -import javax.annotation.Nullable; - /** * Test utility that runs a {@link ReduceFn}, {@link WindowFn}, {@link Trigger} using in-memory stub * implementations to provide the {@link TimerInternals} and {@link WindowingInternals} needed to diff --git a/runners/core-java/src/test/java/org/apache/beam/sdk/util/SimpleDoFnRunnerTest.java b/runners/core-java/src/test/java/org/apache/beam/sdk/util/SimpleDoFnRunnerTest.java index f0c52b99a377a..156b4a92ca17b 100644 --- a/runners/core-java/src/test/java/org/apache/beam/sdk/util/SimpleDoFnRunnerTest.java +++ b/runners/core-java/src/test/java/org/apache/beam/sdk/util/SimpleDoFnRunnerTest.java @@ -20,19 +20,17 @@ import static org.hamcrest.Matchers.is; import static org.mockito.Mockito.mock; +import java.util.Arrays; +import java.util.List; import org.apache.beam.sdk.transforms.OldDoFn; import org.apache.beam.sdk.util.BaseExecutionContext.StepContext; import org.apache.beam.sdk.values.TupleTag; - import org.junit.Rule; import org.junit.Test; import org.junit.rules.ExpectedException; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.util.Arrays; -import java.util.List; - /** * Tests for base {@link DoFnRunnerBase} functionality. */ 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 75e6558bb7ee4..06490dc553556 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 @@ -17,10 +17,6 @@ */ package org.apache.beam.runners.direct; -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; import com.google.auto.value.AutoValue; import com.google.common.base.Preconditions; import java.util.Arrays; @@ -30,6 +26,10 @@ import java.util.concurrent.ConcurrentMap; import javax.annotation.Nullable; import javax.annotation.concurrent.GuardedBy; +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; /** * 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/BoundedReadEvaluatorFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/BoundedReadEvaluatorFactory.java index 0c4b7fd9bc800..2b15ad002eb51 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 @@ -17,6 +17,12 @@ */ package org.apache.beam.runners.direct; +import java.io.IOException; +import java.util.Queue; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentLinkedQueue; +import java.util.concurrent.ConcurrentMap; +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.io.BoundedSource; @@ -28,14 +34,6 @@ import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.PCollection; -import java.io.IOException; -import java.util.Queue; -import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.ConcurrentLinkedQueue; -import java.util.concurrent.ConcurrentMap; - -import javax.annotation.Nullable; - /** * A {@link TransformEvaluatorFactory} that produces {@link TransformEvaluator TransformEvaluators} * for the {@link Bounded Read.Bounded} primitive {@link PTransform}. 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 7e0cd8e1d49a0..5fcf7b313b7f1 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 @@ -18,15 +18,12 @@ package org.apache.beam.runners.direct; -import org.apache.beam.runners.direct.DirectRunner.CommittedBundle; -import org.apache.beam.sdk.transforms.AppliedPTransform; -import org.apache.beam.sdk.transforms.View.CreatePCollectionView; - import com.google.auto.value.AutoValue; - import java.util.Set; - import javax.annotation.Nullable; +import org.apache.beam.runners.direct.DirectRunner.CommittedBundle; +import org.apache.beam.sdk.transforms.AppliedPTransform; +import org.apache.beam.sdk.transforms.View.CreatePCollectionView; /** * A {@link TransformResult} that has been committed. 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 3300723ad298f..4fdfea0717021 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,6 +19,12 @@ import static com.google.common.base.Preconditions.checkState; +import java.util.ArrayList; +import java.util.Collection; +import java.util.HashMap; +import java.util.HashSet; +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.runners.PipelineRunner; @@ -29,13 +35,6 @@ import org.apache.beam.sdk.values.PInput; import org.apache.beam.sdk.values.PValue; -import java.util.ArrayList; -import java.util.Collection; -import java.util.HashMap; -import java.util.HashSet; -import java.util.Map; -import java.util.Set; - /** * Tracks the {@link AppliedPTransform AppliedPTransforms} that consume each {@link PValue} in the * {@link Pipeline}. This is used to schedule consuming {@link PTransform PTransforms} to consume diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectRegistrar.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectRegistrar.java index 772777e0c8357..9a34d3d9dfb44 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectRegistrar.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectRegistrar.java @@ -17,14 +17,13 @@ */ package org.apache.beam.runners.direct; +import com.google.auto.service.AutoService; +import com.google.common.collect.ImmutableList; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.options.PipelineOptionsRegistrar; import org.apache.beam.sdk.runners.PipelineRunner; import org.apache.beam.sdk.runners.PipelineRunnerRegistrar; -import com.google.auto.service.AutoService; -import com.google.common.collect.ImmutableList; - /** * Contains the {@link PipelineRunnerRegistrar} and {@link PipelineOptionsRegistrar} for the * {@link org.apache.beam.runners.direct.DirectRunner}. 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 68184dec7d852..b2d61c30aa3d2 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 @@ -17,6 +17,17 @@ */ package org.apache.beam.runners.direct; +import com.google.common.base.MoreObjects; +import com.google.common.base.Supplier; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.ImmutableSet; +import java.io.IOException; +import java.util.Collection; +import java.util.HashMap; +import java.util.Map; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; import org.apache.beam.runners.direct.DirectGroupByKey.DirectGroupByKeyOnly; import org.apache.beam.runners.direct.DirectRunner.DirectPipelineResult; import org.apache.beam.runners.direct.TestStreamEvaluatorFactory.DirectTestStreamFactory; @@ -46,23 +57,9 @@ import org.apache.beam.sdk.values.PInput; import org.apache.beam.sdk.values.POutput; import org.apache.beam.sdk.values.PValue; - -import com.google.common.base.MoreObjects; -import com.google.common.base.Supplier; -import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableMap; -import com.google.common.collect.ImmutableSet; - import org.joda.time.Duration; import org.joda.time.Instant; -import java.io.IOException; -import java.util.Collection; -import java.util.HashMap; -import java.util.Map; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.Executors; - /** * An In-Memory implementation of the Dataflow Programming Model. Supports Unbounded * {@link PCollection PCollections}. 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 a4705ddce40d5..4003983613851 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 @@ -17,15 +17,13 @@ */ package org.apache.beam.runners.direct; +import javax.annotation.Nullable; import org.apache.beam.runners.direct.WatermarkManager.TimerUpdate; import org.apache.beam.runners.direct.WatermarkManager.TimerUpdate.TimerUpdateBuilder; import org.apache.beam.runners.direct.WatermarkManager.TransformWatermarks; import org.apache.beam.sdk.util.TimerInternals; - import org.joda.time.Instant; -import javax.annotation.Nullable; - /** * An implementation of {@link TimerInternals} where all relevant data exists in memory. */ 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 3f4f2c6a880d6..0e15c18c81100 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 @@ -18,21 +18,18 @@ package org.apache.beam.runners.direct; -import org.apache.beam.sdk.runners.PipelineRunner; -import org.apache.beam.sdk.transforms.DoFn; -import org.apache.beam.sdk.transforms.OldDoFn; -import org.apache.beam.sdk.util.SerializableUtils; - import com.google.common.cache.CacheBuilder; import com.google.common.cache.CacheLoader; import com.google.common.cache.LoadingCache; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - import java.util.ArrayList; import java.util.Collection; import java.util.Iterator; +import org.apache.beam.sdk.runners.PipelineRunner; +import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.transforms.OldDoFn; +import org.apache.beam.sdk.util.SerializableUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * Manages {@link DoFn} setup, teardown, and serialization. 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 523273cdd1ecd..faa06151c31b7 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 @@ -19,7 +19,6 @@ package org.apache.beam.runners.direct; import org.apache.beam.sdk.util.WindowedValue; - import org.slf4j.Logger; import org.slf4j.LoggerFactory; 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 b9f159a2d5988..5af25bc5e8067 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 @@ -19,6 +19,17 @@ import static com.google.common.base.Preconditions.checkNotNull; +import com.google.common.annotations.VisibleForTesting; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.Iterables; +import com.google.common.util.concurrent.MoreExecutors; +import java.util.Collection; +import java.util.EnumSet; +import java.util.List; +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; +import javax.annotation.Nullable; import org.apache.beam.runners.direct.CommittedResult.OutputType; import org.apache.beam.runners.direct.DirectGroupByKey.DirectGroupByKeyOnly; import org.apache.beam.runners.direct.DirectRunner.CommittedBundle; @@ -42,23 +53,8 @@ import org.apache.beam.sdk.values.PCollection.IsBounded; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.PValue; - -import com.google.common.annotations.VisibleForTesting; -import com.google.common.collect.ImmutableList; -import com.google.common.collect.Iterables; -import com.google.common.util.concurrent.MoreExecutors; - import org.joda.time.Instant; -import java.util.Collection; -import java.util.EnumSet; -import java.util.List; -import java.util.Map; -import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.ConcurrentMap; - -import javax.annotation.Nullable; - /** * The evaluation context for a specific pipeline being executed by the * {@link DirectRunner}. Contains state shared within the execution across all 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 35b6239de81c7..401ed7fb80c45 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 @@ -17,20 +17,6 @@ */ package org.apache.beam.runners.direct; -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.TimeDomain; -import org.apache.beam.sdk.util.TimerInternals.TimerData; -import org.apache.beam.sdk.util.WindowedValue; -import org.apache.beam.sdk.values.PCollection; -import org.apache.beam.sdk.values.PCollectionView; -import org.apache.beam.sdk.values.PValue; - import com.google.auto.value.AutoValue; import com.google.common.base.MoreObjects; import com.google.common.base.Optional; @@ -39,10 +25,6 @@ import com.google.common.cache.LoadingCache; import com.google.common.collect.ImmutableList; import com.google.common.collect.Iterables; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - import java.util.ArrayList; import java.util.Collection; import java.util.Collections; @@ -56,8 +38,22 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicLong; import java.util.concurrent.atomic.AtomicReference; - import javax.annotation.Nullable; +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.TimeDomain; +import org.apache.beam.sdk.util.TimerInternals.TimerData; +import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.PCollectionView; +import org.apache.beam.sdk.values.PValue; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * An {@link PipelineExecutor} that uses an underlying {@link ExecutorService} and 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 d16ffa0bf5171..2da70bbe56289 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 @@ -17,6 +17,8 @@ */ package org.apache.beam.runners.direct; +import com.google.common.collect.ImmutableMap; +import java.util.Collections; import org.apache.beam.runners.core.GroupAlsoByWindowViaWindowSetDoFn; import org.apache.beam.runners.direct.DirectExecutionContext.DirectStepContext; import org.apache.beam.runners.direct.DirectGroupByKey.DirectGroupAlsoByWindow; @@ -40,10 +42,6 @@ import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TupleTag; -import com.google.common.collect.ImmutableMap; - -import java.util.Collections; - /** * The {@link DirectRunner} {@link TransformEvaluatorFactory} for the * {@link GroupByKeyOnly} {@link PTransform}. 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 dbdbdaf4ea348..f085a39fda9bf 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 @@ -17,9 +17,14 @@ */ package org.apache.beam.runners.direct; -import static org.apache.beam.sdk.util.CoderUtils.encodeToByteArray; import static com.google.common.base.Preconditions.checkState; +import static org.apache.beam.sdk.util.CoderUtils.encodeToByteArray; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.HashMap; +import java.util.List; +import java.util.Map; import org.apache.beam.runners.direct.DirectGroupByKey.DirectGroupByKeyOnly; import org.apache.beam.runners.direct.DirectRunner.CommittedBundle; import org.apache.beam.runners.direct.DirectRunner.UncommittedBundle; @@ -37,12 +42,6 @@ import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.HashMap; -import java.util.List; -import java.util.Map; - /** * The {@link DirectRunner} {@link TransformEvaluatorFactory} for the * {@link GroupByKeyOnly} {@link PTransform}. 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 8be12fdb36734..d5c0f0c354e60 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 @@ -19,6 +19,8 @@ import static com.google.common.base.Preconditions.checkNotNull; +import com.google.common.collect.HashMultimap; +import com.google.common.collect.SetMultimap; import org.apache.beam.runners.direct.DirectRunner.CommittedBundle; import org.apache.beam.runners.direct.DirectRunner.UncommittedBundle; import org.apache.beam.sdk.coders.Coder; @@ -29,10 +31,6 @@ import org.apache.beam.sdk.util.MutationDetectors; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.PCollection; - -import com.google.common.collect.HashMultimap; -import com.google.common.collect.SetMultimap; - import org.joda.time.Instant; /** 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 b0eb38f13addc..1602f68ab3a90 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 @@ -17,6 +17,8 @@ */ package org.apache.beam.runners.direct; +import java.util.IdentityHashMap; +import java.util.Map; import org.apache.beam.runners.direct.DirectRunner.CommittedBundle; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.CoderException; @@ -27,9 +29,6 @@ import org.apache.beam.sdk.util.UserCodeException; import org.apache.beam.sdk.util.WindowedValue; -import java.util.IdentityHashMap; -import java.util.Map; - /** * {@link ModelEnforcement} that enforces elements are not modified over the course of processing * an element. diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ImmutableListBundleFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ImmutableListBundleFactory.java index 25a0d05cb128d..e79da7baa751b 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ImmutableListBundleFactory.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ImmutableListBundleFactory.java @@ -19,15 +19,13 @@ import static com.google.common.base.Preconditions.checkState; +import com.google.common.base.MoreObjects; +import com.google.common.collect.ImmutableList; import org.apache.beam.runners.direct.DirectRunner.CommittedBundle; import org.apache.beam.runners.direct.DirectRunner.UncommittedBundle; import org.apache.beam.sdk.coders.VoidCoder; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.PCollection; - -import com.google.common.base.MoreObjects; -import com.google.common.collect.ImmutableList; - import org.joda.time.Instant; /** 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 2fea00a6e5d4b..7c4376a2c4b4c 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 @@ -19,15 +19,14 @@ import static com.google.common.base.Preconditions.checkState; +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.transforms.GroupByKey; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.values.PValue; -import java.util.HashSet; -import java.util.Set; - /** * A pipeline visitor that tracks all keyed {@link PValue PValues}. A {@link PValue} is keyed if it * is the result of a {@link PTransform} that produces keyed outputs. A {@link PTransform} that diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/NanosOffsetClock.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/NanosOffsetClock.java index 77fa1967c4345..5a2b18da0f244 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/NanosOffsetClock.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/NanosOffsetClock.java @@ -17,9 +17,8 @@ */ package org.apache.beam.runners.direct; -import org.joda.time.Instant; - import java.util.concurrent.TimeUnit; +import org.joda.time.Instant; /** * A {@link Clock} that uses {@link System#nanoTime()} to track the progress of time. 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 6ef0ffe29c4a0..85a1c6af13858 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 @@ -17,6 +17,12 @@ */ package org.apache.beam.runners.direct; +import com.google.common.collect.ImmutableList; +import java.util.ArrayList; +import java.util.Collection; +import java.util.HashMap; +import java.util.List; +import java.util.Map; import org.apache.beam.runners.direct.DirectExecutionContext.DirectStepContext; import org.apache.beam.runners.direct.DirectRunner.CommittedBundle; import org.apache.beam.runners.direct.DirectRunner.UncommittedBundle; @@ -34,14 +40,6 @@ import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TupleTag; -import com.google.common.collect.ImmutableList; - -import java.util.ArrayList; -import java.util.Collection; -import java.util.HashMap; -import java.util.List; -import java.util.Map; - class ParDoEvaluator implements TransformEvaluator { public static ParDoEvaluator create( EvaluationContext evaluationContext, diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoMultiEvaluatorFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoMultiEvaluatorFactory.java index 2d05e68b66595..6a41adffe6a92 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoMultiEvaluatorFactory.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoMultiEvaluatorFactory.java @@ -17,6 +17,10 @@ */ package org.apache.beam.runners.direct; +import com.google.common.cache.CacheBuilder; +import com.google.common.cache.CacheLoader; +import com.google.common.cache.LoadingCache; +import java.util.Map; import org.apache.beam.runners.direct.DirectExecutionContext.DirectStepContext; import org.apache.beam.runners.direct.DirectRunner.CommittedBundle; import org.apache.beam.sdk.transforms.AppliedPTransform; @@ -26,16 +30,9 @@ import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionTuple; import org.apache.beam.sdk.values.TupleTag; - -import com.google.common.cache.CacheBuilder; -import com.google.common.cache.CacheLoader; -import com.google.common.cache.LoadingCache; - import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.util.Map; - /** * The {@link DirectRunner} {@link TransformEvaluatorFactory} for the * {@link BoundMulti} primitive {@link PTransform}. diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoSingleEvaluatorFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoSingleEvaluatorFactory.java index 97cbfa7f7b2ff..4bb740be012f4 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoSingleEvaluatorFactory.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoSingleEvaluatorFactory.java @@ -17,6 +17,11 @@ */ package org.apache.beam.runners.direct; +import com.google.common.cache.CacheBuilder; +import com.google.common.cache.CacheLoader; +import com.google.common.cache.LoadingCache; +import com.google.common.collect.ImmutableMap; +import java.util.Collections; import org.apache.beam.runners.direct.DirectExecutionContext.DirectStepContext; import org.apache.beam.runners.direct.DirectRunner.CommittedBundle; import org.apache.beam.sdk.transforms.AppliedPTransform; @@ -25,17 +30,9 @@ import org.apache.beam.sdk.transforms.ParDo.Bound; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.TupleTag; - -import com.google.common.cache.CacheBuilder; -import com.google.common.cache.CacheLoader; -import com.google.common.cache.LoadingCache; -import com.google.common.collect.ImmutableMap; - import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.util.Collections; - /** * The {@link DirectRunner} {@link TransformEvaluatorFactory} for the * {@link Bound ParDo.Bound} primitive {@link PTransform}. diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/PipelineExecutor.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/PipelineExecutor.java index 76df11cc2588f..01a5c54831638 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/PipelineExecutor.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/PipelineExecutor.java @@ -17,12 +17,11 @@ */ package org.apache.beam.runners.direct; +import java.util.Collection; import org.apache.beam.runners.direct.DirectRunner.CommittedBundle; import org.apache.beam.sdk.transforms.AppliedPTransform; import org.apache.beam.sdk.transforms.PTransform; -import java.util.Collection; - /** * An executor that schedules and executes {@link AppliedPTransform AppliedPTransforms} for both * source and intermediate {@link PTransform PTransforms}. diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/SideInputContainer.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/SideInputContainer.java index 6458215503b94..cd459e4c0f5d7 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/SideInputContainer.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/SideInputContainer.java @@ -19,15 +19,6 @@ import static com.google.common.base.Preconditions.checkArgument; -import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.transforms.windowing.PaneInfo; -import org.apache.beam.sdk.util.PCollectionViewWindow; -import org.apache.beam.sdk.util.ReadyCheckingSideInputReader; -import org.apache.beam.sdk.util.SideInputReader; -import org.apache.beam.sdk.util.WindowedValue; -import org.apache.beam.sdk.util.WindowingStrategy; -import org.apache.beam.sdk.values.PCollectionView; - import com.google.common.base.MoreObjects; import com.google.common.base.Optional; import com.google.common.cache.CacheBuilder; @@ -36,7 +27,6 @@ import com.google.common.collect.ImmutableSet; import com.google.common.collect.Iterables; import com.google.common.collect.Sets; - import java.util.ArrayList; import java.util.Collection; import java.util.Collections; @@ -44,8 +34,15 @@ import java.util.Map; import java.util.Set; import java.util.concurrent.atomic.AtomicReference; - import javax.annotation.Nullable; +import org.apache.beam.sdk.transforms.windowing.BoundedWindow; +import org.apache.beam.sdk.transforms.windowing.PaneInfo; +import org.apache.beam.sdk.util.PCollectionViewWindow; +import org.apache.beam.sdk.util.ReadyCheckingSideInputReader; +import org.apache.beam.sdk.util.SideInputReader; +import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.util.WindowingStrategy; +import org.apache.beam.sdk.values.PCollectionView; /** * An in-process container for {@link PCollectionView PCollectionViews}, which provides methods for diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/StepAndKey.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/StepAndKey.java index 18fe04ffe9438..e18b2ac9b6aa6 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/StepAndKey.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/StepAndKey.java @@ -17,11 +17,9 @@ */ package org.apache.beam.runners.direct; -import org.apache.beam.sdk.transforms.AppliedPTransform; - import com.google.common.base.MoreObjects; - import java.util.Objects; +import org.apache.beam.sdk.transforms.AppliedPTransform; /** * A (Step, Key) pair. This is useful as a map key or cache key for things that are available 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 12b18cbb8a7ab..1829e4aea92db 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 @@ -17,6 +17,12 @@ */ package org.apache.beam.runners.direct; +import com.google.auto.value.AutoValue; +import com.google.common.collect.ImmutableList; +import java.util.Collection; +import java.util.EnumSet; +import java.util.Set; +import javax.annotation.Nullable; import org.apache.beam.runners.direct.CommittedResult.OutputType; import org.apache.beam.runners.direct.DirectRunner.UncommittedBundle; import org.apache.beam.runners.direct.WatermarkManager.TimerUpdate; @@ -24,18 +30,8 @@ import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.state.CopyOnAccessInMemoryStateInternals; - -import com.google.auto.value.AutoValue; -import com.google.common.collect.ImmutableList; - import org.joda.time.Instant; -import java.util.Collection; -import java.util.EnumSet; -import java.util.Set; - -import javax.annotation.Nullable; - /** * An immutable {@link TransformResult}. */ 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 90a83b0b840f4..e9f37bab7567d 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 @@ -20,6 +20,11 @@ import static com.google.common.base.Preconditions.checkState; +import com.google.common.base.Supplier; +import java.util.List; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicReference; +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.Pipeline; @@ -41,18 +46,9 @@ import org.apache.beam.sdk.values.PInput; import org.apache.beam.sdk.values.POutput; import org.apache.beam.sdk.values.TimestampedValue; - -import com.google.common.base.Supplier; - import org.joda.time.Duration; import org.joda.time.Instant; -import java.util.List; -import java.util.concurrent.atomic.AtomicBoolean; -import java.util.concurrent.atomic.AtomicReference; - -import javax.annotation.Nullable; - /** * The {@link TransformEvaluatorFactory} for the {@link TestStream} primitive. */ diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TransformEvaluatorFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TransformEvaluatorFactory.java index 3655d26dd553c..e9fa06b107f70 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TransformEvaluatorFactory.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TransformEvaluatorFactory.java @@ -17,6 +17,7 @@ */ package org.apache.beam.runners.direct; +import javax.annotation.Nullable; import org.apache.beam.runners.direct.DirectRunner.CommittedBundle; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.io.Read; @@ -24,8 +25,6 @@ import org.apache.beam.sdk.transforms.OldDoFn; import org.apache.beam.sdk.transforms.PTransform; -import javax.annotation.Nullable; - /** * A factory for creating instances of {@link TransformEvaluator} for the application of a * {@link PTransform}. 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 c35e8b14c84a2..9edc50f8f4c6f 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 @@ -19,6 +19,12 @@ import static com.google.common.base.Preconditions.checkState; +import com.google.common.collect.ImmutableMap; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Map; +import java.util.concurrent.atomic.AtomicBoolean; +import javax.annotation.Nullable; import org.apache.beam.runners.direct.DirectGroupByKey.DirectGroupAlsoByWindow; import org.apache.beam.runners.direct.DirectGroupByKey.DirectGroupByKeyOnly; import org.apache.beam.runners.direct.DirectRunner.CommittedBundle; @@ -29,19 +35,9 @@ import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.ParDo; import org.apache.beam.sdk.transforms.windowing.Window; - -import com.google.common.collect.ImmutableMap; - import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.util.ArrayList; -import java.util.Collection; -import java.util.Map; -import java.util.concurrent.atomic.AtomicBoolean; - -import javax.annotation.Nullable; - /** * A {@link TransformEvaluatorFactory} that delegates to primitive {@link TransformEvaluatorFactory} * implementations based on the type of {@link PTransform} of the application. 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 d873bf581a8a2..cc6b5b7a93cab 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 @@ -19,16 +19,14 @@ import static com.google.common.base.Preconditions.checkState; -import org.apache.beam.runners.direct.DirectRunner.CommittedBundle; -import org.apache.beam.sdk.transforms.AppliedPTransform; -import org.apache.beam.sdk.util.WindowedValue; - import java.util.ArrayList; import java.util.Collection; import java.util.concurrent.Callable; import java.util.concurrent.atomic.AtomicReference; - import javax.annotation.Nullable; +import org.apache.beam.runners.direct.DirectRunner.CommittedBundle; +import org.apache.beam.sdk.transforms.AppliedPTransform; +import org.apache.beam.sdk.util.WindowedValue; /** * A {@link Callable} responsible for constructing a {@link TransformEvaluator} from a diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TransformExecutorServices.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TransformExecutorServices.java index ea15f03df1b57..876da9d5cd2ea 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TransformExecutorServices.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TransformExecutorServices.java @@ -18,7 +18,6 @@ package org.apache.beam.runners.direct; import com.google.common.base.MoreObjects; - import java.util.Queue; import java.util.concurrent.ConcurrentLinkedQueue; import java.util.concurrent.ExecutorService; 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 c01fa5647a92d..0b08294ae4b4a 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 @@ -17,6 +17,8 @@ */ package org.apache.beam.runners.direct; +import java.util.Set; +import javax.annotation.Nullable; import org.apache.beam.runners.direct.CommittedResult.OutputType; import org.apache.beam.runners.direct.DirectRunner.UncommittedBundle; import org.apache.beam.runners.direct.WatermarkManager.TimerUpdate; @@ -25,13 +27,8 @@ 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; -import java.util.Set; - -import javax.annotation.Nullable; - /** * The result of evaluating an {@link AppliedPTransform} with a {@link TransformEvaluator}. */ diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/UnboundedReadDeduplicator.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/UnboundedReadDeduplicator.java index 0246236e2b4f3..2371d3b9d3da5 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/UnboundedReadDeduplicator.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/UnboundedReadDeduplicator.java @@ -17,18 +17,15 @@ */ package org.apache.beam.runners.direct; -import org.apache.beam.sdk.coders.ByteArrayCoder; -import org.apache.beam.sdk.io.Read.Unbounded; -import org.apache.beam.sdk.transforms.PTransform; - import com.google.common.cache.CacheBuilder; import com.google.common.cache.CacheLoader; import com.google.common.cache.LoadingCache; - -import org.joda.time.Duration; - import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; +import org.apache.beam.sdk.coders.ByteArrayCoder; +import org.apache.beam.sdk.io.Read.Unbounded; +import org.apache.beam.sdk.transforms.PTransform; +import org.joda.time.Duration; /** * Provides methods to determine if a record is a duplicate within the evaluation of a 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 c4d408b232f04..9f485e0f65ab3 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 @@ -17,6 +17,12 @@ */ package org.apache.beam.runners.direct; +import com.google.common.annotations.VisibleForTesting; +import java.io.IOException; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentLinkedQueue; +import java.util.concurrent.ConcurrentMap; +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.io.Read.Unbounded; @@ -29,18 +35,8 @@ import org.apache.beam.sdk.transforms.windowing.GlobalWindow; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.PCollection; - -import com.google.common.annotations.VisibleForTesting; - import org.joda.time.Instant; -import java.io.IOException; -import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.ConcurrentLinkedQueue; -import java.util.concurrent.ConcurrentMap; - -import javax.annotation.Nullable; - /** * A {@link TransformEvaluatorFactory} that produces {@link TransformEvaluator TransformEvaluators} * for the {@link Unbounded Read.Unbounded} primitive {@link PTransform}. diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/UncommittedBundleOutputManager.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/UncommittedBundleOutputManager.java index 570dc9031e974..41f7e8d64b396 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/UncommittedBundleOutputManager.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/UncommittedBundleOutputManager.java @@ -17,14 +17,13 @@ */ package org.apache.beam.runners.direct; +import java.util.Map; import org.apache.beam.runners.direct.DirectRunner.CommittedBundle; import org.apache.beam.runners.direct.DirectRunner.UncommittedBundle; import org.apache.beam.sdk.util.DoFnRunners.OutputManager; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.TupleTag; -import java.util.Map; - /** * An {@link OutputManager} that outputs to {@link CommittedBundle Bundles} used by the * {@link 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 3b0de4b6f8df0..40ac7f09ac2ca 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 @@ -17,6 +17,8 @@ */ package org.apache.beam.runners.direct; +import java.util.ArrayList; +import java.util.List; import org.apache.beam.runners.direct.CommittedResult.OutputType; import org.apache.beam.runners.direct.DirectRunner.PCollectionViewWriter; import org.apache.beam.runners.direct.StepTransformResult.Builder; @@ -34,9 +36,6 @@ import org.apache.beam.sdk.values.PInput; import org.apache.beam.sdk.values.POutput; -import java.util.ArrayList; -import java.util.List; - /** * The {@link DirectRunner} {@link TransformEvaluatorFactory} for the * {@link CreatePCollectionView} primitive {@link PTransform}. 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 0f73b1da93494..7961f24c6125c 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 @@ -17,19 +17,16 @@ */ package org.apache.beam.runners.direct; -import org.apache.beam.sdk.transforms.AppliedPTransform; -import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.WindowingStrategy; - import com.google.common.collect.ComparisonChain; import com.google.common.collect.Ordering; - -import org.joda.time.Instant; - import java.util.PriorityQueue; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; import java.util.concurrent.Executor; +import org.apache.beam.sdk.transforms.AppliedPTransform; +import org.apache.beam.sdk.transforms.windowing.BoundedWindow; +import org.apache.beam.sdk.util.WindowingStrategy; +import org.joda.time.Instant; /** * Executes callbacks that occur based on the progression of the watermark per-step. 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 a44fa501c434f..ff7428dda2cc4 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 @@ -19,18 +19,6 @@ import static com.google.common.base.Preconditions.checkNotNull; -import org.apache.beam.runners.direct.DirectRunner.CommittedBundle; -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.transforms.windowing.BoundedWindow; -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.WindowedValue; -import org.apache.beam.sdk.values.PCollection; -import org.apache.beam.sdk.values.PValue; - import com.google.auto.value.AutoValue; import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Function; @@ -43,9 +31,6 @@ import com.google.common.collect.Ordering; import com.google.common.collect.SortedMultiset; import com.google.common.collect.TreeMultiset; - -import org.joda.time.Instant; - import java.util.ArrayList; import java.util.Collection; import java.util.Collections; @@ -61,8 +46,19 @@ import java.util.TreeSet; import java.util.concurrent.ConcurrentLinkedQueue; import java.util.concurrent.atomic.AtomicReference; - import javax.annotation.Nullable; +import org.apache.beam.runners.direct.DirectRunner.CommittedBundle; +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.transforms.windowing.BoundedWindow; +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.WindowedValue; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.PValue; +import org.joda.time.Instant; /** * Manages watermarks of {@link PCollection PCollections} and input and output watermarks of 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 f2e62cb0445d8..19c1a98e7ef50 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 @@ -17,6 +17,9 @@ */ package org.apache.beam.runners.direct; +import com.google.common.collect.Iterables; +import java.util.Collection; +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.transforms.AppliedPTransform; @@ -28,15 +31,8 @@ import org.apache.beam.sdk.transforms.windowing.WindowFn; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.PCollection; - -import com.google.common.collect.Iterables; - import org.joda.time.Instant; -import java.util.Collection; - -import javax.annotation.Nullable; - /** * The {@link DirectRunner} {@link TransformEvaluatorFactory} for the * {@link Bound Window.Bound} primitive {@link PTransform}. 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 1ab3403f45bc2..d74cd56735b61 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 @@ -20,6 +20,8 @@ import static com.google.common.base.Preconditions.checkArgument; +import com.google.common.annotations.VisibleForTesting; +import java.util.concurrent.ThreadLocalRandom; import org.apache.beam.sdk.io.Write; import org.apache.beam.sdk.io.Write.Bound; import org.apache.beam.sdk.transforms.Count; @@ -39,13 +41,8 @@ import org.apache.beam.sdk.values.PDone; import org.apache.beam.sdk.values.PInput; import org.apache.beam.sdk.values.POutput; - -import com.google.common.annotations.VisibleForTesting; - import org.joda.time.Duration; -import java.util.concurrent.ThreadLocalRandom; - /** * A {@link PTransformOverrideFactory} that overrides {@link Write} {@link PTransform PTransforms} * with an unspecified number of shards with a write with a specified number of shards. The number 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 b131b4c524c25..c8310c90e79a8 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 @@ -21,10 +21,12 @@ import static org.junit.Assert.assertThat; import static org.mockito.Mockito.when; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.TimeUnit; 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; @@ -34,10 +36,6 @@ import org.mockito.Mock; import org.mockito.MockitoAnnotations; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.Executors; -import java.util.concurrent.TimeUnit; - /** * Tests for {@link AggregatorContainer}. */ 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 9bc4f7b7904b3..cbeb733a1daf5 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 @@ -25,6 +25,11 @@ import static org.junit.Assert.assertThat; import static org.mockito.Mockito.when; +import com.google.common.collect.ImmutableList; +import java.io.IOException; +import java.util.Arrays; +import java.util.List; +import java.util.NoSuchElementException; import org.apache.beam.runners.direct.DirectRunner.CommittedBundle; import org.apache.beam.runners.direct.DirectRunner.UncommittedBundle; import org.apache.beam.sdk.coders.BigEndianLongCoder; @@ -40,9 +45,6 @@ import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.PCollection; - -import com.google.common.collect.ImmutableList; - import org.joda.time.Instant; import org.junit.Before; import org.junit.Test; @@ -51,11 +53,6 @@ import org.mockito.Mock; import org.mockito.MockitoAnnotations; -import java.io.IOException; -import java.util.Arrays; -import java.util.List; -import java.util.NoSuchElementException; - /** * Tests for {@link BoundedReadEvaluatorFactory}. */ 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 a8c647e64682f..efc6d2f81abdb 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 @@ -21,6 +21,11 @@ import static org.hamcrest.Matchers.nullValue; import static org.junit.Assert.assertThat; +import com.google.common.collect.ImmutableList; +import java.io.Serializable; +import java.util.Collections; +import java.util.EnumSet; +import java.util.List; import org.apache.beam.runners.direct.CommittedResult.OutputType; import org.apache.beam.sdk.testing.TestPipeline; import org.apache.beam.sdk.transforms.AppliedPTransform; @@ -31,20 +36,12 @@ import org.apache.beam.sdk.values.PBegin; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PDone; - -import com.google.common.collect.ImmutableList; - import org.hamcrest.Matchers; import org.joda.time.Instant; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.io.Serializable; -import java.util.Collections; -import java.util.EnumSet; -import java.util.List; - /** * Tests for {@link CommittedResult}. */ 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 529316cd30275..1c9b5a6da6d63 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 @@ -20,6 +20,8 @@ import static org.hamcrest.Matchers.emptyIterable; import static org.junit.Assert.assertThat; +import java.io.Serializable; +import java.util.List; import org.apache.beam.sdk.io.CountingInput; import org.apache.beam.sdk.testing.TestPipeline; import org.apache.beam.sdk.transforms.AppliedPTransform; @@ -35,7 +37,6 @@ 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; @@ -43,9 +44,6 @@ import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.io.Serializable; -import java.util.List; - /** * Tests for {@link ConsumerTrackingPipelineVisitor}. */ diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DirectRegistrarTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DirectRegistrarTest.java index d5cdee657180c..603e43e30f6be 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DirectRegistrarTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DirectRegistrarTest.java @@ -20,20 +20,17 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.fail; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.Lists; +import java.util.ServiceLoader; import org.apache.beam.runners.direct.DirectRegistrar.Options; import org.apache.beam.runners.direct.DirectRegistrar.Runner; import org.apache.beam.sdk.options.PipelineOptionsRegistrar; import org.apache.beam.sdk.runners.PipelineRunnerRegistrar; - -import com.google.common.collect.ImmutableList; -import com.google.common.collect.Lists; - import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.util.ServiceLoader; - /** Tests for {@link DirectRegistrar}. */ @RunWith(JUnit4.class) public class DirectRegistrarTest { 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 9739adb0f3662..c7efac388b682 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 @@ -22,6 +22,13 @@ import static org.junit.Assert.assertThat; import static org.junit.Assert.fail; +import com.fasterxml.jackson.annotation.JsonValue; +import com.google.common.collect.ImmutableMap; +import java.io.Serializable; +import java.util.Arrays; +import java.util.List; +import java.util.Map; +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.coders.CoderException; @@ -47,8 +54,6 @@ import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionList; import org.apache.beam.sdk.values.TypeDescriptor; -import com.google.common.collect.ImmutableMap; -import com.fasterxml.jackson.annotation.JsonValue; import org.hamcrest.Matchers; import org.junit.Rule; import org.junit.Test; @@ -56,11 +61,6 @@ import org.junit.rules.ExpectedException; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.io.Serializable; -import java.util.Arrays; -import java.util.List; -import java.util.Map; -import java.util.concurrent.atomic.AtomicInteger; /** * Tests for basic {@link DirectRunner} functionality. diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DirectTimerInternalsTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DirectTimerInternalsTest.java index c038910a47595..51cfeedbaca0b 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DirectTimerInternalsTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DirectTimerInternalsTest.java @@ -29,7 +29,6 @@ import org.apache.beam.sdk.util.TimeDomain; import org.apache.beam.sdk.util.TimerInternals.TimerData; import org.apache.beam.sdk.util.state.StateNamespaces; - import org.joda.time.Duration; import org.joda.time.Instant; import org.junit.Before; 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 67f4ff47789f2..2e4fee24760e9 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 @@ -25,18 +25,16 @@ import static org.junit.Assert.assertThat; import static org.junit.Assert.fail; +import java.util.ArrayList; +import java.util.List; import org.apache.beam.sdk.transforms.OldDoFn; import org.apache.beam.sdk.util.WindowedValue; - import org.hamcrest.Matchers; import org.junit.Before; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.util.ArrayList; -import java.util.List; - /** * Tests for {@link DoFnLifecycleManagerRemovingTransformEvaluator}. */ diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DoFnLifecycleManagerTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DoFnLifecycleManagerTest.java index 9da46f265b7da..1f0af9913acfc 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DoFnLifecycleManagerTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DoFnLifecycleManagerTest.java @@ -25,13 +25,6 @@ import static org.hamcrest.Matchers.theInstance; import static org.junit.Assert.assertThat; -import org.apache.beam.sdk.transforms.OldDoFn; - -import org.hamcrest.Matchers; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.JUnit4; - import java.util.ArrayList; import java.util.List; import java.util.concurrent.Callable; @@ -40,6 +33,11 @@ import java.util.concurrent.Executors; import java.util.concurrent.Future; import java.util.concurrent.TimeUnit; +import org.apache.beam.sdk.transforms.OldDoFn; +import org.hamcrest.Matchers; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; /** * Tests for {@link DoFnLifecycleManager}. diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DoFnLifecycleManagersTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DoFnLifecycleManagersTest.java index 8be3d52763be2..39a4a9d8f39ce 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DoFnLifecycleManagersTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DoFnLifecycleManagersTest.java @@ -20,10 +20,10 @@ import static org.hamcrest.Matchers.equalTo; -import org.apache.beam.sdk.transforms.OldDoFn; - import com.google.common.collect.ImmutableList; - +import java.util.ArrayList; +import java.util.Collection; +import org.apache.beam.sdk.transforms.OldDoFn; import org.hamcrest.BaseMatcher; import org.hamcrest.Description; import org.hamcrest.Matcher; @@ -34,9 +34,6 @@ import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.util.ArrayList; -import java.util.Collection; - /** * Tests for {@link DoFnLifecycleManagers}. */ diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/EncodabilityEnforcementFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/EncodabilityEnforcementFactoryTest.java index b903ef1eb42f7..e0ccbe5461727 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/EncodabilityEnforcementFactoryTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/EncodabilityEnforcementFactoryTest.java @@ -19,6 +19,10 @@ import static org.hamcrest.Matchers.isA; +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.util.Collections; import org.apache.beam.runners.direct.DirectRunner.CommittedBundle; import org.apache.beam.sdk.coders.AtomicCoder; import org.apache.beam.sdk.coders.Coder; @@ -31,7 +35,6 @@ import org.apache.beam.sdk.util.UserCodeException; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.PCollection; - import org.joda.time.Instant; import org.junit.Rule; import org.junit.Test; @@ -39,11 +42,6 @@ import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.io.IOException; -import java.io.InputStream; -import java.io.OutputStream; -import java.util.Collections; - /** * Tests for {@link EncodabilityEnforcementFactory}. */ 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 7ac0caadc55bc..f59dbbaf8568e 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 @@ -25,6 +25,13 @@ import static org.hamcrest.Matchers.not; import static org.junit.Assert.assertThat; +import com.google.common.collect.ImmutableList; +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; import org.apache.beam.runners.direct.DirectRunner.CommittedBundle; import org.apache.beam.runners.direct.DirectRunner.PCollectionViewWriter; @@ -61,10 +68,6 @@ import org.apache.beam.sdk.values.PCollection.IsBounded; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.PValue; - -import com.google.common.collect.ImmutableList; -import com.google.common.collect.Iterables; - import org.hamcrest.Matchers; import org.joda.time.Instant; import org.junit.Before; @@ -72,12 +75,6 @@ import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.util.Collection; -import java.util.Collections; -import java.util.Map; -import java.util.concurrent.CountDownLatch; -import java.util.concurrent.TimeUnit; - /** * Tests for {@link EvaluationContext}. */ 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 0bc3036f6399d..1c46c2411414e 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 @@ -33,7 +33,6 @@ import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionList; - import org.hamcrest.Matchers; import org.joda.time.Instant; import org.junit.Test; 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 9ea71d70878d0..6abaf921943e9 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 @@ -27,7 +27,6 @@ import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.display.DisplayData; import org.apache.beam.sdk.values.PCollection; - import org.junit.Before; import org.junit.Rule; import org.junit.Test; 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 78736c4f80876..8d1f8bdfaf68b 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 @@ -22,6 +22,9 @@ import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; +import com.google.common.collect.HashMultiset; +import com.google.common.collect.ImmutableSet; +import com.google.common.collect.Multiset; 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,11 +39,6 @@ import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; - -import com.google.common.collect.HashMultiset; -import com.google.common.collect.ImmutableSet; -import com.google.common.collect.Multiset; - import org.hamcrest.BaseMatcher; import org.hamcrest.Description; import org.joda.time.Instant; 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 4afd64b52ad42..9f1e916c28cf3 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 @@ -22,6 +22,9 @@ import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; +import com.google.common.collect.HashMultiset; +import com.google.common.collect.ImmutableSet; +import com.google.common.collect.Multiset; 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,11 +39,6 @@ import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; - -import com.google.common.collect.HashMultiset; -import com.google.common.collect.ImmutableSet; -import com.google.common.collect.Multiset; - import org.hamcrest.BaseMatcher; import org.hamcrest.Description; import org.joda.time.Instant; 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 db934e542f61a..d44151ac6556e 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 @@ -33,7 +33,6 @@ import org.apache.beam.sdk.util.IllegalMutationException; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.PCollection; - import org.joda.time.Instant; import org.junit.Before; import org.junit.Rule; 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 e1be120012251..713ae3510942a 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 @@ -17,6 +17,8 @@ */ package org.apache.beam.runners.direct; +import java.io.Serializable; +import java.util.Collections; import org.apache.beam.runners.direct.DirectRunner.CommittedBundle; import org.apache.beam.sdk.testing.TestPipeline; import org.apache.beam.sdk.transforms.AppliedPTransform; @@ -27,7 +29,6 @@ import org.apache.beam.sdk.util.IllegalMutationException; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.PCollection; - import org.joda.time.Instant; import org.junit.Before; import org.junit.Rule; @@ -36,9 +37,6 @@ import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.io.Serializable; -import java.util.Collections; - /** * Tests for {@link ImmutabilityEnforcementFactory}. */ 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 21e4bcbc0819d..43108f8176610 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 @@ -21,6 +21,10 @@ import static org.hamcrest.Matchers.equalTo; import static org.junit.Assert.assertThat; +import com.google.common.collect.ImmutableList; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; import org.apache.beam.runners.direct.DirectRunner.CommittedBundle; import org.apache.beam.runners.direct.DirectRunner.UncommittedBundle; import org.apache.beam.sdk.coders.ByteArrayCoder; @@ -36,9 +40,6 @@ import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; - -import com.google.common.collect.ImmutableList; - import org.hamcrest.Matcher; import org.hamcrest.Matchers; import org.joda.time.Instant; @@ -49,10 +50,6 @@ import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.util.ArrayList; -import java.util.Collection; -import java.util.Collections; - /** * Tests for {@link ImmutableListBundleFactory}. */ 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 9e273ad84292c..ee6b2b4ed2e85 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,6 +21,9 @@ 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.Pipeline; import org.apache.beam.sdk.coders.IterableCoder; import org.apache.beam.sdk.coders.KvCoder; @@ -35,9 +38,6 @@ import org.apache.beam.sdk.transforms.ParDo; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; - -import com.google.common.collect.ImmutableSet; - import org.junit.Before; import org.junit.Rule; import org.junit.Test; @@ -45,9 +45,6 @@ import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.util.Collections; -import java.util.Set; - /** * Tests for {@link KeyedPValueTrackingVisitor}. */ 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 3208841f61e0e..2a54ecb8aada9 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 @@ -23,6 +23,13 @@ import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Iterables; +import java.util.ArrayList; +import java.util.Collection; +import java.util.List; +import javax.annotation.Nullable; import org.apache.beam.runners.direct.DirectExecutionContext.DirectStepContext; import org.apache.beam.runners.direct.DirectRunner.CommittedBundle; import org.apache.beam.runners.direct.DirectRunner.UncommittedBundle; @@ -45,9 +52,6 @@ import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TupleTag; import org.apache.beam.sdk.values.TupleTagList; -import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableMap; -import com.google.common.collect.Iterables; import org.hamcrest.Matchers; import org.joda.time.Instant; import org.junit.Before; @@ -57,10 +61,6 @@ import org.mockito.Mock; import org.mockito.Mockito; import org.mockito.MockitoAnnotations; -import java.util.ArrayList; -import java.util.Collection; -import java.util.List; -import javax.annotation.Nullable; /** * Tests for {@link ParDoEvaluator}. diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ParDoMultiEvaluatorFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ParDoMultiEvaluatorFactoryTest.java index 19094cbfe2843..555219626d72f 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ParDoMultiEvaluatorFactoryTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ParDoMultiEvaluatorFactoryTest.java @@ -25,6 +25,7 @@ import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; +import java.io.Serializable; import org.apache.beam.runners.direct.DirectRunner.CommittedBundle; import org.apache.beam.runners.direct.DirectRunner.UncommittedBundle; import org.apache.beam.runners.direct.WatermarkManager.TimerUpdate; @@ -59,7 +60,6 @@ import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.io.Serializable; /** * Tests for {@link ParDoMultiEvaluatorFactory}. diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ParDoSingleEvaluatorFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ParDoSingleEvaluatorFactoryTest.java index a4fd570d86249..60b6dd990b98d 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ParDoSingleEvaluatorFactoryTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ParDoSingleEvaluatorFactoryTest.java @@ -25,6 +25,7 @@ import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; +import java.io.Serializable; import org.apache.beam.runners.direct.DirectRunner.CommittedBundle; import org.apache.beam.runners.direct.DirectRunner.UncommittedBundle; import org.apache.beam.runners.direct.WatermarkManager.TimerUpdate; @@ -56,7 +57,6 @@ import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.io.Serializable; /** * Tests for {@link ParDoSingleEvaluatorFactory}. 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 ec589da6eadb1..cc7d88a39e7e6 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 @@ -25,6 +25,13 @@ import static org.junit.Assert.fail; import static org.mockito.Mockito.doAnswer; +import com.google.common.collect.ImmutableList; +import java.util.Map; +import java.util.concurrent.Callable; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.Executors; +import java.util.concurrent.Future; +import java.util.concurrent.TimeUnit; import org.apache.beam.sdk.coders.KvCoder; import org.apache.beam.sdk.coders.StringUtf8Coder; import org.apache.beam.sdk.testing.TestPipeline; @@ -44,9 +51,6 @@ import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionView; - -import com.google.common.collect.ImmutableList; - import org.joda.time.Instant; import org.junit.Before; import org.junit.Rule; @@ -60,13 +64,6 @@ import org.mockito.invocation.InvocationOnMock; import org.mockito.stubbing.Answer; -import java.util.Map; -import java.util.concurrent.Callable; -import java.util.concurrent.CountDownLatch; -import java.util.concurrent.Executors; -import java.util.concurrent.Future; -import java.util.concurrent.TimeUnit; - /** * Tests for {@link SideInputContainer}. */ 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 cfc69bc61d0f9..c06eff9606468 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 @@ -29,7 +29,6 @@ import org.apache.beam.sdk.transforms.AppliedPTransform; import org.apache.beam.sdk.transforms.Create; import org.apache.beam.sdk.values.PCollection; - import org.hamcrest.Matchers; import org.junit.Before; import org.junit.Test; diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/StructuralKeyTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/StructuralKeyTest.java index 26514f004ec5d..18aeac688d8a8 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/StructuralKeyTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/StructuralKeyTest.java @@ -27,7 +27,6 @@ import org.apache.beam.sdk.coders.ByteArrayCoder; import org.apache.beam.sdk.coders.StringUtf8Coder; import org.apache.beam.sdk.coders.VarIntCoder; - import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/TransformExecutorServicesTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/TransformExecutorServicesTest.java index 04aa96ffdfd11..b085723a3ccb1 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/TransformExecutorServicesTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/TransformExecutorServicesTest.java @@ -22,7 +22,7 @@ import static org.mockito.Mockito.verify; import com.google.common.util.concurrent.MoreExecutors; - +import java.util.concurrent.ExecutorService; import org.junit.Before; import org.junit.Rule; import org.junit.Test; @@ -30,8 +30,6 @@ import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.util.concurrent.ExecutorService; - /** * Tests for {@link TransformExecutorServices}. */ 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 f6cb8d133d8e3..5af568f9f07ff 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 @@ -27,6 +27,16 @@ import static org.junit.Assert.assertThat; import static org.mockito.Mockito.when; +import com.google.common.util.concurrent.MoreExecutors; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.EnumSet; +import java.util.List; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.Executors; +import java.util.concurrent.Future; +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; @@ -39,9 +49,6 @@ import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; - -import com.google.common.util.concurrent.MoreExecutors; - import org.hamcrest.Matchers; import org.joda.time.Instant; import org.junit.Before; @@ -53,16 +60,6 @@ import org.mockito.Mock; import org.mockito.MockitoAnnotations; -import java.util.ArrayList; -import java.util.Collection; -import java.util.Collections; -import java.util.EnumSet; -import java.util.List; -import java.util.concurrent.CountDownLatch; -import java.util.concurrent.Executors; -import java.util.concurrent.Future; -import java.util.concurrent.atomic.AtomicBoolean; - /** * Tests for {@link TransformExecutor}. diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/UnboundedReadDeduplicatorTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/UnboundedReadDeduplicatorTest.java index b3c9012b60c25..7d2a95cd167c7 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/UnboundedReadDeduplicatorTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/UnboundedReadDeduplicatorTest.java @@ -21,18 +21,16 @@ import static org.hamcrest.Matchers.is; import static org.junit.Assert.assertThat; -import org.apache.beam.runners.direct.UnboundedReadDeduplicator.CachedIdDeduplicator; -import org.apache.beam.runners.direct.UnboundedReadDeduplicator.NeverDeduplicator; - -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.JUnit4; - import java.util.concurrent.CountDownLatch; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; +import org.apache.beam.runners.direct.UnboundedReadDeduplicator.CachedIdDeduplicator; +import org.apache.beam.runners.direct.UnboundedReadDeduplicator.NeverDeduplicator; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; /** * Tests for {@link UnboundedReadDeduplicator}. 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 839badfd6f9e7..3a6add6aac97d 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 @@ -25,6 +25,17 @@ import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; +import com.google.common.collect.ContiguousSet; +import com.google.common.collect.DiscreteDomain; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.Range; +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.util.Arrays; +import java.util.List; +import java.util.NoSuchElementException; +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.coders.AtomicCoder; @@ -43,12 +54,6 @@ import org.apache.beam.sdk.util.CoderUtils; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.PCollection; - -import com.google.common.collect.ContiguousSet; -import com.google.common.collect.DiscreteDomain; -import com.google.common.collect.ImmutableList; -import com.google.common.collect.Range; - import org.hamcrest.Matchers; import org.joda.time.DateTime; import org.joda.time.Instant; @@ -57,15 +62,6 @@ import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; - -import java.io.IOException; -import java.io.InputStream; -import java.io.OutputStream; -import java.util.Arrays; -import java.util.List; -import java.util.NoSuchElementException; - -import javax.annotation.Nullable; /** * Tests for {@link UnboundedReadEvaluatorFactory}. */ 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 68207922855c6..d3ab81d8e7d6a 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 @@ -23,6 +23,7 @@ import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; +import com.google.common.collect.ImmutableList; import org.apache.beam.runners.direct.DirectRunner.CommittedBundle; import org.apache.beam.runners.direct.DirectRunner.PCollectionViewWriter; import org.apache.beam.sdk.coders.KvCoder; @@ -38,9 +39,6 @@ import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionView; - -import com.google.common.collect.ImmutableList; - import org.joda.time.Instant; import org.junit.Test; import org.junit.runner.RunWith; 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 b6b2bf58af9a4..1be9a9884a8ca 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 @@ -20,6 +20,9 @@ import static org.hamcrest.Matchers.equalTo; import static org.junit.Assert.assertThat; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.Executors; +import java.util.concurrent.TimeUnit; import org.apache.beam.sdk.testing.TestPipeline; import org.apache.beam.sdk.transforms.AppliedPTransform; import org.apache.beam.sdk.transforms.Create; @@ -31,7 +34,6 @@ import org.apache.beam.sdk.transforms.windowing.WindowFn; import org.apache.beam.sdk.util.WindowingStrategy; import org.apache.beam.sdk.values.PCollection; - import org.joda.time.Duration; import org.joda.time.Instant; import org.junit.Before; @@ -39,10 +41,6 @@ import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.util.concurrent.CountDownLatch; -import java.util.concurrent.Executors; -import java.util.concurrent.TimeUnit; - /** * Tests for {@link WatermarkCallbackExecutor}. */ 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 22f148a4a3f71..d9dc404cdf07a 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 @@ -25,6 +25,15 @@ import static org.hamcrest.Matchers.nullValue; import static org.junit.Assert.assertThat; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.Iterables; +import java.io.Serializable; +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; import org.apache.beam.runners.direct.DirectRunner.CommittedBundle; import org.apache.beam.runners.direct.DirectRunner.UncommittedBundle; @@ -56,10 +65,6 @@ import org.apache.beam.sdk.values.PCollectionList; import org.apache.beam.sdk.values.PValue; import org.apache.beam.sdk.values.TimestampedValue; - -import com.google.common.collect.ImmutableList; -import com.google.common.collect.Iterables; - import org.hamcrest.BaseMatcher; import org.hamcrest.Description; import org.hamcrest.Matcher; @@ -70,14 +75,6 @@ import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.io.Serializable; -import java.util.Collection; -import java.util.Collections; -import java.util.EnumSet; -import java.util.HashMap; -import java.util.Map; -import javax.annotation.Nullable; - /** * Tests for {@link WatermarkManager}. */ 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 65dcfebe39c4c..63800cfb907bd 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 @@ -19,11 +19,15 @@ import static org.apache.beam.sdk.WindowMatchers.isSingleWindowedValue; import static org.apache.beam.sdk.WindowMatchers.isWindowedValue; - import static org.hamcrest.Matchers.containsInAnyOrder; import static org.junit.Assert.assertThat; import static org.mockito.Mockito.when; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableSet; +import com.google.common.collect.Iterables; +import java.util.Collection; +import java.util.Collections; import org.apache.beam.runners.direct.DirectRunner.CommittedBundle; import org.apache.beam.runners.direct.DirectRunner.UncommittedBundle; import org.apache.beam.sdk.coders.Coder; @@ -44,11 +48,6 @@ import org.apache.beam.sdk.transforms.windowing.WindowFn; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.PCollection; - -import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableSet; -import com.google.common.collect.Iterables; - import org.hamcrest.Matchers; import org.joda.time.Duration; import org.joda.time.Instant; @@ -59,9 +58,6 @@ import org.mockito.Mock; import org.mockito.MockitoAnnotations; -import java.util.Collection; -import java.util.Collections; - /** * Tests for {@link WindowEvaluatorFactory}. */ 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 20b2776890e91..2dd477dd30e4f 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 @@ -28,6 +28,18 @@ import static org.hamcrest.Matchers.not; import static org.junit.Assert.assertThat; +import com.google.common.base.Function; +import com.google.common.collect.Iterables; +import java.io.File; +import java.io.FileReader; +import java.io.Reader; +import java.nio.CharBuffer; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.List; +import java.util.UUID; +import java.util.concurrent.ThreadLocalRandom; import org.apache.beam.runners.direct.WriteWithShardingFactory.KeyBasedOnCountFn; import org.apache.beam.sdk.coders.VarLongCoder; import org.apache.beam.sdk.io.Sink; @@ -46,27 +58,12 @@ import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.PDone; - -import com.google.common.base.Function; -import com.google.common.collect.Iterables; - import org.junit.Rule; import org.junit.Test; import org.junit.rules.TemporaryFolder; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.io.File; -import java.io.FileReader; -import java.io.Reader; -import java.nio.CharBuffer; -import java.util.ArrayList; -import java.util.Collection; -import java.util.Collections; -import java.util.List; -import java.util.UUID; -import java.util.concurrent.ThreadLocalRandom; - /** * Tests for {@link WriteWithShardingFactory}. */ 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 4deca12d14fa1..0ca94a13463bd 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 @@ -17,6 +17,12 @@ */ package org.apache.beam.runners.flink.examples; +import java.io.File; +import java.io.IOException; +import java.net.URI; +import java.net.URISyntaxException; +import java.util.HashSet; +import java.util.Set; import org.apache.beam.runners.flink.FlinkPipelineOptions; import org.apache.beam.runners.flink.FlinkRunner; import org.apache.beam.sdk.Pipeline; @@ -53,17 +59,9 @@ import org.apache.beam.sdk.values.PDone; import org.apache.beam.sdk.values.PInput; import org.apache.beam.sdk.values.TupleTag; - import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.io.File; -import java.io.IOException; -import java.net.URI; -import java.net.URISyntaxException; -import java.util.HashSet; -import java.util.Set; - /** * An example that computes a basic TF-IDF search table for a directory or GCS prefix. * 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 fdffd391ba0d3..ab9297f72d8ef 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 @@ -28,7 +28,6 @@ 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.OldDoFn; import org.apache.beam.sdk.transforms.MapElements; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.ParDo; 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 aff1a357c8ec0..9b5e31d08bbe0 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 @@ -17,6 +17,8 @@ */ package org.apache.beam.runners.flink.examples.streaming; +import java.io.IOException; +import java.util.List; import org.apache.beam.runners.flink.FlinkRunner; import org.apache.beam.runners.flink.translation.wrappers.streaming.io.UnboundedSocketSource; import org.apache.beam.sdk.Pipeline; @@ -47,12 +49,8 @@ import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionList; - import org.joda.time.Duration; -import java.io.IOException; -import java.util.List; - /** * To run the example, first open a socket on a terminal by executing the command: *

  • diff --git a/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/streaming/JoinExamples.java b/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/streaming/JoinExamples.java index 458a263331835..bf5dfc453e19a 100644 --- a/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/streaming/JoinExamples.java +++ b/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/streaming/JoinExamples.java @@ -35,7 +35,6 @@ import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.TupleTag; - import org.joda.time.Duration; /** diff --git a/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/streaming/KafkaIOExamples.java b/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/streaming/KafkaIOExamples.java index 68a9edcc5a8b3..27faefe6d7457 100644 --- a/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/streaming/KafkaIOExamples.java +++ b/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/streaming/KafkaIOExamples.java @@ -17,6 +17,11 @@ */ package org.apache.beam.runners.flink.examples.streaming; +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.io.Serializable; +import java.util.Properties; import org.apache.beam.runners.flink.FlinkPipelineOptions; import org.apache.beam.runners.flink.FlinkRunner; import org.apache.beam.runners.flink.translation.wrappers.streaming.io.UnboundedFlinkSink; @@ -33,7 +38,6 @@ import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.ParDo; import org.apache.beam.sdk.values.PCollection; - import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.typeutils.TypeExtractor; import org.apache.flink.streaming.connectors.kafka.FlinkKafkaConsumer08; @@ -42,12 +46,6 @@ import org.apache.flink.streaming.util.serialization.SerializationSchema; import org.apache.flink.streaming.util.serialization.SimpleStringSchema; -import java.io.ByteArrayInputStream; -import java.io.ByteArrayOutputStream; -import java.io.IOException; -import java.io.Serializable; -import java.util.Properties; - /** * Recipes/Examples that demonstrate how to read/write data from/to Kafka. */ diff --git a/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/streaming/KafkaWindowedWordCountExample.java b/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/streaming/KafkaWindowedWordCountExample.java index 39ce22542c0b4..365fb7b486726 100644 --- a/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/streaming/KafkaWindowedWordCountExample.java +++ b/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/streaming/KafkaWindowedWordCountExample.java @@ -17,6 +17,7 @@ */ package org.apache.beam.runners.flink.examples.streaming; +import java.util.Properties; import org.apache.beam.runners.flink.FlinkRunner; import org.apache.beam.runners.flink.translation.wrappers.streaming.io.UnboundedFlinkSource; import org.apache.beam.sdk.Pipeline; @@ -35,13 +36,10 @@ 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.flink.streaming.connectors.kafka.FlinkKafkaConsumer08; import org.apache.flink.streaming.util.serialization.SimpleStringSchema; import org.joda.time.Duration; -import java.util.Properties; - public class KafkaWindowedWordCountExample { static final String KAFKA_TOPIC = "test"; // Default kafka topic to read from diff --git a/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/streaming/WindowedWordCount.java b/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/streaming/WindowedWordCount.java index fe8e627d1335a..f3361c5716973 100644 --- a/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/streaming/WindowedWordCount.java +++ b/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/streaming/WindowedWordCount.java @@ -17,6 +17,7 @@ */ package org.apache.beam.runners.flink.examples.streaming; +import java.io.IOException; import org.apache.beam.runners.flink.FlinkRunner; import org.apache.beam.runners.flink.translation.wrappers.streaming.io.UnboundedSocketSource; import org.apache.beam.sdk.Pipeline; @@ -35,13 +36,10 @@ 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.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.io.IOException; - /** * To run the example, first open a socket on a terminal by executing the command: *
  • diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/DefaultParallelismFactory.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/DefaultParallelismFactory.java index e512db0e3eaa8..2fe4569e3e406 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/DefaultParallelismFactory.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/DefaultParallelismFactory.java @@ -19,7 +19,6 @@ import org.apache.beam.sdk.options.DefaultValueFactory; import org.apache.beam.sdk.options.PipelineOptions; - import org.apache.flink.configuration.ConfigConstants; import org.apache.flink.configuration.GlobalConfiguration; 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 f4d4ea6d546d7..d1977a445adfa 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 @@ -19,13 +19,13 @@ import static com.google.common.base.Preconditions.checkNotNull; +import java.util.List; import org.apache.beam.runners.flink.translation.FlinkBatchPipelineTranslator; import org.apache.beam.runners.flink.translation.FlinkPipelineTranslator; import org.apache.beam.runners.flink.translation.FlinkStreamingPipelineTranslator; import org.apache.beam.runners.flink.translation.PipelineTranslationOptimizer; import org.apache.beam.runners.flink.translation.TranslationMode; import org.apache.beam.sdk.Pipeline; - import org.apache.flink.api.common.JobExecutionResult; import org.apache.flink.api.java.CollectionEnvironment; import org.apache.flink.api.java.ExecutionEnvironment; @@ -34,8 +34,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.util.List; - /** * The class that instantiates and manages the execution of a given job. * Depending on if the job is a Streaming or Batch processing one, it creates 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 c40473e108530..6d1a8d040f1a5 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 @@ -18,21 +18,19 @@ package org.apache.beam.runners.flink; +import com.fasterxml.jackson.annotation.JsonIgnore; +import java.util.List; import org.apache.beam.sdk.options.ApplicationNameOptions; 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.StreamingOptions; - -import com.fasterxml.jackson.annotation.JsonIgnore; import org.joda.time.DateTimeUtils; import org.joda.time.DateTimeZone; import org.joda.time.format.DateTimeFormat; import org.joda.time.format.DateTimeFormatter; -import java.util.List; - /** * Options which can be used to configure a Flink PipelineRunner. */ 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 b0e88b7db7e2b..8b1f42e5447cd 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 @@ -17,6 +17,16 @@ */ package org.apache.beam.runners.flink; +import com.google.common.base.Joiner; +import com.google.common.collect.ImmutableMap; +import java.io.File; +import java.net.URISyntaxException; +import java.net.URL; +import java.net.URLClassLoader; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.Map; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.CoderRegistry; @@ -37,23 +47,10 @@ import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.PInput; import org.apache.beam.sdk.values.POutput; - -import com.google.common.base.Joiner; -import com.google.common.collect.ImmutableMap; - import org.apache.flink.api.common.JobExecutionResult; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.io.File; -import java.net.URISyntaxException; -import java.net.URL; -import java.net.URLClassLoader; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.List; -import java.util.Map; - /** * A {@link PipelineRunner} that executes the operations in the * pipeline by first translating them to a Flink Plan and then executing them either locally diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkRunnerRegistrar.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkRunnerRegistrar.java index 52b2a8ddb5d08..f328279bfba02 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkRunnerRegistrar.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkRunnerRegistrar.java @@ -18,14 +18,13 @@ package org.apache.beam.runners.flink; +import com.google.auto.service.AutoService; +import com.google.common.collect.ImmutableList; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.options.PipelineOptionsRegistrar; import org.apache.beam.sdk.runners.PipelineRunner; import org.apache.beam.sdk.runners.PipelineRunnerRegistrar; -import com.google.auto.service.AutoService; -import com.google.common.collect.ImmutableList; - /** * AuteService registrar - will register FlinkRunner and FlinkOptions diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkRunnerResult.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkRunnerResult.java index 923d54c566313..dd0733a671c98 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkRunnerResult.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkRunnerResult.java @@ -17,17 +17,15 @@ */ package org.apache.beam.runners.flink; -import org.apache.beam.sdk.PipelineResult; +import java.io.IOException; +import java.util.Collections; +import java.util.Map; import org.apache.beam.sdk.AggregatorRetrievalException; import org.apache.beam.sdk.AggregatorValues; +import org.apache.beam.sdk.PipelineResult; import org.apache.beam.sdk.transforms.Aggregator; - import org.joda.time.Duration; -import java.io.IOException; -import java.util.Collections; -import java.util.Map; - /** * Result of executing a {@link org.apache.beam.sdk.Pipeline} with Flink. This * has methods to query to job runtime and the final values of diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/TestFlinkRunner.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/TestFlinkRunner.java index 6a4f990cda332..dd231d6e6bdd4 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/TestFlinkRunner.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/TestFlinkRunner.java @@ -26,8 +26,6 @@ import org.apache.beam.sdk.values.PInput; import org.apache.beam.sdk.values.POutput; -import org.apache.flink.runtime.client.JobExecutionException; - public class TestFlinkRunner extends PipelineRunner { private FlinkRunner delegate; 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 8f9a37a6c2238..66c48b07e23b1 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 @@ -22,7 +22,6 @@ import org.apache.beam.sdk.runners.TransformTreeNode; 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; import org.apache.flink.api.java.io.DiscardingOutputFormat; 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 01a3ab25b3122..935a9ac439b94 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkBatchTransformTranslators.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkBatchTransformTranslators.java @@ -17,6 +17,13 @@ */ package org.apache.beam.runners.flink.translation; +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; 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; @@ -39,9 +46,9 @@ 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.OldDoFn; 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; @@ -60,10 +67,6 @@ import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.PValue; import org.apache.beam.sdk.values.TupleTag; - -import com.google.common.collect.Lists; -import com.google.common.collect.Maps; - import org.apache.flink.api.common.functions.FilterFunction; import org.apache.flink.api.common.functions.FlatMapFunction; import org.apache.flink.api.common.typeinfo.TypeInformation; @@ -77,12 +80,6 @@ import org.apache.flink.api.java.operators.SingleInputUdfOperator; import org.apache.flink.util.Collector; -import java.util.ArrayList; -import java.util.Collections; -import java.util.HashMap; -import java.util.List; -import java.util.Map; - /** * Translators for transforming {@link PTransform PTransforms} to * Flink {@link DataSet DataSets}. diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkBatchTranslationContext.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkBatchTranslationContext.java index a73bf135a2b1d..835648e43ea8f 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkBatchTranslationContext.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkBatchTranslationContext.java @@ -17,6 +17,8 @@ */ package org.apache.beam.runners.flink.translation; +import java.util.HashMap; +import java.util.Map; import org.apache.beam.runners.flink.translation.types.CoderTypeInformation; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.options.PipelineOptions; @@ -29,14 +31,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.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.DataSet; import org.apache.flink.api.java.ExecutionEnvironment; -import java.util.HashMap; -import java.util.Map; - /** * Helper for {@link FlinkBatchPipelineTranslator} and translators in * {@link FlinkBatchTransformTranslators}. 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 3bb8c5916089e..b12745512c93e 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 @@ -22,7 +22,6 @@ 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; import org.slf4j.Logger; import org.slf4j.LoggerFactory; 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 6c2c703aace93..07e2191d4e0c2 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,6 +18,17 @@ package org.apache.beam.runners.flink.translation; +import com.google.api.client.util.Maps; +import com.google.common.collect.Iterables; +import com.google.common.collect.Lists; +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Set; import org.apache.beam.runners.flink.FlinkRunner; import org.apache.beam.runners.flink.translation.types.CoderTypeInformation; import org.apache.beam.runners.flink.translation.types.FlinkCoder; @@ -60,11 +71,6 @@ import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.PValue; import org.apache.beam.sdk.values.TupleTag; - -import com.google.api.client.util.Maps; -import com.google.common.collect.Iterables; -import com.google.common.collect.Lists; - import org.apache.flink.api.common.functions.FlatMapFunction; import org.apache.flink.api.common.functions.MapFunction; import org.apache.flink.api.common.functions.RichFlatMapFunction; @@ -85,15 +91,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.nio.ByteBuffer; -import java.util.ArrayList; -import java.util.Collection; -import java.util.Collections; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.Set; - /** * This class contains all the mappings between Beam and Flink * streaming transformations. The {@link FlinkStreamingPipelineTranslator} diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkStreamingTranslationContext.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkStreamingTranslationContext.java index a75ef034c850c..bc80d42834ff9 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkStreamingTranslationContext.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkStreamingTranslationContext.java @@ -19,6 +19,8 @@ import static com.google.common.base.Preconditions.checkNotNull; +import java.util.HashMap; +import java.util.Map; import org.apache.beam.runners.flink.translation.types.CoderTypeInformation; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.options.PipelineOptions; @@ -29,14 +31,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.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import java.util.HashMap; -import java.util.Map; - /** * Helper for keeping track of which {@link DataStream DataStreams} map * to which {@link PTransform PTransforms}. diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkAssignContext.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkAssignContext.java index 6abb8ffcbb86a..447b1e507e1a5 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkAssignContext.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkAssignContext.java @@ -19,16 +19,12 @@ import static com.google.common.base.Preconditions.checkArgument; +import com.google.common.collect.Iterables; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.WindowFn; import org.apache.beam.sdk.util.WindowedValue; - -import com.google.common.collect.Iterables; - import org.joda.time.Instant; -import java.util.Collection; - /** * {@link org.apache.beam.sdk.transforms.windowing.WindowFn.AssignContext} for * Flink functions. diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkAssignWindows.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkAssignWindows.java index e07e49a2f0609..f241ad0dc5c0f 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkAssignWindows.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkAssignWindows.java @@ -17,15 +17,13 @@ */ package org.apache.beam.runners.flink.translation.functions; +import java.util.Collection; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.WindowFn; import org.apache.beam.sdk.util.WindowedValue; - import org.apache.flink.api.common.functions.FlatMapFunction; import org.apache.flink.util.Collector; -import java.util.Collection; - /** * Flink {@link FlatMapFunction} for implementing * {@link org.apache.beam.sdk.transforms.windowing.Window.Bound}. 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 733d3d46d9c0c..ac5b345db6783 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 @@ -17,19 +17,17 @@ */ package org.apache.beam.runners.flink.translation.functions; +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.OldDoFn; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.WindowingStrategy; import org.apache.beam.sdk.values.PCollectionView; - import org.apache.flink.api.common.functions.RichMapPartitionFunction; import org.apache.flink.configuration.Configuration; import org.apache.flink.util.Collector; -import java.util.Map; - /** * Encapsulates a {@link OldDoFn} * inside a Flink {@link org.apache.flink.api.common.functions.RichMapPartitionFunction}. 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 2d360436e52f2..dbaab1706ec5f 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 @@ -17,6 +17,13 @@ */ package org.apache.beam.runners.flink.translation.functions; +import com.google.common.collect.Iterables; +import com.google.common.collect.Lists; +import java.util.Collections; +import java.util.Comparator; +import java.util.Iterator; +import java.util.List; +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.CombineFnBase; @@ -31,20 +38,10 @@ import org.apache.beam.sdk.util.WindowingStrategy; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollectionView; - -import com.google.common.collect.Iterables; -import com.google.common.collect.Lists; - import org.apache.flink.api.common.functions.RichGroupReduceFunction; import org.apache.flink.util.Collector; import org.joda.time.Instant; -import java.util.Collections; -import java.util.Comparator; -import java.util.Iterator; -import java.util.List; -import java.util.Map; - /** * Special version of {@link FlinkReduceFunction} that supports merging windows. This * assumes that the windows are {@link IntervalWindow IntervalWindows} and exhibits the 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 c12e4204a3f0a..bc09bdf05c2fd 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 @@ -17,6 +17,13 @@ */ package org.apache.beam.runners.flink.translation.functions; +import com.google.common.collect.Iterables; +import com.google.common.collect.Lists; +import java.util.Collections; +import java.util.Comparator; +import java.util.Iterator; +import java.util.List; +import java.util.Map; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.CombineFnBase; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; @@ -29,19 +36,9 @@ import org.apache.beam.sdk.util.WindowingStrategy; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollectionView; - -import com.google.common.collect.Iterables; -import com.google.common.collect.Lists; - import org.apache.flink.util.Collector; import org.joda.time.Instant; -import java.util.Collections; -import java.util.Comparator; -import java.util.Iterator; -import java.util.List; -import java.util.Map; - /** * Special version of {@link FlinkPartialReduceFunction} that supports merging windows. This * assumes that the windows are {@link IntervalWindow IntervalWindows} and exhibits the 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 07d1c97415336..4050f47a2ec48 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 @@ -17,6 +17,15 @@ */ package org.apache.beam.runners.flink.translation.functions; +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.Collections; +import java.util.Comparator; +import java.util.Iterator; +import java.util.List; +import java.util.Map; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.CombineFnBase; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; @@ -29,21 +38,9 @@ import org.apache.beam.sdk.util.WindowingStrategy; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollectionView; - -import com.google.common.collect.ImmutableList; -import com.google.common.collect.Iterables; -import com.google.common.collect.Lists; - import org.apache.flink.util.Collector; import org.joda.time.Instant; -import java.util.ArrayList; -import java.util.Collections; -import java.util.Comparator; -import java.util.Iterator; -import java.util.List; -import java.util.Map; - /** * Special version of {@link FlinkReduceFunction} that supports merging windows. This * assumes that the windows are {@link IntervalWindow IntervalWindows} and exhibits the 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 ef7587876aa81..9cc84ca659252 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 @@ -17,6 +17,7 @@ */ package org.apache.beam.runners.flink.translation.functions; +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.OldDoFn; @@ -25,13 +26,10 @@ import org.apache.beam.sdk.util.WindowingStrategy; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TupleTag; - import org.apache.flink.api.common.functions.RichMapPartitionFunction; import org.apache.flink.configuration.Configuration; import org.apache.flink.util.Collector; -import java.util.Map; - /** * Encapsulates a {@link OldDoFn} that can emit to multiple * outputs inside a Flink {@link org.apache.flink.api.common.functions.RichMapPartitionFunction}. diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkMultiOutputProcessContext.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkMultiOutputProcessContext.java index fab3c856bfea2..153a2d7bdd7f6 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkMultiOutputProcessContext.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkMultiOutputProcessContext.java @@ -17,6 +17,8 @@ */ package org.apache.beam.runners.flink.translation.functions; +import java.util.Collection; +import java.util.Map; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.OldDoFn; import org.apache.beam.sdk.transforms.join.RawUnionValue; @@ -26,14 +28,10 @@ import org.apache.beam.sdk.util.WindowingStrategy; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TupleTag; - import org.apache.flink.api.common.functions.RuntimeContext; import org.apache.flink.util.Collector; import org.joda.time.Instant; -import java.util.Collection; -import java.util.Map; - /** * {@link OldDoFn.ProcessContext} for {@link FlinkMultiOutputDoFnFunction} that supports * side outputs. diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkMultiOutputPruningFunction.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkMultiOutputPruningFunction.java index 9205a5520f827..b72750a366d7e 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkMultiOutputPruningFunction.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkMultiOutputPruningFunction.java @@ -19,7 +19,6 @@ import org.apache.beam.sdk.transforms.join.RawUnionValue; import org.apache.beam.sdk.util.WindowedValue; - import org.apache.flink.api.common.functions.FlatMapFunction; import org.apache.flink.util.Collector; diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkNoElementAssignContext.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkNoElementAssignContext.java index 98446f9c7ba4a..c89027262cd9c 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkNoElementAssignContext.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkNoElementAssignContext.java @@ -20,7 +20,6 @@ import org.apache.beam.sdk.transforms.OldDoFn; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.WindowFn; - import org.joda.time.Instant; /** 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 2db4b7b53b36f..fa2ce4d46a053 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkPartialReduceFunction.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkPartialReduceFunction.java @@ -17,6 +17,13 @@ */ package org.apache.beam.runners.flink.translation.functions; +import com.google.common.collect.Iterables; +import com.google.common.collect.Lists; +import java.util.ArrayList; +import java.util.Collections; +import java.util.Comparator; +import java.util.Iterator; +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.CombineFnBase; @@ -30,20 +37,10 @@ import org.apache.beam.sdk.util.WindowingStrategy; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollectionView; - -import com.google.common.collect.Iterables; -import com.google.common.collect.Lists; - import org.apache.flink.api.common.functions.RichGroupCombineFunction; import org.apache.flink.util.Collector; import org.joda.time.Instant; -import java.util.ArrayList; -import java.util.Collections; -import java.util.Comparator; -import java.util.Iterator; -import java.util.Map; - /** * This is is the first step for executing a {@link org.apache.beam.sdk.transforms.Combine.PerKey} * on Flink. The second part is {@link FlinkReduceFunction}. This function performs a local diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkProcessContext.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkProcessContext.java index 64b93c89513ef..fa5eb1ab10eec 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkProcessContext.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkProcessContext.java @@ -19,6 +19,12 @@ import static com.google.common.base.Preconditions.checkNotNull; +import com.google.common.collect.Iterables; +import java.io.IOException; +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; @@ -34,19 +40,10 @@ import org.apache.beam.sdk.util.state.StateInternals; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TupleTag; - -import com.google.common.collect.Iterables; - import org.apache.flink.api.common.functions.RuntimeContext; import org.apache.flink.util.Collector; import org.joda.time.Instant; -import java.io.IOException; -import java.util.Collection; -import java.util.Collections; -import java.util.Iterator; -import java.util.Map; - /** * {@link OldDoFn.ProcessContext} for our Flink Wrappers. */ 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 b1729a42a40f5..c9b24b456c030 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkReduceFunction.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkReduceFunction.java @@ -17,6 +17,15 @@ */ package org.apache.beam.runners.flink.translation.functions; +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.Collections; +import java.util.Comparator; +import java.util.Iterator; +import java.util.List; +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.CombineFnBase; @@ -30,22 +39,10 @@ import org.apache.beam.sdk.util.WindowingStrategy; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollectionView; - -import com.google.common.collect.ImmutableList; -import com.google.common.collect.Iterables; -import com.google.common.collect.Lists; - import org.apache.flink.api.common.functions.RichGroupReduceFunction; import org.apache.flink.util.Collector; import org.joda.time.Instant; -import java.util.ArrayList; -import java.util.Collections; -import java.util.Comparator; -import java.util.Iterator; -import java.util.List; -import java.util.Map; - /** * This is the second part for executing a {@link org.apache.beam.sdk.transforms.Combine.PerKey} * on Flink, the second part is {@link FlinkReduceFunction}. This function performs the final diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/SideInputInitializer.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/SideInputInitializer.java index a577b684b849f..12222b499d746 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/SideInputInitializer.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/SideInputInitializer.java @@ -17,16 +17,14 @@ */ package org.apache.beam.runners.flink.translation.functions; -import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.WindowedValue; -import org.apache.beam.sdk.values.PCollectionView; - -import org.apache.flink.api.common.functions.BroadcastVariableInitializer; - import java.util.ArrayList; import java.util.HashMap; import java.util.List; import java.util.Map; +import org.apache.beam.sdk.transforms.windowing.BoundedWindow; +import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.PCollectionView; +import org.apache.flink.api.common.functions.BroadcastVariableInitializer; /** * {@link BroadcastVariableInitializer} that initializes the broadcast input as a {@code Map} diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/CoderTypeInformation.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/CoderTypeInformation.java index 61abf9af5ea2f..9b449aabc8b59 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/CoderTypeInformation.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/CoderTypeInformation.java @@ -20,7 +20,6 @@ import static com.google.common.base.Preconditions.checkNotNull; import org.apache.beam.sdk.coders.Coder; - import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.typeinfo.AtomicType; import org.apache.flink.api.common.typeinfo.TypeInformation; diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/CoderTypeSerializer.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/CoderTypeSerializer.java index 46219511213bb..4eda357f64eb7 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/CoderTypeSerializer.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/CoderTypeSerializer.java @@ -17,21 +17,17 @@ */ package org.apache.beam.runners.flink.translation.types; +import java.io.EOFException; +import java.io.IOException; import org.apache.beam.runners.flink.translation.wrappers.DataInputViewWrapper; import org.apache.beam.runners.flink.translation.wrappers.DataOutputViewWrapper; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.CoderException; import org.apache.beam.sdk.util.CoderUtils; - import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.core.memory.DataInputView; import org.apache.flink.core.memory.DataOutputView; -import java.io.ByteArrayInputStream; -import java.io.EOFException; -import java.io.IOException; -import java.io.ObjectInputStream; - /** * Flink {@link org.apache.flink.api.common.typeutils.TypeSerializer} for * Dataflow {@link org.apache.beam.sdk.coders.Coder Coders}. diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/EncodedValueComparator.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/EncodedValueComparator.java index 69bcb415c3179..667ef4591a3f0 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/EncodedValueComparator.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/EncodedValueComparator.java @@ -17,16 +17,14 @@ */ package org.apache.beam.runners.flink.translation.types; +import java.io.IOException; +import java.util.Arrays; import org.apache.beam.sdk.coders.Coder; - import org.apache.flink.api.common.typeutils.TypeComparator; import org.apache.flink.core.memory.DataInputView; import org.apache.flink.core.memory.DataOutputView; import org.apache.flink.core.memory.MemorySegment; -import java.io.IOException; -import java.util.Arrays; - /** * Flink {@link org.apache.flink.api.common.typeutils.TypeComparator} for Beam values that have * been encoded to byte data by a {@link Coder}. diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/EncodedValueSerializer.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/EncodedValueSerializer.java index 33af8d9577239..f3e667d575a5f 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/EncodedValueSerializer.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/EncodedValueSerializer.java @@ -17,14 +17,12 @@ */ package org.apache.beam.runners.flink.translation.types; +import java.io.IOException; import org.apache.beam.sdk.coders.Coder; - import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.core.memory.DataInputView; import org.apache.flink.core.memory.DataOutputView; -import java.io.IOException; - /** * {@link TypeSerializer} for values that were encoded using a {@link Coder}. */ diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/EncodedValueTypeInformation.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/EncodedValueTypeInformation.java index 46c854f087b81..0315ae3704fb7 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/EncodedValueTypeInformation.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/EncodedValueTypeInformation.java @@ -18,16 +18,12 @@ package org.apache.beam.runners.flink.translation.types; import org.apache.beam.sdk.coders.Coder; - import org.apache.flink.annotation.PublicEvolving; import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.typeinfo.AtomicType; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.common.typeutils.TypeComparator; import org.apache.flink.api.common.typeutils.TypeSerializer; -import org.apache.flink.api.common.typeutils.base.array.BytePrimitiveArraySerializer; - -import java.util.Objects; /** * Flink {@link TypeInformation} for Beam values that have been encoded to byte data diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/FlinkCoder.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/FlinkCoder.java index 3b1e66e8e8ff9..8b90c73a26fbd 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/FlinkCoder.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/FlinkCoder.java @@ -17,6 +17,11 @@ */ package org.apache.beam.runners.flink.translation.types; +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.util.Collections; +import java.util.List; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.StandardCoder; import org.apache.flink.api.common.ExecutionConfig; @@ -25,12 +30,6 @@ import org.apache.flink.core.memory.DataInputViewStreamWrapper; import org.apache.flink.core.memory.DataOutputViewStreamWrapper; -import java.io.IOException; -import java.io.InputStream; -import java.io.OutputStream; -import java.util.Collections; -import java.util.List; - /** * A Coder that uses Flink's serialization system. * @param The type of the value to be encoded diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/KvKeySelector.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/KvKeySelector.java index 80d20cae9d9bd..9df683620ec1f 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/KvKeySelector.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/KvKeySelector.java @@ -21,7 +21,6 @@ import org.apache.beam.sdk.util.CoderUtils; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; - import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.functions.KeySelector; import org.apache.flink.api.java.typeutils.ResultTypeQueryable; diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/utils/SerializedPipelineOptions.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/utils/SerializedPipelineOptions.java index 44af0ea406a7a..0c6cea8dfd29a 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/utils/SerializedPipelineOptions.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/utils/SerializedPipelineOptions.java @@ -20,13 +20,11 @@ import static com.google.common.base.Preconditions.checkNotNull; -import org.apache.beam.sdk.options.PipelineOptions; - import com.fasterxml.jackson.databind.ObjectMapper; - import java.io.ByteArrayOutputStream; import java.io.IOException; import java.io.Serializable; +import org.apache.beam.sdk.options.PipelineOptions; /** * Encapsulates the PipelineOptions in serialized form to ship them to the cluster. diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/DataInputViewWrapper.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/DataInputViewWrapper.java index f1b8c7387e40e..82a2c4ed2b9f1 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/DataInputViewWrapper.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/DataInputViewWrapper.java @@ -17,11 +17,10 @@ */ package org.apache.beam.runners.flink.translation.wrappers; -import org.apache.flink.core.memory.DataInputView; - import java.io.EOFException; import java.io.IOException; import java.io.InputStream; +import org.apache.flink.core.memory.DataInputView; /** * Wrapper for {@link DataInputView}. We need this because Flink reads data using a diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/DataOutputViewWrapper.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/DataOutputViewWrapper.java index 148f9607c21e6..2cb9b188ff064 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/DataOutputViewWrapper.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/DataOutputViewWrapper.java @@ -17,10 +17,9 @@ */ package org.apache.beam.runners.flink.translation.wrappers; -import org.apache.flink.core.memory.DataOutputView; - import java.io.IOException; import java.io.OutputStream; +import org.apache.flink.core.memory.DataOutputView; /** * Wrapper for {@link org.apache.flink.core.memory.DataOutputView}. We need this because diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/SerializableFnAggregatorWrapper.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/SerializableFnAggregatorWrapper.java index 82d3fb8ffae30..25d777a3b1252 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/SerializableFnAggregatorWrapper.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/SerializableFnAggregatorWrapper.java @@ -17,15 +17,12 @@ */ package org.apache.beam.runners.flink.translation.wrappers; -import org.apache.beam.sdk.transforms.Aggregator; -import org.apache.beam.sdk.transforms.Combine; - import com.google.common.collect.ImmutableList; import com.google.common.collect.Lists; - -import org.apache.flink.api.common.accumulators.Accumulator; - import java.io.Serializable; +import org.apache.beam.sdk.transforms.Aggregator; +import org.apache.beam.sdk.transforms.Combine; +import org.apache.flink.api.common.accumulators.Accumulator; /** * Wrapper that wraps a {@link org.apache.beam.sdk.transforms.Combine.CombineFn} diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/SourceInputFormat.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/SourceInputFormat.java index 1d06b1ac2fc96..443378f8d1f7a 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/SourceInputFormat.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/SourceInputFormat.java @@ -17,6 +17,8 @@ */ package org.apache.beam.runners.flink.translation.wrappers; +import java.io.IOException; +import java.util.List; import org.apache.beam.runners.flink.translation.utils.SerializedPipelineOptions; import org.apache.beam.sdk.io.BoundedSource; import org.apache.beam.sdk.io.Source; @@ -24,7 +26,6 @@ import org.apache.beam.sdk.transforms.windowing.GlobalWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.util.WindowedValue; - import org.apache.flink.api.common.io.DefaultInputSplitAssigner; import org.apache.flink.api.common.io.InputFormat; import org.apache.flink.api.common.io.statistics.BaseStatistics; @@ -34,9 +35,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.io.IOException; -import java.util.List; - /** * Wrapper for executing a {@link Source} as a Flink {@link InputFormat}. diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/SourceInputSplit.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/SourceInputSplit.java index c3672c0c3ce76..e4a738695be01 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/SourceInputSplit.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/SourceInputSplit.java @@ -18,7 +18,6 @@ package org.apache.beam.runners.flink.translation.wrappers; import org.apache.beam.sdk.io.Source; - import org.apache.flink.core.io.InputSplit; /** 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 092a22665f8cb..000d69f34823e 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 @@ -18,7 +18,15 @@ package org.apache.beam.runners.flink.translation.wrappers.streaming; import avro.shaded.com.google.common.base.Preconditions; - +import com.google.common.collect.Iterables; +import java.io.IOException; +import java.io.Serializable; +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.Collection; +import java.util.HashMap; +import java.util.List; +import java.util.Map; import org.apache.beam.runners.core.SideInputHandler; import org.apache.beam.runners.flink.translation.utils.SerializedPipelineOptions; import org.apache.beam.runners.flink.translation.wrappers.SerializableFnAggregatorWrapper; @@ -44,9 +52,6 @@ import org.apache.beam.sdk.util.state.StateInternals; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TupleTag; - -import com.google.common.collect.Iterables; - import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.functions.ReduceFunction; import org.apache.flink.api.common.state.ListState; @@ -69,15 +74,6 @@ import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; import org.apache.flink.streaming.runtime.tasks.StreamTaskState; -import java.io.IOException; -import java.io.Serializable; -import java.nio.ByteBuffer; -import java.util.ArrayList; -import java.util.Collection; -import java.util.HashMap; -import java.util.List; -import java.util.Map; - /** * Flink operator for executing {@link DoFn DoFns}. * diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/FlinkStateInternals.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/FlinkStateInternals.java index a3cf2e268740c..2e10400591080 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/FlinkStateInternals.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/FlinkStateInternals.java @@ -17,6 +17,11 @@ */ package org.apache.beam.runners.flink.translation.wrappers.streaming; +import com.google.common.collect.Iterables; +import com.google.common.collect.Lists; +import java.nio.ByteBuffer; +import java.util.HashMap; +import java.util.Map; import org.apache.beam.runners.flink.translation.types.CoderTypeInformation; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.CoderException; @@ -38,20 +43,12 @@ import org.apache.beam.sdk.util.state.StateTag; import org.apache.beam.sdk.util.state.ValueState; import org.apache.beam.sdk.util.state.WatermarkHoldState; - -import com.google.common.collect.Iterables; -import com.google.common.collect.Lists; - import org.apache.flink.api.common.state.ListStateDescriptor; import org.apache.flink.api.common.state.ValueStateDescriptor; import org.apache.flink.api.common.typeutils.base.StringSerializer; import org.apache.flink.runtime.state.AbstractStateBackend; import org.joda.time.Instant; -import java.nio.ByteBuffer; -import java.util.HashMap; -import java.util.Map; - /** * {@link StateInternals} that uses a Flink {@link AbstractStateBackend} to * manage state. 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 94bf3af451ce6..5751aac781239 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 @@ -17,12 +17,11 @@ */ package org.apache.beam.runners.flink.translation.wrappers.streaming; +import java.util.Collections; import org.apache.beam.sdk.util.KeyedWorkItem; import org.apache.beam.sdk.util.TimerInternals; import org.apache.beam.sdk.util.WindowedValue; -import java.util.Collections; - public class SingletonKeyedWorkItem implements KeyedWorkItem { final K key; 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 323f5722dafac..5e583e9013c27 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 @@ -19,6 +19,13 @@ import static com.google.common.base.Preconditions.checkArgument; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.collect.ImmutableList; +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.util.List; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.CoderException; import org.apache.beam.sdk.coders.StandardCoder; @@ -28,16 +35,6 @@ import org.apache.beam.sdk.util.PropertyNames; import org.apache.beam.sdk.util.WindowedValue; -import com.google.common.collect.ImmutableList; - -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonProperty; - -import java.io.IOException; -import java.io.InputStream; -import java.io.OutputStream; -import java.util.List; - public class SingletonKeyedWorkItemCoder extends StandardCoder> { /** * Create a new {@link KeyedWorkItemCoder} with the provided key coder, element coder, and window 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 73c1eed96136c..c6dde5197dce0 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 @@ -17,6 +17,22 @@ */ package org.apache.beam.runners.flink.translation.wrappers.streaming; +import java.io.DataInputStream; +import java.io.DataOutputStream; +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.nio.ByteBuffer; +import java.util.Collection; +import java.util.Collections; +import java.util.Comparator; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.PriorityQueue; +import java.util.Queue; +import java.util.Set; +import javax.annotation.Nullable; import org.apache.beam.runners.core.GroupAlsoByWindowViaWindowSetDoFn; import org.apache.beam.runners.flink.translation.wrappers.DataInputViewWrapper; import org.apache.beam.sdk.coders.Coder; @@ -37,7 +53,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; @@ -47,23 +62,6 @@ import org.apache.flink.streaming.runtime.tasks.StreamTaskState; import org.joda.time.Instant; -import java.io.DataInputStream; -import java.io.DataOutputStream; -import java.io.IOException; -import java.io.InputStream; -import java.io.OutputStream; -import java.nio.ByteBuffer; -import java.util.Collection; -import java.util.Collections; -import java.util.Comparator; -import java.util.HashSet; -import java.util.List; -import java.util.Map; -import java.util.PriorityQueue; -import java.util.Queue; -import java.util.Set; -import javax.annotation.Nullable; - /** * Flink operator for executing window {@link DoFn DoFns}. * 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 2bbed58f63e72..51d9e0c9399cb 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 @@ -17,19 +17,16 @@ */ package org.apache.beam.runners.flink.translation.wrappers.streaming; +import java.nio.ByteBuffer; import org.apache.beam.sdk.coders.Coder; -import org.apache.beam.sdk.coders.VoidCoder; 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; import org.apache.flink.api.java.typeutils.GenericTypeInfo; import org.apache.flink.api.java.typeutils.ResultTypeQueryable; -import java.nio.ByteBuffer; - /** * {@link KeySelector} that retrieves a key from a {@link KeyedWorkItem}. This will return * the key as encoded by the provided {@link Coder} in a {@link ByteBuffer}. This ensures diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/BoundedSourceWrapper.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/BoundedSourceWrapper.java index 9d983b06cc6a6..3cb93c0519b3e 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/BoundedSourceWrapper.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/BoundedSourceWrapper.java @@ -17,15 +17,15 @@ */ package org.apache.beam.runners.flink.translation.wrappers.streaming.io; +import com.google.common.annotations.VisibleForTesting; +import java.util.ArrayList; +import java.util.List; import org.apache.beam.runners.flink.translation.utils.SerializedPipelineOptions; import org.apache.beam.sdk.io.BoundedSource; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.windowing.GlobalWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.util.WindowedValue; - -import com.google.common.annotations.VisibleForTesting; - import org.apache.flink.streaming.api.functions.source.RichParallelSourceFunction; import org.apache.flink.streaming.api.operators.StreamSource; import org.apache.flink.streaming.api.watermark.Watermark; @@ -33,9 +33,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.util.ArrayList; -import java.util.List; - /** * Wrapper for executing {@link BoundedSource UnboundedSources} as a Flink Source. */ diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/UnboundedFlinkSink.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/UnboundedFlinkSink.java index 098473dec2301..2117e9d193533 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/UnboundedFlinkSink.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/UnboundedFlinkSink.java @@ -18,23 +18,19 @@ package org.apache.beam.runners.flink.translation.wrappers.streaming.io; +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.util.Collection; +import java.util.List; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.CoderException; import org.apache.beam.sdk.io.Sink; import org.apache.beam.sdk.io.UnboundedSource; -import org.apache.beam.sdk.io.Write; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.util.CloudObject; import org.apache.beam.sdk.util.common.ElementByteSizeObserver; import org.apache.flink.streaming.api.functions.sink.SinkFunction; -import org.apache.flink.streaming.api.functions.source.SourceFunction; - -import javax.annotation.Nullable; -import java.io.IOException; -import java.io.InputStream; -import java.io.OutputStream; -import java.util.Collection; -import java.util.List; /** * A wrapper translating Flink sinks implementing the {@link SinkFunction} interface, into diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/UnboundedFlinkSource.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/UnboundedFlinkSource.java index 716ca304e5399..c6e0825648f46 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/UnboundedFlinkSource.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/UnboundedFlinkSource.java @@ -19,18 +19,15 @@ import static com.google.common.base.Preconditions.checkNotNull; +import java.util.List; +import javax.annotation.Nullable; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.io.UnboundedSource; import org.apache.beam.sdk.options.PipelineOptions; - import org.apache.flink.streaming.api.functions.AssignerWithPeriodicWatermarks; import org.apache.flink.streaming.api.functions.IngestionTimeExtractor; import org.apache.flink.streaming.api.functions.source.SourceFunction; -import java.util.List; - -import javax.annotation.Nullable; - /** * A wrapper translating Flink Sources implementing the {@link SourceFunction} interface, into * unbounded Beam sources (see {@link UnboundedSource}). diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/UnboundedSocketSource.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/UnboundedSocketSource.java index 08bdb509da70e..8d37fe70dfb8b 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/UnboundedSocketSource.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/UnboundedSocketSource.java @@ -19,15 +19,6 @@ import static com.google.common.base.Preconditions.checkArgument; -import org.apache.beam.sdk.coders.Coder; -import org.apache.beam.sdk.coders.StringUtf8Coder; -import org.apache.beam.sdk.io.UnboundedSource; -import org.apache.beam.sdk.options.PipelineOptions; - -import org.joda.time.Instant; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - import java.io.BufferedReader; import java.io.IOException; import java.io.InputStreamReader; @@ -37,8 +28,14 @@ import java.util.Collections; import java.util.List; import java.util.NoSuchElementException; - import javax.annotation.Nullable; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.coders.StringUtf8Coder; +import org.apache.beam.sdk.io.UnboundedSource; +import org.apache.beam.sdk.options.PipelineOptions; +import org.joda.time.Instant; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * An example unbounded Beam source that reads input from a socket. This is used mainly for testing and debugging. diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/UnboundedSourceWrapper.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/UnboundedSourceWrapper.java index 7f26a658bc118..8647322dfb865 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/UnboundedSourceWrapper.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/UnboundedSourceWrapper.java @@ -17,6 +17,12 @@ */ package org.apache.beam.runners.flink.translation.wrappers.streaming.io; +import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Function; +import com.google.common.collect.Lists; +import java.io.ByteArrayInputStream; +import java.util.ArrayList; +import java.util.List; import org.apache.beam.runners.flink.translation.utils.SerializedPipelineOptions; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.KvCoder; @@ -29,11 +35,6 @@ import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.TypeDescriptor; - -import com.google.common.annotations.VisibleForTesting; -import com.google.common.base.Function; -import com.google.common.collect.Lists; - import org.apache.commons.io.output.ByteArrayOutputStream; import org.apache.flink.configuration.Configuration; import org.apache.flink.streaming.api.checkpoint.Checkpointed; @@ -46,10 +47,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.io.ByteArrayInputStream; -import java.util.ArrayList; -import java.util.List; - /** * Wrapper for executing {@link UnboundedSource UnboundedSources} as a Flink Source. */ diff --git a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/EncodedValueComparatorTest.java b/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/EncodedValueComparatorTest.java index 68ede89954cbd..10d6d9d357ff8 100644 --- a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/EncodedValueComparatorTest.java +++ b/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/EncodedValueComparatorTest.java @@ -22,7 +22,6 @@ import org.apache.beam.sdk.coders.CoderException; import org.apache.beam.sdk.coders.StringUtf8Coder; import org.apache.beam.sdk.util.CoderUtils; - import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.typeutils.ComparatorTestBase; import org.apache.flink.api.common.typeutils.TypeComparator; diff --git a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/FlinkRunnerRegistrarTest.java b/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/FlinkRunnerRegistrarTest.java index ff1025f3ded77..d9d174c8248e9 100644 --- a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/FlinkRunnerRegistrarTest.java +++ b/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/FlinkRunnerRegistrarTest.java @@ -22,7 +22,6 @@ import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.options.PipelineOptionsFactory; - import org.junit.Test; /** 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 77e8a477e76af..32339dce5165f 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 @@ -21,6 +21,8 @@ import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertTrue; +import java.util.Collections; +import java.util.HashMap; import org.apache.beam.runners.flink.translation.utils.SerializedPipelineOptions; import org.apache.beam.runners.flink.translation.wrappers.streaming.DoFnOperator; import org.apache.beam.sdk.options.Default; @@ -34,7 +36,6 @@ import org.apache.beam.sdk.util.WindowingStrategy; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TupleTag; - import org.apache.commons.lang.SerializationUtils; import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.typeinfo.TypeHint; @@ -46,9 +47,6 @@ import org.junit.BeforeClass; import org.junit.Test; -import java.util.Collections; -import java.util.HashMap; - /** * Tests for serialization and deserialization of {@link PipelineOptions} in {@link DoFnOperator}. */ diff --git a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/ReadSourceITCase.java b/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/ReadSourceITCase.java index 516c7bae75e52..44c90178fefcf 100644 --- a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/ReadSourceITCase.java +++ b/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/ReadSourceITCase.java @@ -17,20 +17,17 @@ */ package org.apache.beam.runners.flink; +import com.google.common.base.Joiner; +import java.io.File; +import java.net.URI; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.io.CountingInput; import org.apache.beam.sdk.io.TextIO; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.ParDo; import org.apache.beam.sdk.values.PCollection; - -import com.google.common.base.Joiner; - import org.apache.flink.test.util.JavaProgramTestBase; -import java.io.File; -import java.net.URI; - /** * Reads from a bounded source in batch execution. */ diff --git a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/ReadSourceStreamingITCase.java b/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/ReadSourceStreamingITCase.java index ea58d0dd86570..79b78826ede9e 100644 --- a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/ReadSourceStreamingITCase.java +++ b/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/ReadSourceStreamingITCase.java @@ -17,14 +17,12 @@ */ package org.apache.beam.runners.flink; +import com.google.common.base.Joiner; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.io.CountingInput; import org.apache.beam.sdk.io.TextIO; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.ParDo; - -import com.google.common.base.Joiner; - import org.apache.flink.streaming.util.StreamingProgramTestBase; /** diff --git a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/WriteSinkITCase.java b/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/WriteSinkITCase.java index f1d9097b5924a..09881463f6c23 100644 --- a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/WriteSinkITCase.java +++ b/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/WriteSinkITCase.java @@ -20,6 +20,11 @@ import static org.junit.Assert.assertNotNull; +import com.google.common.base.Joiner; +import java.io.File; +import java.io.IOException; +import java.io.PrintWriter; +import java.net.URI; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.StringUtf8Coder; @@ -27,18 +32,10 @@ import org.apache.beam.sdk.io.Write; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.Create; - -import com.google.common.base.Joiner; - import org.apache.flink.core.fs.FileSystem; import org.apache.flink.core.fs.Path; import org.apache.flink.test.util.JavaProgramTestBase; -import java.io.File; -import java.io.IOException; -import java.io.PrintWriter; -import java.net.URI; - /** * Tests the translation of custom Write.Bound sinks. */ 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 fb1b1e8d1afcc..5f1b066d620b1 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 @@ -20,6 +20,10 @@ import static org.hamcrest.collection.IsIterableContainingInOrder.contains; import static org.junit.Assert.assertThat; +import com.google.common.collect.ImmutableList; +import java.util.Collections; +import java.util.HashMap; +import javax.annotation.Nullable; import org.apache.beam.runners.flink.FlinkPipelineOptions; import org.apache.beam.runners.flink.translation.types.CoderTypeInformation; import org.apache.beam.runners.flink.translation.wrappers.streaming.DoFnOperator; @@ -36,9 +40,6 @@ import org.apache.beam.sdk.util.WindowingStrategy; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TupleTag; - -import com.google.common.collect.ImmutableList; - import org.apache.flink.shaded.com.google.common.base.Function; import org.apache.flink.shaded.com.google.common.base.Predicate; import org.apache.flink.shaded.com.google.common.collect.FluentIterable; @@ -53,10 +54,6 @@ import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.util.Collections; -import java.util.HashMap; -import javax.annotation.Nullable; - /** * Tests for {@link DoFnOperator}. */ diff --git a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/streaming/FlinkStateInternalsTest.java b/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/streaming/FlinkStateInternalsTest.java index 627f5458dc1f3..711ae0001f795 100644 --- a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/streaming/FlinkStateInternalsTest.java +++ b/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/streaming/FlinkStateInternalsTest.java @@ -22,6 +22,8 @@ import static org.junit.Assert.assertNotEquals; import static org.junit.Assert.assertThat; +import java.nio.ByteBuffer; +import java.util.Arrays; import org.apache.beam.runners.flink.translation.wrappers.streaming.FlinkStateInternals; import org.apache.beam.sdk.coders.CoderException; import org.apache.beam.sdk.coders.StringUtf8Coder; @@ -42,7 +44,6 @@ 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.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.java.typeutils.GenericTypeInfo; import org.apache.flink.runtime.operators.testutils.DummyEnvironment; @@ -54,9 +55,6 @@ import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.nio.ByteBuffer; -import java.util.Arrays; - /** * Tests for {@link FlinkStateInternals}. This is based on the tests for {@code InMemoryStateInternals}. */ 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 ca183a855b180..ab98c27733b3c 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 @@ -17,28 +17,25 @@ */ package org.apache.beam.runners.flink.streaming; +import com.google.common.base.Joiner; +import java.io.Serializable; +import java.util.Arrays; import org.apache.beam.runners.flink.FlinkTestPipeline; 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.OldDoFn; 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; import org.apache.beam.sdk.transforms.windowing.Window; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; - -import com.google.common.base.Joiner; - import org.apache.flink.streaming.util.StreamingProgramTestBase; import org.joda.time.Duration; import org.joda.time.Instant; -import java.io.Serializable; -import java.util.Arrays; - public class GroupByNullKeyTest extends StreamingProgramTestBase implements Serializable { diff --git a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/streaming/TestCountingSource.java b/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/streaming/TestCountingSource.java index 3ced02ee3038c..9251d42f4404b 100644 --- a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/streaming/TestCountingSource.java +++ b/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/streaming/TestCountingSource.java @@ -19,6 +19,11 @@ import static org.apache.beam.sdk.util.CoderUtils.encodeToByteArray; +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.ThreadLocalRandom; +import javax.annotation.Nullable; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.DelegateCoder; import org.apache.beam.sdk.coders.KvCoder; @@ -26,17 +31,10 @@ import org.apache.beam.sdk.io.UnboundedSource; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.values.KV; - import org.joda.time.Instant; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.io.IOException; -import java.util.ArrayList; -import java.util.List; -import java.util.concurrent.ThreadLocalRandom; -import javax.annotation.Nullable; - /** * An unbounded source for testing the unbounded sources framework code. * 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 7912aee2409f9..64f978fa0f234 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 @@ -17,6 +17,10 @@ */ package org.apache.beam.runners.flink.streaming; +import com.google.api.services.bigquery.model.TableRow; +import com.google.common.base.Joiner; +import java.io.Serializable; +import java.util.Arrays; import org.apache.beam.runners.flink.FlinkTestPipeline; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.io.TextIO; @@ -28,17 +32,10 @@ import org.apache.beam.sdk.transforms.windowing.Window; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; - -import com.google.api.services.bigquery.model.TableRow; -import com.google.common.base.Joiner; - import org.apache.flink.streaming.util.StreamingProgramTestBase; import org.joda.time.Duration; import org.joda.time.Instant; -import java.io.Serializable; -import java.util.Arrays; - /** * Session window test diff --git a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/streaming/UnboundedSourceWrapperTest.java b/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/streaming/UnboundedSourceWrapperTest.java index 11221797456b7..a70ad49b9f73a 100644 --- a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/streaming/UnboundedSourceWrapperTest.java +++ b/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/streaming/UnboundedSourceWrapperTest.java @@ -23,12 +23,14 @@ import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; +import java.util.Collections; +import java.util.HashSet; +import java.util.Set; import org.apache.beam.runners.flink.translation.wrappers.streaming.io.UnboundedSourceWrapper; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; - import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.accumulators.Accumulator; import org.apache.flink.configuration.Configuration; @@ -43,10 +45,6 @@ import org.apache.flink.streaming.runtime.tasks.StreamTask; import org.junit.Test; -import java.util.Collections; -import java.util.HashSet; -import java.util.Set; - /** * Tests for {@link UnboundedSourceWrapper}. */ diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/BlockingDataflowRunner.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/BlockingDataflowRunner.java index e7cd67e6b8735..d26536136a79f 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/BlockingDataflowRunner.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/BlockingDataflowRunner.java @@ -17,6 +17,8 @@ */ package org.apache.beam.runners.dataflow; +import java.io.IOException; +import javax.annotation.Nullable; import org.apache.beam.runners.dataflow.options.BlockingDataflowPipelineOptions; import org.apache.beam.runners.dataflow.util.MonitoringUtil; import org.apache.beam.sdk.Pipeline; @@ -28,15 +30,10 @@ import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.values.PInput; import org.apache.beam.sdk.values.POutput; - import org.joda.time.Duration; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.io.IOException; - -import javax.annotation.Nullable; - /** * A {@link PipelineRunner} that's like {@link DataflowRunner} * but that waits for the launched job to finish. diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowJobException.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowJobException.java index 74c0f80684a25..6e32fdecaf9b8 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowJobException.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowJobException.java @@ -18,7 +18,6 @@ package org.apache.beam.runners.dataflow; import java.util.Objects; - import javax.annotation.Nullable; /** 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 3d0f1458fbae2..9a515fa6140f3 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 @@ -19,17 +19,6 @@ import static org.apache.beam.runners.dataflow.util.TimeUtil.fromCloudTime; -import org.apache.beam.runners.dataflow.internal.DataflowAggregatorTransforms; -import org.apache.beam.runners.dataflow.internal.DataflowMetricUpdateExtractor; -import org.apache.beam.runners.dataflow.options.DataflowPipelineOptions; -import org.apache.beam.runners.dataflow.util.MonitoringUtil; -import org.apache.beam.sdk.AggregatorRetrievalException; -import org.apache.beam.sdk.AggregatorValues; -import org.apache.beam.sdk.PipelineResult; -import org.apache.beam.sdk.transforms.Aggregator; -import org.apache.beam.sdk.util.AttemptAndTimeBoundedExponentialBackOff; -import org.apache.beam.sdk.util.AttemptBoundedExponentialBackOff; - import com.google.api.client.googleapis.json.GoogleJsonResponseException; import com.google.api.client.util.BackOff; import com.google.api.client.util.BackOffUtils; @@ -41,18 +30,25 @@ import com.google.api.services.dataflow.model.MetricUpdate; import com.google.common.annotations.VisibleForTesting; import com.google.common.base.MoreObjects; - -import org.joda.time.Duration; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - import java.io.IOException; import java.net.SocketTimeoutException; import java.util.List; import java.util.Map; import java.util.concurrent.TimeUnit; - import javax.annotation.Nullable; +import org.apache.beam.runners.dataflow.internal.DataflowAggregatorTransforms; +import org.apache.beam.runners.dataflow.internal.DataflowMetricUpdateExtractor; +import org.apache.beam.runners.dataflow.options.DataflowPipelineOptions; +import org.apache.beam.runners.dataflow.util.MonitoringUtil; +import org.apache.beam.sdk.AggregatorRetrievalException; +import org.apache.beam.sdk.AggregatorValues; +import org.apache.beam.sdk.PipelineResult; +import org.apache.beam.sdk.transforms.Aggregator; +import org.apache.beam.sdk.util.AttemptAndTimeBoundedExponentialBackOff; +import org.apache.beam.sdk.util.AttemptBoundedExponentialBackOff; +import org.joda.time.Duration; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * A DataflowPipelineJob represents a job submitted to Dataflow using diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineRegistrar.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineRegistrar.java index 4c88c4e0f0107..5090a8a9b616d 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineRegistrar.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineRegistrar.java @@ -17,6 +17,8 @@ */ package org.apache.beam.runners.dataflow; +import com.google.auto.service.AutoService; +import com.google.common.collect.ImmutableList; import org.apache.beam.runners.dataflow.options.BlockingDataflowPipelineOptions; import org.apache.beam.runners.dataflow.options.DataflowPipelineOptions; import org.apache.beam.sdk.options.PipelineOptions; @@ -24,9 +26,6 @@ import org.apache.beam.sdk.runners.PipelineRunner; import org.apache.beam.sdk.runners.PipelineRunnerRegistrar; -import com.google.auto.service.AutoService; -import com.google.common.collect.ImmutableList; - /** * Contains the {@link PipelineOptionsRegistrar} and {@link PipelineRunnerRegistrar} for the * {@link DataflowRunner}. 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 b8d94452872c1..0d72881229955 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 @@ -17,6 +17,10 @@ */ package org.apache.beam.runners.dataflow; +import static com.google.common.base.Preconditions.checkArgument; +import static com.google.common.base.Preconditions.checkNotNull; +import static com.google.common.base.Preconditions.checkState; +import static com.google.common.base.Strings.isNullOrEmpty; import static org.apache.beam.sdk.util.SerializableUtils.serializeToByteArray; import static org.apache.beam.sdk.util.StringUtils.byteArrayToJsonString; import static org.apache.beam.sdk.util.StringUtils.jsonStringToByteArray; @@ -28,11 +32,24 @@ import static org.apache.beam.sdk.util.Structs.addString; import static org.apache.beam.sdk.util.Structs.getString; -import static com.google.common.base.Preconditions.checkArgument; -import static com.google.common.base.Preconditions.checkNotNull; -import static com.google.common.base.Preconditions.checkState; -import static com.google.common.base.Strings.isNullOrEmpty; - +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.api.services.dataflow.model.AutoscalingSettings; +import com.google.api.services.dataflow.model.DataflowPackage; +import com.google.api.services.dataflow.model.Disk; +import com.google.api.services.dataflow.model.Environment; +import com.google.api.services.dataflow.model.Job; +import com.google.api.services.dataflow.model.Step; +import com.google.api.services.dataflow.model.WorkerPool; +import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.LinkedList; +import java.util.List; +import java.util.Map; +import javax.annotation.Nullable; import org.apache.beam.runners.dataflow.DataflowRunner.GroupByKeyAndSortValuesOnly; import org.apache.beam.runners.dataflow.internal.ReadTranslator; import org.apache.beam.runners.dataflow.options.DataflowPipelineOptions; @@ -71,30 +88,9 @@ import org.apache.beam.sdk.values.PValue; import org.apache.beam.sdk.values.TupleTag; import org.apache.beam.sdk.values.TypedPValue; - -import com.google.api.services.dataflow.model.AutoscalingSettings; -import com.google.api.services.dataflow.model.DataflowPackage; -import com.google.api.services.dataflow.model.Disk; -import com.google.api.services.dataflow.model.Environment; -import com.google.api.services.dataflow.model.Job; -import com.google.api.services.dataflow.model.Step; -import com.google.api.services.dataflow.model.WorkerPool; - -import com.fasterxml.jackson.core.JsonProcessingException; -import com.fasterxml.jackson.databind.ObjectMapper; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.io.IOException; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collections; -import java.util.HashMap; -import java.util.LinkedList; -import java.util.List; -import java.util.Map; -import javax.annotation.Nullable; - /** * {@link DataflowPipelineTranslator} knows how to translate {@link Pipeline} objects * into Cloud Dataflow Service API {@link Job}s. 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 7f632a1a72027..a0e24b14a5755 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 @@ -17,13 +17,61 @@ */ package org.apache.beam.runners.dataflow; -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 com.google.common.base.Preconditions.checkArgument; import static com.google.common.base.Preconditions.checkState; 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 com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.api.client.googleapis.json.GoogleJsonResponseException; +import com.google.api.services.clouddebugger.v2.Clouddebugger; +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; +import com.google.api.services.dataflow.model.WorkerPool; +import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Function; +import com.google.common.base.Joiner; +import com.google.common.base.Optional; +import com.google.common.base.Utf8; +import com.google.common.collect.ForwardingMap; +import com.google.common.collect.HashMultimap; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Iterables; +import com.google.common.collect.Maps; +import com.google.common.collect.Multimap; +import java.io.File; +import java.io.FileNotFoundException; +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.io.PrintWriter; +import java.io.Serializable; +import java.net.URISyntaxException; +import java.net.URL; +import java.net.URLClassLoader; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Random; +import java.util.Set; +import java.util.SortedSet; +import java.util.TreeSet; +import javax.annotation.Nullable; import org.apache.beam.runners.dataflow.DataflowPipelineTranslator.JobSpecification; import org.apache.beam.runners.dataflow.DataflowPipelineTranslator.TransformTranslator; import org.apache.beam.runners.dataflow.DataflowPipelineTranslator.TranslationContext; @@ -114,65 +162,12 @@ import org.apache.beam.sdk.values.PValue; import org.apache.beam.sdk.values.TupleTag; import org.apache.beam.sdk.values.TupleTagList; - -import com.google.api.client.googleapis.json.GoogleJsonResponseException; -import com.google.api.services.clouddebugger.v2.Clouddebugger; -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; -import com.google.api.services.dataflow.model.WorkerPool; -import com.google.common.annotations.VisibleForTesting; -import com.google.common.base.Function; -import com.google.common.base.Joiner; -import com.google.common.base.Optional; -import com.google.common.base.Utf8; -import com.google.common.collect.ForwardingMap; -import com.google.common.collect.HashMultimap; -import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableMap; -import com.google.common.collect.Iterables; -import com.google.common.collect.Maps; -import com.google.common.collect.Multimap; - -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonProperty; - import org.joda.time.DateTimeUtils; import org.joda.time.DateTimeZone; import org.joda.time.format.DateTimeFormat; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.io.File; -import java.io.FileNotFoundException; -import java.io.IOException; -import java.io.InputStream; -import java.io.OutputStream; -import java.io.PrintWriter; -import java.io.Serializable; -import java.net.URISyntaxException; -import java.net.URL; -import java.net.URLClassLoader; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collection; -import java.util.Collections; -import java.util.HashMap; -import java.util.HashSet; -import java.util.Iterator; -import java.util.List; -import java.util.Map; -import java.util.Random; -import java.util.Set; -import java.util.SortedSet; -import java.util.TreeSet; - -import javax.annotation.Nullable; - /** * A {@link PipelineRunner} that executes the operations in the pipeline by first translating them * to the Dataflow representation using the {@link DataflowPipelineTranslator} and then submitting diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunnerHooks.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunnerHooks.java index 05297ec52046d..b67421dd386a2 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunnerHooks.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunnerHooks.java @@ -17,9 +17,8 @@ */ package org.apache.beam.runners.dataflow; -import org.apache.beam.sdk.annotations.Experimental; - import com.google.api.services.dataflow.model.Environment; +import org.apache.beam.sdk.annotations.Experimental; /** * An instance of this class can be passed to the diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/internal/CustomSources.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/internal/CustomSources.java index 73e5da02f464c..aeb81035c6d08 100755 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/internal/CustomSources.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/internal/CustomSources.java @@ -17,30 +17,26 @@ */ package org.apache.beam.runners.dataflow.internal; +import static com.google.api.client.util.Base64.encodeBase64String; +import static com.google.common.base.Preconditions.checkArgument; import static org.apache.beam.sdk.util.SerializableUtils.serializeToByteArray; import static org.apache.beam.sdk.util.Structs.addString; import static org.apache.beam.sdk.util.Structs.addStringList; -import static com.google.api.client.util.Base64.encodeBase64String; -import static com.google.common.base.Preconditions.checkArgument; - +import com.google.api.services.dataflow.model.SourceMetadata; +import com.google.common.annotations.VisibleForTesting; +import com.google.protobuf.ByteString; +import java.util.ArrayList; +import java.util.List; import org.apache.beam.runners.dataflow.options.DataflowPipelineOptions; import org.apache.beam.sdk.io.BoundedSource; 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.util.CloudObject; - -import com.google.api.services.dataflow.model.SourceMetadata; -import com.google.common.annotations.VisibleForTesting; -import com.google.protobuf.ByteString; - import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.util.ArrayList; -import java.util.List; - /** * A helper class for supporting sources defined as {@code Source}. diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/internal/DataflowAggregatorTransforms.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/internal/DataflowAggregatorTransforms.java index 7a08fde8fc170..fb7897361fa5b 100755 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/internal/DataflowAggregatorTransforms.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/internal/DataflowAggregatorTransforms.java @@ -17,19 +17,17 @@ */ package org.apache.beam.runners.dataflow.internal; -import org.apache.beam.sdk.Pipeline; -import org.apache.beam.sdk.transforms.Aggregator; -import org.apache.beam.sdk.transforms.AppliedPTransform; -import org.apache.beam.sdk.transforms.PTransform; - import com.google.common.collect.BiMap; import com.google.common.collect.HashBiMap; import com.google.common.collect.HashMultimap; import com.google.common.collect.Multimap; - import java.util.Collection; import java.util.HashSet; import java.util.Map; +import org.apache.beam.sdk.Pipeline; +import org.apache.beam.sdk.transforms.Aggregator; +import org.apache.beam.sdk.transforms.AppliedPTransform; +import org.apache.beam.sdk.transforms.PTransform; /** * A mapping relating {@link Aggregator}s and the {@link PTransform} in which they are used. diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/internal/DataflowMetricUpdateExtractor.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/internal/DataflowMetricUpdateExtractor.java index 8ab59fc911d6f..d715437ae68d0 100755 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/internal/DataflowMetricUpdateExtractor.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/internal/DataflowMetricUpdateExtractor.java @@ -17,18 +17,16 @@ */ package org.apache.beam.runners.dataflow.internal; -import org.apache.beam.sdk.transforms.Aggregator; -import org.apache.beam.sdk.transforms.AppliedPTransform; -import org.apache.beam.sdk.transforms.Combine.CombineFn; -import org.apache.beam.sdk.transforms.PTransform; - import com.google.api.services.dataflow.model.MetricStructuredName; import com.google.api.services.dataflow.model.MetricUpdate; - import java.util.Collection; import java.util.HashMap; import java.util.List; import java.util.Map; +import org.apache.beam.sdk.transforms.Aggregator; +import org.apache.beam.sdk.transforms.AppliedPTransform; +import org.apache.beam.sdk.transforms.Combine.CombineFn; +import org.apache.beam.sdk.transforms.PTransform; /** * Methods for extracting the values of an {@link Aggregator} from a collection of {@link 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 283f01221ceb6..85f5e734a2537 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 @@ -17,10 +17,25 @@ */ package org.apache.beam.runners.dataflow.internal; -import static org.apache.beam.sdk.util.StringUtils.approximateSimpleName; import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.base.Preconditions.checkNotNull; +import static org.apache.beam.sdk.util.StringUtils.approximateSimpleName; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Function; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.Lists; +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.util.Arrays; +import java.util.Collections; +import java.util.Iterator; +import java.util.List; +import java.util.NoSuchElementException; +import javax.annotation.Nullable; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.CoderException; import org.apache.beam.sdk.coders.ListCoder; @@ -39,30 +54,10 @@ import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PInput; import org.apache.beam.sdk.values.TimestampedValue; - -import com.google.common.annotations.VisibleForTesting; -import com.google.common.base.Function; -import com.google.common.collect.ImmutableList; -import com.google.common.collect.Lists; - -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonProperty; - import org.joda.time.Instant; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.io.IOException; -import java.io.InputStream; -import java.io.OutputStream; -import java.util.Arrays; -import java.util.Collections; -import java.util.Iterator; -import java.util.List; -import java.util.NoSuchElementException; - -import javax.annotation.Nullable; - /** * {@link PTransform} that converts a {@link BoundedSource} as an {@link UnboundedSource}. * diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/internal/IsmFormat.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/internal/IsmFormat.java index d8bfe42740252..6f4a18bf9d05d 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/internal/IsmFormat.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/internal/IsmFormat.java @@ -17,12 +17,25 @@ */ package org.apache.beam.runners.dataflow.internal; -import static org.apache.beam.sdk.util.Structs.addLong; - import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.base.Preconditions.checkNotNull; import static com.google.common.base.Preconditions.checkState; +import static org.apache.beam.sdk.util.Structs.addLong; +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 com.google.common.hash.HashFunction; +import com.google.common.hash.Hashing; +import java.io.DataInputStream; +import java.io.DataOutputStream; +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.util.ArrayList; +import java.util.List; +import javax.annotation.Nullable; import org.apache.beam.runners.dataflow.util.RandomAccessData; import org.apache.beam.sdk.coders.AtomicCoder; import org.apache.beam.sdk.coders.ByteArrayCoder; @@ -38,24 +51,6 @@ import org.apache.beam.sdk.util.VarInt; import org.apache.beam.sdk.values.PCollection; -import com.google.auto.value.AutoValue; -import com.google.common.collect.ImmutableList; -import com.google.common.hash.HashFunction; -import com.google.common.hash.Hashing; - -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonProperty; - -import java.io.DataInputStream; -import java.io.DataOutputStream; -import java.io.IOException; -import java.io.InputStream; -import java.io.OutputStream; -import java.util.ArrayList; -import java.util.List; - -import javax.annotation.Nullable; - /** * An Ism file is a prefix encoded composite key value file broken into shards. Each composite * key is composed of a fixed number of component keys. A fixed number of those sub keys represent diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/internal/ReadTranslator.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/internal/ReadTranslator.java index 373738a717098..094f4052fc87d 100755 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/internal/ReadTranslator.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/internal/ReadTranslator.java @@ -21,6 +21,9 @@ import static org.apache.beam.sdk.util.Structs.addDictionary; import static org.apache.beam.sdk.util.Structs.addLong; +import com.google.api.services.dataflow.model.SourceMetadata; +import java.util.HashMap; +import java.util.Map; import org.apache.beam.runners.dataflow.DataflowPipelineTranslator; import org.apache.beam.runners.dataflow.DataflowPipelineTranslator.TransformTranslator; import org.apache.beam.runners.dataflow.DataflowPipelineTranslator.TranslationContext; @@ -31,11 +34,6 @@ import org.apache.beam.sdk.util.PropertyNames; import org.apache.beam.sdk.values.PValue; -import com.google.api.services.dataflow.model.SourceMetadata; - -import java.util.HashMap; -import java.util.Map; - /** * Translator for the {@code Read} {@code PTransform} for the Dataflow back-end. */ diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/options/CloudDebuggerOptions.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/options/CloudDebuggerOptions.java index dbfbb16f30220..d1c8e7a7affd8 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/options/CloudDebuggerOptions.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/options/CloudDebuggerOptions.java @@ -17,15 +17,13 @@ */ package org.apache.beam.runners.dataflow.options; +import com.google.api.services.clouddebugger.v2.model.Debuggee; +import javax.annotation.Nullable; import org.apache.beam.sdk.annotations.Experimental; import org.apache.beam.sdk.options.Default; import org.apache.beam.sdk.options.Description; import org.apache.beam.sdk.options.Hidden; -import com.google.api.services.clouddebugger.v2.model.Debuggee; - -import javax.annotation.Nullable; - /** * Options for controlling Cloud Debugger. */ diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/options/DataflowPipelineDebugOptions.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/options/DataflowPipelineDebugOptions.java index ac2e0b72babf4..dfe538da9f057 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/options/DataflowPipelineDebugOptions.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/options/DataflowPipelineDebugOptions.java @@ -17,6 +17,10 @@ */ package org.apache.beam.runners.dataflow.options; +import com.fasterxml.jackson.annotation.JsonIgnore; +import com.google.api.services.dataflow.Dataflow; +import java.util.List; +import java.util.Map; import org.apache.beam.runners.dataflow.util.DataflowTransport; import org.apache.beam.runners.dataflow.util.GcsStager; import org.apache.beam.runners.dataflow.util.Stager; @@ -28,13 +32,6 @@ import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.util.InstanceBuilder; -import com.google.api.services.dataflow.Dataflow; - -import com.fasterxml.jackson.annotation.JsonIgnore; - -import java.util.List; -import java.util.Map; - /** * Internal. Options used to control execution of the Dataflow SDK for * debugging and testing purposes. 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 841741f5562a5..8ef43c5efaf66 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 @@ -20,6 +20,8 @@ import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.base.Strings.isNullOrEmpty; +import com.google.common.base.MoreObjects; +import java.io.IOException; import org.apache.beam.runners.dataflow.DataflowRunner; import org.apache.beam.sdk.options.ApplicationNameOptions; import org.apache.beam.sdk.options.BigQueryOptions; @@ -33,16 +35,11 @@ import org.apache.beam.sdk.options.StreamingOptions; import org.apache.beam.sdk.options.Validation; import org.apache.beam.sdk.util.IOChannelUtils; - -import com.google.common.base.MoreObjects; - import org.joda.time.DateTimeUtils; import org.joda.time.DateTimeZone; import org.joda.time.format.DateTimeFormat; import org.joda.time.format.DateTimeFormatter; -import java.io.IOException; - /** * Options that can be used to configure the {@link DataflowRunner}. */ 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 9b7a9ceb7f742..6c59f381ac6bd 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 @@ -17,6 +17,9 @@ */ package org.apache.beam.runners.dataflow.options; +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.sdk.annotations.Experimental; import org.apache.beam.sdk.options.Default; @@ -25,12 +28,6 @@ import org.apache.beam.sdk.options.Hidden; import org.apache.beam.sdk.options.PipelineOptions; -import com.fasterxml.jackson.annotation.JsonIgnore; - -import java.util.List; - -import javax.annotation.Nullable; - /** * Options that are used to configure the Dataflow pipeline worker pool. */ 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 c7b4c91a0998c..ae8801b495559 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 @@ -17,12 +17,11 @@ */ package org.apache.beam.runners.dataflow.options; +import java.util.HashMap; import org.apache.beam.sdk.annotations.Experimental; import org.apache.beam.sdk.options.Description; import org.apache.beam.sdk.options.Hidden; -import java.util.HashMap; - /** * Options for controlling profiling of pipeline execution. */ diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/options/DataflowWorkerLoggingOptions.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/options/DataflowWorkerLoggingOptions.java index 91ac62aa09473..38c139c68274b 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/options/DataflowWorkerLoggingOptions.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/options/DataflowWorkerLoggingOptions.java @@ -19,15 +19,13 @@ import static com.google.common.base.Preconditions.checkNotNull; -import org.apache.beam.sdk.options.Default; -import org.apache.beam.sdk.options.Description; -import org.apache.beam.sdk.options.PipelineOptions; - import com.fasterxml.jackson.annotation.JsonCreator; - import java.util.Arrays; import java.util.HashMap; import java.util.Map; +import org.apache.beam.sdk.options.Default; +import org.apache.beam.sdk.options.Description; +import org.apache.beam.sdk.options.PipelineOptions; /** * Options that are used to control logging configuration on the Dataflow worker. 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 f74f4dd170397..9be773b9b0840 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 @@ -19,6 +19,18 @@ import static org.hamcrest.MatcherAssert.assertThat; +import com.google.api.services.dataflow.model.JobMessage; +import com.google.api.services.dataflow.model.JobMetrics; +import com.google.api.services.dataflow.model.MetricUpdate; +import com.google.common.base.Joiner; +import com.google.common.base.Optional; +import com.google.common.base.Throwables; +import java.io.IOException; +import java.math.BigDecimal; +import java.util.List; +import java.util.concurrent.Callable; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.Future; import org.apache.beam.runners.dataflow.DataflowJobExecutionException; import org.apache.beam.runners.dataflow.DataflowPipelineJob; import org.apache.beam.runners.dataflow.DataflowRunner; @@ -34,25 +46,10 @@ import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.values.PInput; import org.apache.beam.sdk.values.POutput; - -import com.google.api.services.dataflow.model.JobMessage; -import com.google.api.services.dataflow.model.JobMetrics; -import com.google.api.services.dataflow.model.MetricUpdate; -import com.google.common.base.Joiner; -import com.google.common.base.Optional; -import com.google.common.base.Throwables; - import org.joda.time.Duration; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.io.IOException; -import java.math.BigDecimal; -import java.util.List; -import java.util.concurrent.Callable; -import java.util.concurrent.ExecutionException; -import java.util.concurrent.Future; - /** * {@link TestDataflowRunner} is a pipeline runner that wraps a * {@link DataflowRunner} when running tests against the {@link TestPipeline}. diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/DataflowTransport.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/DataflowTransport.java index f9887491555da..0391594ed57e8 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/DataflowTransport.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/DataflowTransport.java @@ -20,18 +20,16 @@ import static org.apache.beam.sdk.util.Transport.getJsonFactory; import static org.apache.beam.sdk.util.Transport.getTransport; -import org.apache.beam.runners.dataflow.options.DataflowPipelineOptions; -import org.apache.beam.sdk.util.RetryHttpRequestInitializer; - import com.google.api.client.auth.oauth2.Credential; import com.google.api.client.http.HttpRequestInitializer; import com.google.api.services.clouddebugger.v2.Clouddebugger; import com.google.api.services.dataflow.Dataflow; import com.google.cloud.hadoop.util.ChainingHttpRequestInitializer; import com.google.common.collect.ImmutableList; - import java.net.MalformedURLException; import java.net.URL; +import org.apache.beam.runners.dataflow.options.DataflowPipelineOptions; +import org.apache.beam.sdk.util.RetryHttpRequestInitializer; /** * Helpers for cloud communication. diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/DoFnInfo.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/DoFnInfo.java index 2017313b6ee61..139db9da0ab98 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/DoFnInfo.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/DoFnInfo.java @@ -17,13 +17,12 @@ */ package org.apache.beam.runners.dataflow.util; +import java.io.Serializable; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.transforms.OldDoFn; import org.apache.beam.sdk.util.WindowingStrategy; import org.apache.beam.sdk.values.PCollectionView; -import java.io.Serializable; - /** * Wrapper class holding the necessary information to serialize a OldDoFn. * diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/GcsStager.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/GcsStager.java index bf25ce47a98e0..6ca4c3f789130 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/GcsStager.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/GcsStager.java @@ -19,14 +19,12 @@ import static com.google.common.base.Preconditions.checkNotNull; +import com.google.api.services.dataflow.model.DataflowPackage; +import java.util.List; import org.apache.beam.runners.dataflow.options.DataflowPipelineDebugOptions; import org.apache.beam.runners.dataflow.options.DataflowPipelineOptions; import org.apache.beam.sdk.options.PipelineOptions; -import com.google.api.services.dataflow.model.DataflowPackage; - -import java.util.List; - /** * Utility class for staging files to GCS. */ 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 4d12e665c7459..d0146230c66ff 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 @@ -19,9 +19,6 @@ import static org.apache.beam.runners.dataflow.util.TimeUtil.fromCloudTime; -import org.apache.beam.runners.dataflow.options.DataflowPipelineOptions; -import org.apache.beam.sdk.PipelineResult.State; - 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; @@ -29,11 +26,6 @@ import com.google.common.base.MoreObjects; import com.google.common.base.Strings; import com.google.common.collect.ImmutableMap; - -import org.joda.time.Instant; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - import java.io.IOException; import java.io.UnsupportedEncodingException; import java.net.URLEncoder; @@ -42,8 +34,12 @@ import java.util.Comparator; import java.util.List; import java.util.Map; - import javax.annotation.Nullable; +import org.apache.beam.runners.dataflow.options.DataflowPipelineOptions; +import org.apache.beam.sdk.PipelineResult.State; +import org.joda.time.Instant; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * A helper class for monitoring jobs submitted to the service. diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/PackageUtil.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/PackageUtil.java index cff7e2b9c68fe..bf1f666c32c5e 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/PackageUtil.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/PackageUtil.java @@ -17,11 +17,7 @@ */ package org.apache.beam.runners.dataflow.util; -import org.apache.beam.sdk.util.AttemptBoundedExponentialBackOff; -import org.apache.beam.sdk.util.IOChannelUtils; -import org.apache.beam.sdk.util.MimeTypes; -import org.apache.beam.sdk.util.ZipFiles; - +import com.fasterxml.jackson.core.Base64Variants; import com.google.api.client.util.BackOffUtils; import com.google.api.client.util.Sleeper; import com.google.api.services.dataflow.model.DataflowPackage; @@ -31,12 +27,6 @@ import com.google.common.hash.Hashing; import com.google.common.io.CountingOutputStream; import com.google.common.io.Files; - -import com.fasterxml.jackson.core.Base64Variants; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - import java.io.File; import java.io.FileNotFoundException; import java.io.IOException; @@ -47,6 +37,12 @@ import java.util.Collection; import java.util.List; import java.util.Objects; +import org.apache.beam.sdk.util.AttemptBoundedExponentialBackOff; +import org.apache.beam.sdk.util.IOChannelUtils; +import org.apache.beam.sdk.util.MimeTypes; +import org.apache.beam.sdk.util.ZipFiles; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** Helper routines for packages. */ public class PackageUtil { diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/RandomAccessData.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/RandomAccessData.java index 9e102422b7dbd..683e16b3e9a00 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/RandomAccessData.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/RandomAccessData.java @@ -20,26 +20,22 @@ import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.base.Preconditions.checkNotNull; -import org.apache.beam.sdk.coders.AtomicCoder; -import org.apache.beam.sdk.coders.ByteArrayCoder; -import org.apache.beam.sdk.coders.Coder; -import org.apache.beam.sdk.coders.CoderException; -import org.apache.beam.sdk.util.VarInt; - +import com.fasterxml.jackson.annotation.JsonCreator; import com.google.common.base.MoreObjects; import com.google.common.io.ByteStreams; import com.google.common.primitives.UnsignedBytes; - -import com.fasterxml.jackson.annotation.JsonCreator; - import java.io.ByteArrayInputStream; import java.io.IOException; import java.io.InputStream; import java.io.OutputStream; import java.util.Arrays; import java.util.Comparator; - import javax.annotation.concurrent.NotThreadSafe; +import org.apache.beam.sdk.coders.AtomicCoder; +import org.apache.beam.sdk.coders.ByteArrayCoder; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.coders.CoderException; +import org.apache.beam.sdk.util.VarInt; /** * An elastic-sized byte array which allows you to manipulate it as a stream, or access diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/Stager.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/Stager.java index 2172eb4711cf1..232e03260fe28 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/Stager.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/Stager.java @@ -18,7 +18,6 @@ package org.apache.beam.runners.dataflow.util; import com.google.api.services.dataflow.model.DataflowPackage; - import java.util.List; /** diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/TimeUtil.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/TimeUtil.java index 434c83f7da226..bff379fc1cc70 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/TimeUtil.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/TimeUtil.java @@ -17,6 +17,9 @@ */ package org.apache.beam.runners.dataflow.util; +import java.util.regex.Matcher; +import java.util.regex.Pattern; +import javax.annotation.Nullable; import org.joda.time.DateTime; import org.joda.time.Duration; import org.joda.time.Instant; @@ -24,11 +27,6 @@ import org.joda.time.ReadableInstant; import org.joda.time.chrono.ISOChronology; -import java.util.regex.Matcher; -import java.util.regex.Pattern; - -import javax.annotation.Nullable; - /** * A helper class for converting between Dataflow API and SDK time * representations. diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/BlockingDataflowRunnerTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/BlockingDataflowRunnerTest.java index 7bdac3de46afa..4572a64aeaa9f 100644 --- a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/BlockingDataflowRunnerTest.java +++ b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/BlockingDataflowRunnerTest.java @@ -34,7 +34,6 @@ import org.apache.beam.sdk.testing.TestPipeline; import org.apache.beam.sdk.util.NoopPathValidator; import org.apache.beam.sdk.util.TestCredential; - import org.hamcrest.Description; import org.hamcrest.Factory; import org.hamcrest.Matcher; 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 e6277d9c98c3e..22b5400b0fdf4 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 @@ -32,6 +32,21 @@ import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; +import com.google.api.services.dataflow.Dataflow; +import com.google.api.services.dataflow.Dataflow.Projects.Jobs.Get; +import com.google.api.services.dataflow.Dataflow.Projects.Jobs.GetMetrics; +import com.google.api.services.dataflow.Dataflow.Projects.Jobs.Messages; +import com.google.api.services.dataflow.model.Job; +import com.google.api.services.dataflow.model.JobMetrics; +import com.google.api.services.dataflow.model.MetricStructuredName; +import com.google.api.services.dataflow.model.MetricUpdate; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.ImmutableSetMultimap; +import java.io.IOException; +import java.math.BigDecimal; +import java.net.SocketTimeoutException; +import java.util.concurrent.TimeUnit; import org.apache.beam.runners.dataflow.internal.DataflowAggregatorTransforms; import org.apache.beam.runners.dataflow.testing.TestDataflowPipelineOptions; import org.apache.beam.runners.dataflow.util.MonitoringUtil; @@ -48,19 +63,6 @@ import org.apache.beam.sdk.util.AttemptBoundedExponentialBackOff; import org.apache.beam.sdk.values.PInput; import org.apache.beam.sdk.values.POutput; - -import com.google.api.services.dataflow.Dataflow; -import com.google.api.services.dataflow.Dataflow.Projects.Jobs.Get; -import com.google.api.services.dataflow.Dataflow.Projects.Jobs.GetMetrics; -import com.google.api.services.dataflow.Dataflow.Projects.Jobs.Messages; -import com.google.api.services.dataflow.model.Job; -import com.google.api.services.dataflow.model.JobMetrics; -import com.google.api.services.dataflow.model.MetricStructuredName; -import com.google.api.services.dataflow.model.MetricUpdate; -import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableMap; -import com.google.common.collect.ImmutableSetMultimap; - import org.joda.time.Duration; import org.junit.Before; import org.junit.Rule; @@ -71,11 +73,6 @@ import org.mockito.Mock; import org.mockito.MockitoAnnotations; -import java.io.IOException; -import java.math.BigDecimal; -import java.net.SocketTimeoutException; -import java.util.concurrent.TimeUnit; - /** * Tests for DataflowPipelineJob. */ diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowPipelineRegistrarTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowPipelineRegistrarTest.java index 388a85ac50d70..f08475735e535 100644 --- a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowPipelineRegistrarTest.java +++ b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowPipelineRegistrarTest.java @@ -20,20 +20,17 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.fail; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.Lists; +import java.util.ServiceLoader; import org.apache.beam.runners.dataflow.options.BlockingDataflowPipelineOptions; import org.apache.beam.runners.dataflow.options.DataflowPipelineOptions; import org.apache.beam.sdk.options.PipelineOptionsRegistrar; import org.apache.beam.sdk.runners.PipelineRunnerRegistrar; - -import com.google.common.collect.ImmutableList; -import com.google.common.collect.Lists; - import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.util.ServiceLoader; - /** Tests for {@link DataflowPipelineRegistrar}. */ @RunWith(JUnit4.class) public class DataflowPipelineRegistrarTest { 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 2a01c0397ec1f..2b7013dc9936f 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 @@ -20,7 +20,6 @@ import static org.apache.beam.sdk.util.Structs.addObject; import static org.apache.beam.sdk.util.Structs.getDictionary; 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.hasKey; @@ -36,6 +35,23 @@ import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; +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.Step; +import com.google.api.services.dataflow.model.WorkerPool; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.ImmutableSet; +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.HashMap; +import java.util.LinkedList; +import java.util.List; +import java.util.Map; import org.apache.beam.runners.dataflow.DataflowPipelineTranslator.TranslationContext; import org.apache.beam.runners.dataflow.options.DataflowPipelineOptions; import org.apache.beam.runners.dataflow.options.DataflowPipelineWorkerPoolOptions; @@ -65,17 +81,6 @@ import org.apache.beam.sdk.values.PCollectionTuple; import org.apache.beam.sdk.values.PDone; import org.apache.beam.sdk.values.TupleTag; - -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.Step; -import com.google.api.services.dataflow.model.WorkerPool; -import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableMap; -import com.google.common.collect.ImmutableSet; -import com.google.common.collect.Iterables; - import org.junit.Assert; import org.junit.Rule; import org.junit.Test; @@ -87,15 +92,6 @@ import org.mockito.invocation.InvocationOnMock; import org.mockito.stubbing.Answer; -import java.io.IOException; -import java.io.Serializable; -import java.util.Collection; -import java.util.Collections; -import java.util.HashMap; -import java.util.LinkedList; -import java.util.List; -import java.util.Map; - /** * Tests for DataflowPipelineTranslator. */ 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 92a6bcb4c7298..208e84c966789 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,7 +18,6 @@ package org.apache.beam.runners.dataflow; import static org.apache.beam.sdk.util.WindowedValue.valueInGlobalWindow; - import static org.hamcrest.Matchers.containsInAnyOrder; import static org.hamcrest.Matchers.containsString; import static org.hamcrest.Matchers.equalTo; @@ -38,6 +37,28 @@ import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; +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; +import com.google.common.base.Throwables; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import java.io.File; +import java.io.IOException; +import java.net.URL; +import java.net.URLClassLoader; +import java.nio.channels.FileChannel; +import java.nio.channels.SeekableByteChannel; +import java.nio.file.Files; +import java.nio.file.StandardOpenOption; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.LinkedList; +import java.util.List; +import java.util.Map; +import java.util.regex.Pattern; import org.apache.beam.runners.dataflow.DataflowRunner.BatchViewAsList; import org.apache.beam.runners.dataflow.DataflowRunner.BatchViewAsMap; import org.apache.beam.runners.dataflow.DataflowRunner.BatchViewAsMultimap; @@ -84,15 +105,6 @@ import org.apache.beam.sdk.values.PInput; import org.apache.beam.sdk.values.TimestampedValue; import org.apache.beam.sdk.values.TupleTag; - -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; -import com.google.common.base.Throwables; -import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableMap; - import org.hamcrest.Description; import org.hamcrest.Matchers; import org.hamcrest.TypeSafeMatcher; @@ -109,22 +121,6 @@ import org.mockito.invocation.InvocationOnMock; import org.mockito.stubbing.Answer; -import java.io.File; -import java.io.IOException; -import java.net.URL; -import java.net.URLClassLoader; -import java.nio.channels.FileChannel; -import java.nio.channels.SeekableByteChannel; -import java.nio.file.Files; -import java.nio.file.StandardOpenOption; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collections; -import java.util.LinkedList; -import java.util.List; -import java.util.Map; -import java.util.regex.Pattern; - /** * Tests for the {@link DataflowRunner}. */ 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 8ebc4bd1f4085..2090877f41ffc 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 @@ -17,14 +17,13 @@ */ package org.apache.beam.runners.dataflow; +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.transforms.PTransform; import org.apache.beam.sdk.values.PValue; -import java.util.ArrayList; -import java.util.List; - /** * Provides a simple {@link org.apache.beam.sdk.Pipeline.PipelineVisitor} * that records the transformation tree. diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/options/DataflowPipelineDebugOptionsTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/options/DataflowPipelineDebugOptionsTest.java index 71b6b5721458c..d376acf14b914 100644 --- a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/options/DataflowPipelineDebugOptionsTest.java +++ b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/options/DataflowPipelineDebugOptionsTest.java @@ -22,7 +22,6 @@ import static org.junit.Assert.assertThat; import org.apache.beam.sdk.options.PipelineOptionsFactory; - import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; 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 c0422ee83de6f..61d39923ae5dd 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 @@ -26,7 +26,6 @@ import org.apache.beam.sdk.testing.RestoreSystemProperties; import org.apache.beam.sdk.util.IOChannelUtils; import org.apache.beam.sdk.util.NoopPathValidator; - import org.junit.Rule; import org.junit.Test; import org.junit.rules.ExpectedException; 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 58f851468460b..87c74a4e37b93 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 @@ -20,10 +20,8 @@ import static org.junit.Assert.assertThat; import static org.junit.Assert.assertTrue; -import org.apache.beam.sdk.options.PipelineOptionsFactory; - import com.fasterxml.jackson.databind.ObjectMapper; - +import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.hamcrest.Matchers; import org.junit.Test; import org.junit.runner.RunWith; diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/options/DataflowWorkerLoggingOptionsTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/options/DataflowWorkerLoggingOptionsTest.java index 5b12ad5f07ad4..b463dcb866865 100644 --- a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/options/DataflowWorkerLoggingOptionsTest.java +++ b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/options/DataflowWorkerLoggingOptionsTest.java @@ -18,15 +18,11 @@ package org.apache.beam.runners.dataflow.options; import static org.apache.beam.runners.dataflow.options.DataflowWorkerLoggingOptions.Level.WARN; - import static org.junit.Assert.assertEquals; -import org.apache.beam.runners.dataflow.options.DataflowWorkerLoggingOptions.WorkerLogLevelOverrides; - -import com.google.common.collect.ImmutableMap; - import com.fasterxml.jackson.databind.ObjectMapper; - +import com.google.common.collect.ImmutableMap; +import org.apache.beam.runners.dataflow.options.DataflowWorkerLoggingOptions.WorkerLogLevelOverrides; import org.junit.Rule; import org.junit.Test; import org.junit.rules.ExpectedException; 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 b4bbd39c371f3..24d5b2388539c 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 @@ -31,6 +31,22 @@ import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; +import com.google.api.client.http.LowLevelHttpResponse; +import com.google.api.client.json.Json; +import com.google.api.client.testing.http.MockHttpTransport; +import com.google.api.client.testing.http.MockLowLevelHttpRequest; +import com.google.api.client.testing.http.MockLowLevelHttpResponse; +import com.google.api.services.dataflow.Dataflow; +import com.google.api.services.dataflow.model.JobMessage; +import com.google.api.services.dataflow.model.JobMetrics; +import com.google.api.services.dataflow.model.MetricStructuredName; +import com.google.api.services.dataflow.model.MetricUpdate; +import com.google.common.base.Optional; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Lists; +import java.io.IOException; +import java.math.BigDecimal; +import java.util.Arrays; import org.apache.beam.runners.dataflow.DataflowPipelineJob; import org.apache.beam.runners.dataflow.DataflowRunner; import org.apache.beam.runners.dataflow.util.MonitoringUtil; @@ -50,21 +66,6 @@ import org.apache.beam.sdk.util.TestCredential; import org.apache.beam.sdk.util.Transport; import org.apache.beam.sdk.values.PCollection; - -import com.google.api.client.http.LowLevelHttpResponse; -import com.google.api.client.json.Json; -import com.google.api.client.testing.http.MockHttpTransport; -import com.google.api.client.testing.http.MockLowLevelHttpRequest; -import com.google.api.client.testing.http.MockLowLevelHttpResponse; -import com.google.api.services.dataflow.Dataflow; -import com.google.api.services.dataflow.model.JobMessage; -import com.google.api.services.dataflow.model.JobMetrics; -import com.google.api.services.dataflow.model.MetricStructuredName; -import com.google.api.services.dataflow.model.MetricUpdate; -import com.google.common.base.Optional; -import com.google.common.collect.ImmutableMap; -import com.google.common.collect.Lists; - import org.hamcrest.BaseMatcher; import org.hamcrest.Description; import org.joda.time.Duration; @@ -81,10 +82,6 @@ import org.mockito.invocation.InvocationOnMock; import org.mockito.stubbing.Answer; -import java.io.IOException; -import java.math.BigDecimal; -import java.util.Arrays; - /** Tests for {@link TestDataflowRunner}. */ @RunWith(JUnit4.class) public class TestDataflowRunnerTest { 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 b219ea2e3046a..6d5a63ab44d73 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,8 @@ */ package org.apache.beam.runners.dataflow.transforms; +import java.util.Arrays; +import java.util.List; import org.apache.beam.runners.dataflow.DataflowRunner; import org.apache.beam.runners.dataflow.options.DataflowPipelineOptions; import org.apache.beam.sdk.Pipeline; @@ -35,7 +37,6 @@ import org.apache.beam.sdk.values.PBegin; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.TypeDescriptor; - import org.joda.time.Duration; import org.junit.Rule; import org.junit.Test; @@ -43,9 +44,6 @@ import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.util.Arrays; -import java.util.List; - /** Tests for {@link GroupByKey} for the {@link DataflowRunner}. */ @RunWith(JUnit4.class) public class DataflowGroupByKeyTest { 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 95cbaae8ca74e..2eddcf78c22f0 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 @@ -34,7 +34,6 @@ import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TypeDescriptor; - import org.hamcrest.Matchers; import org.joda.time.Duration; import org.junit.Rule; 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 da075150f7fc4..98d8ab60e8151 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 @@ -22,17 +22,19 @@ 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.options.DataflowPipelineOptions; import org.apache.beam.runners.dataflow.util.MonitoringUtil.LoggingHandler; import org.apache.beam.sdk.PipelineResult.State; import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.testing.ExpectedLogs; import org.apache.beam.sdk.util.TestCredential; - -import com.google.api.services.dataflow.Dataflow; -import com.google.api.services.dataflow.model.JobMessage; -import com.google.api.services.dataflow.model.ListJobMessagesResponse; - import org.joda.time.DateTime; import org.joda.time.Instant; import org.joda.time.chrono.ISOChronology; @@ -42,11 +44,6 @@ import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.io.IOException; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.List; - /** * Tests for MonitoringUtil. */ diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/util/PackageUtilTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/util/PackageUtilTest.java index 41ad05dc044bc..02aceef34b5d1 100644 --- a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/util/PackageUtilTest.java +++ b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/util/PackageUtilTest.java @@ -32,15 +32,6 @@ import static org.mockito.Mockito.verifyNoMoreInteractions; import static org.mockito.Mockito.when; -import org.apache.beam.runners.dataflow.util.PackageUtil.PackageAttributes; -import org.apache.beam.sdk.options.GcsOptions; -import org.apache.beam.sdk.options.PipelineOptionsFactory; -import org.apache.beam.sdk.testing.ExpectedLogs; -import org.apache.beam.sdk.testing.FastNanoClockAndSleeper; -import org.apache.beam.sdk.util.GcsUtil; -import org.apache.beam.sdk.util.IOChannelUtils; -import org.apache.beam.sdk.util.gcsfs.GcsPath; - import com.google.api.client.googleapis.json.GoogleJsonError.ErrorInfo; import com.google.api.client.googleapis.json.GoogleJsonResponseException; import com.google.api.client.http.HttpRequest; @@ -62,19 +53,6 @@ import com.google.common.collect.Lists; import com.google.common.io.Files; import com.google.common.io.LineReader; - -import org.hamcrest.BaseMatcher; -import org.hamcrest.Description; -import org.hamcrest.Matchers; -import org.junit.Before; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; -import org.junit.runner.RunWith; -import org.junit.runners.JUnit4; -import org.mockito.Mock; -import org.mockito.MockitoAnnotations; - import java.io.File; import java.io.FileNotFoundException; import java.io.IOException; @@ -88,6 +66,25 @@ import java.util.regex.Pattern; import java.util.zip.ZipEntry; import java.util.zip.ZipInputStream; +import org.apache.beam.runners.dataflow.util.PackageUtil.PackageAttributes; +import org.apache.beam.sdk.options.GcsOptions; +import org.apache.beam.sdk.options.PipelineOptionsFactory; +import org.apache.beam.sdk.testing.ExpectedLogs; +import org.apache.beam.sdk.testing.FastNanoClockAndSleeper; +import org.apache.beam.sdk.util.GcsUtil; +import org.apache.beam.sdk.util.IOChannelUtils; +import org.apache.beam.sdk.util.gcsfs.GcsPath; +import org.hamcrest.BaseMatcher; +import org.hamcrest.Description; +import org.hamcrest.Matchers; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; +import org.mockito.Mock; +import org.mockito.MockitoAnnotations; /** Tests for PackageUtil. */ @RunWith(JUnit4.class) diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/util/RandomAccessDataTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/util/RandomAccessDataTest.java index c5c77cfaab162..042e14578b780 100644 --- a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/util/RandomAccessDataTest.java +++ b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/util/RandomAccessDataTest.java @@ -23,24 +23,21 @@ import static org.junit.Assert.assertSame; import static org.junit.Assert.assertTrue; +import com.google.common.primitives.UnsignedBytes; +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.InputStream; +import java.util.Arrays; import org.apache.beam.runners.dataflow.util.RandomAccessData.RandomAccessDataCoder; import org.apache.beam.sdk.coders.Coder.Context; import org.apache.beam.sdk.coders.CoderException; import org.apache.beam.sdk.testing.CoderProperties; - -import com.google.common.primitives.UnsignedBytes; - import org.junit.Rule; import org.junit.Test; import org.junit.rules.ExpectedException; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.io.ByteArrayInputStream; -import java.io.ByteArrayOutputStream; -import java.io.InputStream; -import java.util.Arrays; - /** * Tests for {@link RandomAccessData}. */ diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/util/TimeUtilTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/util/TimeUtilTest.java index c22c223e27475..e0785d424fe70 100644 --- a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/util/TimeUtilTest.java +++ b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/util/TimeUtilTest.java @@ -21,7 +21,6 @@ import static org.apache.beam.runners.dataflow.util.TimeUtil.fromCloudTime; import static org.apache.beam.runners.dataflow.util.TimeUtil.toCloudDuration; import static org.apache.beam.runners.dataflow.util.TimeUtil.toCloudTime; - import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNull; 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 be50f70277d61..2ce1ff697e42a 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 @@ -40,7 +40,6 @@ import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PInput; import org.apache.beam.sdk.values.POutput; - import org.apache.spark.SparkException; import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.streaming.Duration; diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkRunnerRegistrar.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkRunnerRegistrar.java index 7a317539e4721..bedfda419f41c 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkRunnerRegistrar.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkRunnerRegistrar.java @@ -18,14 +18,13 @@ package org.apache.beam.runners.spark; +import com.google.auto.service.AutoService; +import com.google.common.collect.ImmutableList; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.options.PipelineOptionsRegistrar; import org.apache.beam.sdk.runners.PipelineRunner; import org.apache.beam.sdk.runners.PipelineRunnerRegistrar; -import com.google.auto.service.AutoService; -import com.google.common.collect.ImmutableList; - /** * Contains the {@link PipelineRunnerRegistrar} and {@link PipelineOptionsRegistrar} for the * {@link SparkRunner}. diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/aggregators/NamedAggregators.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/aggregators/NamedAggregators.java index 6ab6dc9a303f6..c15e276c43dbb 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/aggregators/NamedAggregators.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/aggregators/NamedAggregators.java @@ -18,19 +18,17 @@ package org.apache.beam.runners.spark.aggregators; -import org.apache.beam.runners.spark.translation.SparkRuntimeContext; -import org.apache.beam.sdk.coders.CannotProvideCoderException; -import org.apache.beam.sdk.coders.Coder; -import org.apache.beam.sdk.transforms.Combine; - import com.google.common.collect.ImmutableList; - import java.io.IOException; import java.io.ObjectInputStream; import java.io.ObjectOutputStream; import java.io.Serializable; import java.util.Map; import java.util.TreeMap; +import org.apache.beam.runners.spark.translation.SparkRuntimeContext; +import org.apache.beam.sdk.coders.CannotProvideCoderException; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.transforms.Combine; /** * This class wraps a map of named aggregators. Spark expects that all accumulators be declared diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/coders/CoderHelpers.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/coders/CoderHelpers.java index 07587fcac76c2..0df66c237a45d 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/coders/CoderHelpers.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/coders/CoderHelpers.java @@ -18,20 +18,16 @@ package org.apache.beam.runners.spark.coders; -import org.apache.beam.runners.spark.util.ByteArray; -import org.apache.beam.sdk.coders.Coder; - import com.google.common.collect.Iterables; - -import org.apache.spark.api.java.function.Function; -import org.apache.spark.api.java.function.PairFunction; - import java.io.ByteArrayInputStream; import java.io.ByteArrayOutputStream; import java.io.IOException; import java.util.LinkedList; import java.util.List; - +import org.apache.beam.runners.spark.util.ByteArray; +import org.apache.beam.sdk.coders.Coder; +import org.apache.spark.api.java.function.Function; +import org.apache.spark.api.java.function.PairFunction; import scala.Tuple2; /** diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/coders/NullWritableCoder.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/coders/NullWritableCoder.java index 7cff3251cad7b..ebbab1a235b94 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/coders/NullWritableCoder.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/coders/NullWritableCoder.java @@ -18,13 +18,11 @@ package org.apache.beam.runners.spark.coders; -import org.apache.beam.sdk.coders.Coder; - import com.fasterxml.jackson.annotation.JsonCreator; -import org.apache.hadoop.io.NullWritable; - import java.io.InputStream; import java.io.OutputStream; +import org.apache.beam.sdk.coders.Coder; +import org.apache.hadoop.io.NullWritable; /** * Simple writable coder for Null. diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/coders/WritableCoder.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/coders/WritableCoder.java index 4719e46b23d1f..b40e0229f282f 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/coders/WritableCoder.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/coders/WritableCoder.java @@ -18,16 +18,8 @@ package org.apache.beam.runners.spark.coders; -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.util.CloudObject; - import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; -import org.apache.hadoop.io.NullWritable; -import org.apache.hadoop.io.Writable; - import java.io.DataInputStream; import java.io.DataOutputStream; import java.io.IOException; @@ -35,6 +27,12 @@ import java.io.OutputStream; import java.lang.reflect.InvocationTargetException; import java.util.List; +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.util.CloudObject; +import org.apache.hadoop.io.NullWritable; +import org.apache.hadoop.io.Writable; /** * A {@code WritableCoder} is a {@link Coder} for a Java class that implements {@link Writable}. diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/io/KafkaIO.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/io/KafkaIO.java index 13171f3d619eb..f57c114e69fd1 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/io/KafkaIO.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/io/KafkaIO.java @@ -19,17 +19,15 @@ import static com.google.common.base.Preconditions.checkNotNull; +import java.util.Map; +import java.util.Set; +import kafka.serializer.Decoder; import org.apache.beam.sdk.transforms.PTransform; 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.PInput; -import java.util.Map; -import java.util.Set; - -import kafka.serializer.Decoder; - /** * Read stream from Kafka. */ 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 1177a578dfe73..7b106105ee190 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 @@ -20,6 +20,8 @@ import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.base.Preconditions.checkNotNull; +import java.util.HashMap; +import java.util.Map; import org.apache.beam.sdk.io.ShardNameTemplate; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.util.WindowingStrategy; @@ -27,13 +29,9 @@ import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PDone; import org.apache.beam.sdk.values.PInput; - import org.apache.hadoop.mapreduce.lib.input.FileInputFormat; import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat; -import java.util.HashMap; -import java.util.Map; - /** * Spark native HadoopIO. */ diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/io/hadoop/ShardNameBuilder.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/io/hadoop/ShardNameBuilder.java index 6b36427bc52cd..11b4b53528794 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/io/hadoop/ShardNameBuilder.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/io/hadoop/ShardNameBuilder.java @@ -18,10 +18,9 @@ package org.apache.beam.runners.spark.io.hadoop; -import org.apache.hadoop.fs.Path; - import java.util.regex.Matcher; import java.util.regex.Pattern; +import org.apache.hadoop.fs.Path; /** * Shard name builder. diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/io/hadoop/ShardNameTemplateHelper.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/io/hadoop/ShardNameTemplateHelper.java index d06b0162b1e56..7f8e2978c439d 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/io/hadoop/ShardNameTemplateHelper.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/io/hadoop/ShardNameTemplateHelper.java @@ -20,6 +20,7 @@ import static org.apache.beam.runners.spark.io.hadoop.ShardNameBuilder.replaceShardNumber; +import java.io.IOException; import org.apache.hadoop.fs.Path; import org.apache.hadoop.mapreduce.TaskAttemptContext; import org.apache.hadoop.mapreduce.TaskID; @@ -28,8 +29,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.io.IOException; - /** * Shard name template helper. */ diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/io/hadoop/TemplatedAvroKeyOutputFormat.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/io/hadoop/TemplatedAvroKeyOutputFormat.java index f747e7be3b6e3..62a610ba57f22 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/io/hadoop/TemplatedAvroKeyOutputFormat.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/io/hadoop/TemplatedAvroKeyOutputFormat.java @@ -18,14 +18,13 @@ package org.apache.beam.runners.spark.io.hadoop; +import java.io.IOException; +import java.io.OutputStream; import org.apache.avro.mapreduce.AvroKeyOutputFormat; import org.apache.hadoop.fs.Path; import org.apache.hadoop.mapreduce.JobContext; import org.apache.hadoop.mapreduce.TaskAttemptContext; -import java.io.IOException; -import java.io.OutputStream; - /** * Templated Avro key output format. */ diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/io/hadoop/TemplatedSequenceFileOutputFormat.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/io/hadoop/TemplatedSequenceFileOutputFormat.java index bd2ee4d20d725..ab1263b031f6a 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/io/hadoop/TemplatedSequenceFileOutputFormat.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/io/hadoop/TemplatedSequenceFileOutputFormat.java @@ -18,13 +18,12 @@ package org.apache.beam.runners.spark.io.hadoop; +import java.io.IOException; import org.apache.hadoop.fs.Path; import org.apache.hadoop.mapreduce.JobContext; import org.apache.hadoop.mapreduce.TaskAttemptContext; import org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat; -import java.io.IOException; - /** * Templated sequence file output format. */ diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/io/hadoop/TemplatedTextOutputFormat.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/io/hadoop/TemplatedTextOutputFormat.java index 8725a95c5e4e8..5a6e9a9386533 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/io/hadoop/TemplatedTextOutputFormat.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/io/hadoop/TemplatedTextOutputFormat.java @@ -18,13 +18,12 @@ package org.apache.beam.runners.spark.io.hadoop; +import java.io.IOException; import org.apache.hadoop.fs.Path; import org.apache.hadoop.mapreduce.JobContext; import org.apache.hadoop.mapreduce.TaskAttemptContext; import org.apache.hadoop.mapreduce.lib.output.TextOutputFormat; -import java.io.IOException; - /** * Templates text output format. */ diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/DoFnFunction.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/DoFnFunction.java index c08d1850d4127..800d614e5e964 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/DoFnFunction.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/DoFnFunction.java @@ -18,20 +18,18 @@ package org.apache.beam.runners.spark.translation; +import java.util.Iterator; +import java.util.LinkedList; +import java.util.List; +import java.util.Map; import org.apache.beam.runners.spark.util.BroadcastHelper; import org.apache.beam.sdk.transforms.OldDoFn; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.TupleTag; - import org.apache.spark.api.java.function.FlatMapFunction; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.util.Iterator; -import java.util.LinkedList; -import java.util.List; -import java.util.Map; - /** * Dataflow's Do functions correspond to Spark's FlatMap functions. * 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 4ccac0e1a0b23..724f54fd056ed 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 @@ -20,6 +20,14 @@ import static com.google.common.base.Preconditions.checkArgument; +import com.google.common.base.Function; +import com.google.common.collect.Iterables; +import java.io.IOException; +import java.util.LinkedHashMap; +import java.util.LinkedHashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; import org.apache.beam.runners.spark.EvaluationResult; import org.apache.beam.runners.spark.coders.CoderHelpers; import org.apache.beam.sdk.AggregatorRetrievalException; @@ -38,21 +46,10 @@ import org.apache.beam.sdk.values.PInput; import org.apache.beam.sdk.values.POutput; import org.apache.beam.sdk.values.PValue; - -import com.google.common.base.Function; -import com.google.common.collect.Iterables; - import org.apache.spark.api.java.JavaRDDLike; import org.apache.spark.api.java.JavaSparkContext; import org.joda.time.Duration; -import java.io.IOException; -import java.util.LinkedHashMap; -import java.util.LinkedHashSet; -import java.util.List; -import java.util.Map; -import java.util.Set; - /** * Evaluation context allows us to define how pipeline instructions. diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/MultiDoFnFunction.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/MultiDoFnFunction.java index abf0e8379a2ee..4c44ffd04a9d1 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/MultiDoFnFunction.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/MultiDoFnFunction.java @@ -18,22 +18,18 @@ package org.apache.beam.runners.spark.translation; -import org.apache.beam.runners.spark.util.BroadcastHelper; -import org.apache.beam.sdk.transforms.OldDoFn; -import org.apache.beam.sdk.util.WindowedValue; -import org.apache.beam.sdk.values.TupleTag; - import com.google.common.base.Function; import com.google.common.collect.Iterators; import com.google.common.collect.LinkedListMultimap; import com.google.common.collect.Multimap; - -import org.apache.spark.api.java.function.PairFlatMapFunction; -import org.joda.time.Instant; - import java.util.Iterator; import java.util.Map; - +import org.apache.beam.runners.spark.util.BroadcastHelper; +import org.apache.beam.sdk.transforms.OldDoFn; +import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.TupleTag; +import org.apache.spark.api.java.function.PairFlatMapFunction; +import org.joda.time.Instant; import scala.Tuple2; /** 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 1cdbd92010d8a..566a2729d055e 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 @@ -18,6 +18,12 @@ package org.apache.beam.runners.spark.translation; +import com.google.common.collect.AbstractIterator; +import com.google.common.collect.Iterables; +import java.io.IOException; +import java.util.Collection; +import java.util.Iterator; +import java.util.Map; import org.apache.beam.runners.spark.util.BroadcastHelper; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.options.PipelineOptions; @@ -33,19 +39,10 @@ import org.apache.beam.sdk.util.state.StateInternals; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TupleTag; - -import com.google.common.collect.AbstractIterator; -import com.google.common.collect.Iterables; - import org.joda.time.Instant; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.io.IOException; -import java.util.Collection; -import java.util.Iterator; -import java.util.Map; - /** * Spark runner process context. */ 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 c2edd023c7e16..4bc0c00973f8b 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 @@ -18,6 +18,14 @@ package org.apache.beam.runners.spark.translation; +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.aggregators.AggAccumParam; import org.apache.beam.runners.spark.aggregators.NamedAggregators; import org.apache.beam.sdk.AggregatorValues; @@ -32,20 +40,9 @@ import org.apache.beam.sdk.transforms.Min; import org.apache.beam.sdk.transforms.Sum; import org.apache.beam.sdk.values.TypeDescriptor; - -import com.google.common.collect.ImmutableList; - -import com.fasterxml.jackson.core.JsonProcessingException; -import com.fasterxml.jackson.databind.ObjectMapper; import org.apache.spark.Accumulator; import org.apache.spark.api.java.JavaSparkContext; -import java.io.IOException; -import java.io.Serializable; -import java.util.Collection; -import java.util.HashMap; -import java.util.Map; - /** * The SparkRuntimeContext allows us to define useful features on the client side before our diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/TransformEvaluator.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/TransformEvaluator.java index c5c7128feb52a..fbfa84d76b505 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/TransformEvaluator.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/TransformEvaluator.java @@ -18,9 +18,8 @@ package org.apache.beam.runners.spark.translation; -import org.apache.beam.sdk.transforms.PTransform; - import java.io.Serializable; +import org.apache.beam.sdk.transforms.PTransform; /** * Describe a {@link PTransform} evaluator. 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 c51a50037415c..08e3fda7a36b5 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 @@ -24,6 +24,19 @@ import static org.apache.beam.runners.spark.io.hadoop.ShardNameBuilder.getOutputFileTemplate; import static org.apache.beam.runners.spark.io.hadoop.ShardNameBuilder.replaceShardCount; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; +import java.io.IOException; +import java.io.Serializable; +import java.lang.reflect.Field; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import org.apache.avro.mapred.AvroKey; +import org.apache.avro.mapreduce.AvroJob; +import org.apache.avro.mapreduce.AvroKeyInputFormat; import org.apache.beam.runners.spark.coders.CoderHelpers; import org.apache.beam.runners.spark.io.hadoop.HadoopIO; import org.apache.beam.runners.spark.io.hadoop.ShardNameTemplateHelper; @@ -65,14 +78,6 @@ import org.apache.beam.sdk.values.PCollectionTuple; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TupleTag; - -import com.google.common.collect.ImmutableMap; -import com.google.common.collect.Lists; -import com.google.common.collect.Maps; - -import org.apache.avro.mapred.AvroKey; -import org.apache.avro.mapreduce.AvroJob; -import org.apache.avro.mapreduce.AvroKeyInputFormat; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.io.NullWritable; import org.apache.hadoop.io.Text; @@ -86,15 +91,6 @@ import org.apache.spark.api.java.function.Function2; import org.apache.spark.api.java.function.PairFlatMapFunction; import org.apache.spark.api.java.function.PairFunction; - -import java.io.IOException; -import java.io.Serializable; -import java.lang.reflect.Field; -import java.util.Arrays; -import java.util.Collections; -import java.util.List; -import java.util.Map; - import scala.Tuple2; /** diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/streaming/StreamingEvaluationContext.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/streaming/StreamingEvaluationContext.java index afcca932e9fab..2e4da4467723b 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/streaming/StreamingEvaluationContext.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/streaming/StreamingEvaluationContext.java @@ -18,6 +18,12 @@ package org.apache.beam.runners.spark.translation.streaming; +import java.util.LinkedHashMap; +import java.util.LinkedHashSet; +import java.util.Map; +import java.util.Queue; +import java.util.Set; +import java.util.concurrent.LinkedBlockingQueue; import org.apache.beam.runners.spark.translation.EvaluationContext; import org.apache.beam.runners.spark.translation.SparkRuntimeContext; import org.apache.beam.sdk.Pipeline; @@ -28,7 +34,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.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaRDDLike; import org.apache.spark.api.java.JavaSparkContext; @@ -37,13 +42,6 @@ import org.apache.spark.streaming.api.java.JavaDStreamLike; import org.apache.spark.streaming.api.java.JavaStreamingContext; -import java.util.LinkedHashMap; -import java.util.LinkedHashSet; -import java.util.Map; -import java.util.Queue; -import java.util.Set; -import java.util.concurrent.LinkedBlockingQueue; - /** * Streaming evaluation context helps to handle streaming. 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 2ce2c298bc4ef..43160f5500669 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 @@ -17,6 +17,17 @@ */ package org.apache.beam.runners.spark.translation.streaming; +import com.google.api.client.util.Lists; +import com.google.api.client.util.Maps; +import com.google.api.client.util.Sets; +import com.google.common.reflect.TypeToken; +import java.lang.reflect.ParameterizedType; +import java.lang.reflect.Type; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.Set; +import kafka.serializer.Decoder; import org.apache.beam.runners.spark.io.ConsoleIO; import org.apache.beam.runners.spark.io.CreateStream; import org.apache.beam.runners.spark.io.KafkaIO; @@ -43,12 +54,6 @@ import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollectionList; import org.apache.beam.sdk.values.PDone; - -import com.google.api.client.util.Lists; -import com.google.api.client.util.Maps; -import com.google.api.client.util.Sets; -import com.google.common.reflect.TypeToken; - import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.function.Function; import org.apache.spark.api.java.function.VoidFunction; @@ -59,15 +64,6 @@ import org.apache.spark.streaming.api.java.JavaPairInputDStream; import org.apache.spark.streaming.api.java.JavaStreamingContext; import org.apache.spark.streaming.kafka.KafkaUtils; - -import java.lang.reflect.ParameterizedType; -import java.lang.reflect.Type; -import java.util.Collections; -import java.util.List; -import java.util.Map; -import java.util.Set; - -import kafka.serializer.Decoder; import scala.Tuple2; diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/util/BroadcastHelper.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/util/BroadcastHelper.java index d3fa05a751421..29c2dd995b6e6 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/util/BroadcastHelper.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/util/BroadcastHelper.java @@ -18,18 +18,16 @@ package org.apache.beam.runners.spark.util; +import java.io.ByteArrayInputStream; +import java.io.IOException; +import java.io.Serializable; import org.apache.beam.runners.spark.coders.CoderHelpers; import org.apache.beam.sdk.coders.Coder; - import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.broadcast.Broadcast; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.io.ByteArrayInputStream; -import java.io.IOException; -import java.io.Serializable; - /** * Broadcast helper. */ diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/util/ByteArray.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/util/ByteArray.java index 8c493f5bc2b90..bb0c43912c97d 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/util/ByteArray.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/util/ByteArray.java @@ -18,7 +18,6 @@ package org.apache.beam.runners.spark.util; import com.google.common.primitives.UnsignedBytes; - import java.io.Serializable; import java.util.Arrays; diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/DeDupTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/DeDupTest.java index 9a167446847e0..2d06a207a3717 100644 --- a/runners/spark/src/test/java/org/apache/beam/runners/spark/DeDupTest.java +++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/DeDupTest.java @@ -18,6 +18,10 @@ package org.apache.beam.runners.spark; +import com.google.common.collect.ImmutableSet; +import java.util.Arrays; +import java.util.List; +import java.util.Set; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.coders.StringUtf8Coder; import org.apache.beam.sdk.options.PipelineOptionsFactory; @@ -25,15 +29,8 @@ import org.apache.beam.sdk.transforms.Create; import org.apache.beam.sdk.transforms.RemoveDuplicates; import org.apache.beam.sdk.values.PCollection; - -import com.google.common.collect.ImmutableSet; - import org.junit.Test; -import java.util.Arrays; -import java.util.List; -import java.util.Set; - /** * A test based on {@code DeDupExample} from the SDK. */ diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/EmptyInputTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/EmptyInputTest.java index c2e331f78364e..90ce4278bdf60 100644 --- a/runners/spark/src/test/java/org/apache/beam/runners/spark/EmptyInputTest.java +++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/EmptyInputTest.java @@ -20,6 +20,9 @@ import static org.junit.Assert.assertEquals; +import com.google.common.collect.Iterables; +import java.util.Collections; +import java.util.List; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.coders.StringUtf8Coder; import org.apache.beam.sdk.options.PipelineOptionsFactory; @@ -27,14 +30,8 @@ import org.apache.beam.sdk.transforms.Create; import org.apache.beam.sdk.transforms.SerializableFunction; import org.apache.beam.sdk.values.PCollection; - -import com.google.common.collect.Iterables; - import org.junit.Test; -import java.util.Collections; -import java.util.List; - /** * Empty input test. */ diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/SimpleWordCountTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/SimpleWordCountTest.java index 441d92d2af65c..f644765682f18 100644 --- a/runners/spark/src/test/java/org/apache/beam/runners/spark/SimpleWordCountTest.java +++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/SimpleWordCountTest.java @@ -21,6 +21,12 @@ import static org.hamcrest.Matchers.containsInAnyOrder; import static org.junit.Assert.assertThat; +import com.google.common.collect.ImmutableSet; +import com.google.common.collect.Sets; +import java.io.File; +import java.util.Arrays; +import java.util.List; +import java.util.Set; import org.apache.beam.runners.spark.examples.WordCount; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.coders.StringUtf8Coder; @@ -30,20 +36,11 @@ import org.apache.beam.sdk.transforms.Create; import org.apache.beam.sdk.transforms.MapElements; import org.apache.beam.sdk.values.PCollection; - -import com.google.common.collect.ImmutableSet; -import com.google.common.collect.Sets; - import org.apache.commons.io.FileUtils; import org.junit.Rule; import org.junit.Test; import org.junit.rules.TemporaryFolder; -import java.io.File; -import java.util.Arrays; -import java.util.List; -import java.util.Set; - /** * Simple word count test. */ diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/SparkRunnerRegistrarTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/SparkRunnerRegistrarTest.java index 3ca9df4ad8396..4e1fd7c5afeee 100644 --- a/runners/spark/src/test/java/org/apache/beam/runners/spark/SparkRunnerRegistrarTest.java +++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/SparkRunnerRegistrarTest.java @@ -21,18 +21,15 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.fail; -import org.apache.beam.sdk.options.PipelineOptionsRegistrar; -import org.apache.beam.sdk.runners.PipelineRunnerRegistrar; - import com.google.common.collect.ImmutableList; import com.google.common.collect.Lists; - +import java.util.ServiceLoader; +import org.apache.beam.sdk.options.PipelineOptionsRegistrar; +import org.apache.beam.sdk.runners.PipelineRunnerRegistrar; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.util.ServiceLoader; - /** * Test {@link SparkRunnerRegistrar}. */ diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/TfIdfTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/TfIdfTest.java index 17bf6ddbe1b10..d911bfb965b0f 100644 --- a/runners/spark/src/test/java/org/apache/beam/runners/spark/TfIdfTest.java +++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/TfIdfTest.java @@ -18,6 +18,8 @@ package org.apache.beam.runners.spark; +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.options.PipelineOptionsFactory; @@ -36,16 +38,12 @@ import org.apache.beam.sdk.transforms.join.KeyedPCollectionTuple; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; - import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TupleTag; import org.junit.Test; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.net.URI; -import java.util.Arrays; - /** * A test based on {@code TfIdf} from the SDK. */ diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/coders/WritableCoderTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/coders/WritableCoderTest.java index f2bd4d3e76fae..538fd97a8bb02 100644 --- a/runners/spark/src/test/java/org/apache/beam/runners/spark/coders/WritableCoderTest.java +++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/coders/WritableCoderTest.java @@ -19,7 +19,6 @@ package org.apache.beam.runners.spark.coders; import org.apache.beam.sdk.testing.CoderProperties; - import org.apache.hadoop.io.IntWritable; import org.apache.hadoop.io.NullWritable; import org.junit.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 d86242402c7cd..413a71c1d7b2f 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 @@ -20,17 +20,12 @@ import static org.junit.Assert.assertEquals; -import org.apache.beam.runners.spark.EvaluationResult; -import org.apache.beam.runners.spark.SparkRunner; -import org.apache.beam.sdk.Pipeline; -import org.apache.beam.sdk.io.AvroIO; -import org.apache.beam.sdk.options.PipelineOptions; -import org.apache.beam.sdk.options.PipelineOptionsFactory; -import org.apache.beam.sdk.values.PCollection; - import com.google.common.collect.Lists; import com.google.common.io.Resources; - +import java.io.File; +import java.io.FileOutputStream; +import java.io.IOException; +import java.util.List; import org.apache.avro.Schema; import org.apache.avro.file.DataFileReader; import org.apache.avro.file.DataFileWriter; @@ -38,16 +33,18 @@ import org.apache.avro.generic.GenericDatumReader; import org.apache.avro.generic.GenericDatumWriter; import org.apache.avro.generic.GenericRecord; +import org.apache.beam.runners.spark.EvaluationResult; +import org.apache.beam.runners.spark.SparkRunner; +import org.apache.beam.sdk.Pipeline; +import org.apache.beam.sdk.io.AvroIO; +import org.apache.beam.sdk.options.PipelineOptions; +import org.apache.beam.sdk.options.PipelineOptionsFactory; +import org.apache.beam.sdk.values.PCollection; import org.junit.Before; import org.junit.Rule; import org.junit.Test; import org.junit.rules.TemporaryFolder; -import java.io.File; -import java.io.FileOutputStream; -import java.io.IOException; -import java.util.List; - /** * Avro pipeline test. */ 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 9c65917ece381..81803c338b91b 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 @@ -21,6 +21,15 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; +import com.google.common.base.Charsets; +import com.google.common.collect.Sets; +import com.google.common.io.Files; +import java.io.File; +import java.io.FileFilter; +import java.io.IOException; +import java.util.Arrays; +import java.util.List; +import java.util.Set; import org.apache.beam.runners.spark.EvaluationResult; import org.apache.beam.runners.spark.SparkPipelineOptions; import org.apache.beam.runners.spark.SparkRunner; @@ -32,23 +41,11 @@ import org.apache.beam.sdk.transforms.Create; import org.apache.beam.sdk.transforms.MapElements; import org.apache.beam.sdk.values.PCollection; - -import com.google.common.base.Charsets; -import com.google.common.collect.Sets; -import com.google.common.io.Files; - import org.junit.Before; import org.junit.Rule; import org.junit.Test; import org.junit.rules.TemporaryFolder; -import java.io.File; -import java.io.FileFilter; -import java.io.IOException; -import java.util.Arrays; -import java.util.List; -import java.util.Set; - /** * Number of shards test. */ 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 01aa839bde2e3..b14465d7c9b37 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 @@ -20,6 +20,8 @@ import static org.junit.Assert.assertEquals; +import java.io.File; +import java.io.IOException; import org.apache.beam.runners.spark.EvaluationResult; import org.apache.beam.runners.spark.SparkRunner; import org.apache.beam.runners.spark.coders.WritableCoder; @@ -29,7 +31,6 @@ import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; import org.apache.hadoop.io.IntWritable; @@ -45,9 +46,6 @@ import org.junit.Test; import org.junit.rules.TemporaryFolder; -import java.io.File; -import java.io.IOException; - /** * Pipeline on the Hadoop file format test. */ diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/io/hadoop/ShardNameBuilderTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/io/hadoop/ShardNameBuilderTest.java index e1620dbc7c975..1f2cf639ba8b5 100644 --- a/runners/spark/src/test/java/org/apache/beam/runners/spark/io/hadoop/ShardNameBuilderTest.java +++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/io/hadoop/ShardNameBuilderTest.java @@ -23,7 +23,6 @@ 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.io.hadoop.ShardNameBuilder.replaceShardNumber; - import static org.junit.Assert.assertEquals; import org.junit.Test; diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/CombineGloballyTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/CombineGloballyTest.java index e4ef7d7052126..8022d064c1687 100644 --- a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/CombineGloballyTest.java +++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/CombineGloballyTest.java @@ -20,6 +20,9 @@ import static org.junit.Assert.assertEquals; +import com.google.common.collect.Iterables; +import java.util.Arrays; +import java.util.List; import org.apache.beam.runners.spark.EvaluationResult; import org.apache.beam.runners.spark.SparkPipelineOptions; import org.apache.beam.runners.spark.SparkRunner; @@ -29,14 +32,8 @@ import org.apache.beam.sdk.transforms.Combine; import org.apache.beam.sdk.transforms.Create; import org.apache.beam.sdk.values.PCollection; - -import com.google.common.collect.Iterables; - import org.junit.Test; -import java.util.Arrays; -import java.util.List; - /** * Combine globally test. */ diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/CombinePerKeyTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/CombinePerKeyTest.java index cdf2cfbb892f8..281144f362f9c 100644 --- a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/CombinePerKeyTest.java +++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/CombinePerKeyTest.java @@ -18,6 +18,10 @@ package org.apache.beam.runners.spark.translation; +import com.google.common.collect.ImmutableList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; import org.apache.beam.runners.spark.EvaluationResult; import org.apache.beam.runners.spark.SparkRunner; import org.apache.beam.sdk.Pipeline; @@ -33,16 +37,9 @@ import org.apache.beam.sdk.transforms.Sum; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; - -import com.google.common.collect.ImmutableList; - import org.junit.Assert; import org.junit.Test; -import java.util.HashMap; -import java.util.List; -import java.util.Map; - /** * Combine per key function test. */ diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/DoFnOutputTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/DoFnOutputTest.java index e4b25bb06f74b..31e0dd8d4ff02 100644 --- a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/DoFnOutputTest.java +++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/DoFnOutputTest.java @@ -18,6 +18,7 @@ package org.apache.beam.runners.spark.translation; +import java.io.Serializable; import org.apache.beam.runners.spark.EvaluationResult; import org.apache.beam.runners.spark.SparkPipelineOptions; import org.apache.beam.runners.spark.SparkRunner; @@ -28,11 +29,8 @@ import org.apache.beam.sdk.transforms.OldDoFn; import org.apache.beam.sdk.transforms.ParDo; import org.apache.beam.sdk.values.PCollection; - import org.junit.Test; -import java.io.Serializable; - /** * DoFN output test. */ diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/MultiOutputWordCountTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/MultiOutputWordCountTest.java index 0d0c0b41efd8e..517596aab7822 100644 --- a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/MultiOutputWordCountTest.java +++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/MultiOutputWordCountTest.java @@ -18,6 +18,9 @@ package org.apache.beam.runners.spark.translation; +import com.google.common.collect.ImmutableSet; +import com.google.common.collect.Iterables; +import java.util.Set; import org.apache.beam.runners.spark.EvaluationResult; import org.apache.beam.runners.spark.SparkRunner; import org.apache.beam.sdk.AggregatorValues; @@ -44,15 +47,9 @@ import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TupleTag; import org.apache.beam.sdk.values.TupleTagList; - -import com.google.common.collect.ImmutableSet; -import com.google.common.collect.Iterables; - import org.junit.Assert; import org.junit.Test; -import java.util.Set; - /** * Multi-output word count test. */ diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/SerializationTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/SerializationTest.java index 019b107e0ddd9..0e9121ca87f97 100644 --- a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/SerializationTest.java +++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/SerializationTest.java @@ -18,6 +18,16 @@ package org.apache.beam.runners.spark.translation; +import com.google.common.base.Function; +import com.google.common.collect.ImmutableSet; +import com.google.common.collect.Lists; +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.util.Arrays; +import java.util.List; +import java.util.Set; +import java.util.regex.Pattern; import org.apache.beam.runners.spark.EvaluationResult; import org.apache.beam.runners.spark.SparkPipelineOptions; import org.apache.beam.runners.spark.SparkRunner; @@ -36,21 +46,8 @@ import org.apache.beam.sdk.transforms.Sum; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; - -import com.google.common.base.Function; -import com.google.common.collect.ImmutableSet; -import com.google.common.collect.Lists; - import org.junit.Test; -import java.io.IOException; -import java.io.InputStream; -import java.io.OutputStream; -import java.util.Arrays; -import java.util.List; -import java.util.Set; -import java.util.regex.Pattern; - /** * Serialization test. */ 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 index 6cefa49ac04b7..0d15d12d3f3b0 100644 --- 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 @@ -22,6 +22,8 @@ import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; +import java.io.Serializable; +import java.net.URI; import org.apache.beam.runners.spark.SparkPipelineOptions; import org.apache.beam.runners.spark.SparkRunner; import org.apache.beam.sdk.Pipeline; @@ -30,14 +32,10 @@ import org.apache.beam.sdk.transforms.Create; import org.apache.beam.sdk.transforms.OldDoFn; import org.apache.beam.sdk.transforms.ParDo; - import org.junit.After; import org.junit.Before; import org.junit.Test; -import java.io.Serializable; -import java.net.URI; - /** * Side effects test. */ diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/SparkPipelineOptionsTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/SparkPipelineOptionsTest.java index 59888c253ef56..2b655cbc8d740 100644 --- a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/SparkPipelineOptionsTest.java +++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/SparkPipelineOptionsTest.java @@ -20,7 +20,6 @@ import org.apache.beam.runners.spark.SparkPipelineOptions; import org.apache.beam.sdk.options.PipelineOptionsFactory; - import org.junit.Assert; import org.junit.Test; diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/TransformTranslatorTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/TransformTranslatorTest.java index e1789f145e54c..f61ad1c9a9fda 100644 --- a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/TransformTranslatorTest.java +++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/TransformTranslatorTest.java @@ -21,6 +21,15 @@ import static org.hamcrest.Matchers.containsInAnyOrder; import static org.junit.Assert.assertThat; +import com.google.common.base.Charsets; +import com.google.common.io.Files; +import java.io.File; +import java.io.IOException; +import java.nio.charset.StandardCharsets; +import java.nio.file.Paths; +import java.util.ArrayList; +import java.util.List; +import java.util.regex.Pattern; import org.apache.beam.runners.spark.SparkRunner; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.io.TextIO; @@ -28,10 +37,6 @@ import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.runners.PipelineRunner; import org.apache.beam.sdk.values.PCollection; - -import com.google.common.base.Charsets; -import com.google.common.io.Files; - import org.apache.commons.io.FileUtils; import org.junit.Rule; import org.junit.Test; @@ -39,14 +44,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.io.File; -import java.io.IOException; -import java.nio.charset.StandardCharsets; -import java.nio.file.Paths; -import java.util.ArrayList; -import java.util.List; -import java.util.regex.Pattern; - /** * A test for the transforms registered in TransformTranslator. * Builds a regular Beam pipeline with each of the mapped diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/WindowedWordCountTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/WindowedWordCountTest.java index b70e090ddae7a..69b2943e6f2d1 100644 --- a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/WindowedWordCountTest.java +++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/WindowedWordCountTest.java @@ -18,6 +18,9 @@ package org.apache.beam.runners.spark.translation; +import com.google.common.collect.ImmutableList; +import java.util.Arrays; +import java.util.List; import org.apache.beam.runners.spark.EvaluationResult; import org.apache.beam.runners.spark.SparkRunner; import org.apache.beam.runners.spark.examples.WordCount; @@ -32,15 +35,9 @@ import org.apache.beam.sdk.transforms.windowing.SlidingWindows; import org.apache.beam.sdk.transforms.windowing.Window; import org.apache.beam.sdk.values.PCollection; - -import com.google.common.collect.ImmutableList; - import org.joda.time.Duration; import org.junit.Test; -import java.util.Arrays; -import java.util.List; - /** * Windowed word count test. */ 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 6f4d8fbf2d080..a6fe755c43de1 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 @@ -17,6 +17,9 @@ */ package org.apache.beam.runners.spark.translation.streaming; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; import org.apache.beam.runners.spark.EvaluationResult; import org.apache.beam.runners.spark.SparkPipelineOptions; import org.apache.beam.runners.spark.SparkRunner; @@ -30,14 +33,9 @@ import org.apache.beam.sdk.transforms.windowing.Window; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionList; - import org.joda.time.Duration; import org.junit.Test; -import java.util.Arrays; -import java.util.Collections; -import java.util.List; - /** * Test Flatten (union) implementation for streaming. */ 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 25271522aed59..27d6f5ea60689 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 @@ -17,6 +17,12 @@ */ package org.apache.beam.runners.spark.translation.streaming; +import com.google.common.collect.ImmutableMap; +import java.io.IOException; +import java.util.Collections; +import java.util.Map; +import java.util.Properties; +import kafka.serializer.StringDecoder; import org.apache.beam.runners.spark.EvaluationResult; import org.apache.beam.runners.spark.SparkPipelineOptions; import org.apache.beam.runners.spark.SparkRunner; @@ -33,9 +39,6 @@ import org.apache.beam.sdk.transforms.windowing.Window; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; - -import com.google.common.collect.ImmutableMap; - import org.apache.kafka.clients.producer.KafkaProducer; import org.apache.kafka.clients.producer.ProducerRecord; import org.apache.kafka.common.serialization.Serializer; @@ -45,13 +48,6 @@ import org.junit.AfterClass; import org.junit.BeforeClass; import org.junit.Test; - -import java.io.IOException; -import java.util.Collections; -import java.util.Map; -import java.util.Properties; - -import kafka.serializer.StringDecoder; /** * Test Kafka as input. */ 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 c761faed0e2e6..671d227d3ac90 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 @@ -18,6 +18,10 @@ package org.apache.beam.runners.spark.translation.streaming; +import java.io.Serializable; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; import org.apache.beam.runners.spark.EvaluationResult; import org.apache.beam.runners.spark.SparkPipelineOptions; import org.apache.beam.runners.spark.SparkRunner; @@ -31,16 +35,10 @@ import org.apache.beam.sdk.transforms.windowing.FixedWindows; import org.apache.beam.sdk.transforms.windowing.Window; import org.apache.beam.sdk.values.PCollection; - import org.apache.spark.streaming.Durations; import org.joda.time.Duration; import org.junit.Test; -import java.io.Serializable; -import java.util.Arrays; -import java.util.Collections; -import java.util.List; - /** * Simple word count streaming test. */ diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/utils/EmbeddedKafkaCluster.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/utils/EmbeddedKafkaCluster.java index cd326ef03355e..d1729a4ca0a63 100644 --- a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/utils/EmbeddedKafkaCluster.java +++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/utils/EmbeddedKafkaCluster.java @@ -17,12 +17,6 @@ */ package org.apache.beam.runners.spark.translation.streaming.utils; -import org.apache.zookeeper.server.NIOServerCnxnFactory; -import org.apache.zookeeper.server.ServerCnxnFactory; -import org.apache.zookeeper.server.ZooKeeperServer; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - import java.io.File; import java.io.FileNotFoundException; import java.io.IOException; @@ -33,10 +27,14 @@ import java.util.List; import java.util.Properties; import java.util.Random; - import kafka.server.KafkaConfig; import kafka.server.KafkaServer; import kafka.utils.Time; +import org.apache.zookeeper.server.NIOServerCnxnFactory; +import org.apache.zookeeper.server.ServerCnxnFactory; +import org.apache.zookeeper.server.ZooKeeperServer; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * Embedded Kafka cluster. 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 f85c4409df090..97425bd32c8bc 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 @@ -20,6 +20,7 @@ import static org.hamcrest.Matchers.containsInAnyOrder; import static org.junit.Assert.assertThat; +import java.io.Serializable; import org.apache.beam.runners.spark.EvaluationResult; import org.apache.beam.sdk.transforms.GroupByKey; import org.apache.beam.sdk.transforms.MapElements; @@ -27,11 +28,8 @@ import org.apache.beam.sdk.transforms.Values; import org.apache.beam.sdk.transforms.WithKeys; import org.apache.beam.sdk.values.PCollection; - import org.junit.Assert; -import java.io.Serializable; - /** * Since PAssert doesn't propagate assert exceptions, use Aggregators to assert streaming * success/failure counters. 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 ac215c9ed1f06..0e79abe9683eb 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 @@ -17,6 +17,11 @@ */ package org.apache.beam.sdk; +import com.google.common.collect.HashMultimap; +import com.google.common.collect.SetMultimap; +import java.util.Collection; +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.transforms.Aggregator; @@ -25,13 +30,6 @@ import org.apache.beam.sdk.transforms.ParDo; import org.apache.beam.sdk.values.PValue; -import com.google.common.collect.HashMultimap; -import com.google.common.collect.SetMultimap; - -import java.util.Collection; -import java.util.Collections; -import java.util.Map; - /** * Retrieves {@link Aggregator Aggregators} at each {@link ParDo} and returns a {@link Map} of * {@link Aggregator} to the {@link PTransform PTransforms} in which it is present. diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/AggregatorValues.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/AggregatorValues.java index efaad85b67b54..6297085319e2a 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/AggregatorValues.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/AggregatorValues.java @@ -17,13 +17,12 @@ */ package org.apache.beam.sdk; +import java.util.Collection; +import java.util.Map; import org.apache.beam.sdk.transforms.Aggregator; import org.apache.beam.sdk.transforms.Combine.CombineFn; import org.apache.beam.sdk.transforms.OldDoFn; -import java.util.Collection; -import java.util.Map; - /** * A collection of values associated with an {@link Aggregator}. Aggregators declared in a * {@link OldDoFn} are emitted on a per-{@code OldDoFn}-application basis. 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 1bbc56f1fad22..53f46f6f06aba 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 @@ -19,6 +19,15 @@ import static com.google.common.base.Preconditions.checkState; +import com.google.common.collect.HashMultimap; +import com.google.common.collect.Iterables; +import com.google.common.collect.Multimap; +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; import org.apache.beam.sdk.io.Read; import org.apache.beam.sdk.options.PipelineOptions; @@ -36,21 +45,9 @@ import org.apache.beam.sdk.values.PInput; import org.apache.beam.sdk.values.POutput; import org.apache.beam.sdk.values.PValue; - -import com.google.common.collect.HashMultimap; -import com.google.common.collect.Iterables; -import com.google.common.collect.Multimap; - import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.util.ArrayList; -import java.util.Collection; -import java.util.HashSet; -import java.util.List; -import java.util.Map; -import java.util.Set; - /** * A {@link Pipeline} manages a directed acyclic graph of {@link PTransform PTransforms}, and the * {@link PCollection PCollections} that the {@link PTransform}s consume and produce. diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/PipelineResult.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/PipelineResult.java index edfc9248f527a..d9cdc16a3da47 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/PipelineResult.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/PipelineResult.java @@ -17,12 +17,10 @@ */ package org.apache.beam.sdk; +import java.io.IOException; import org.apache.beam.sdk.transforms.Aggregator; - import org.joda.time.Duration; -import java.io.IOException; - /** * Result of {@link Pipeline#run()}. */ diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/AvroCoder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/AvroCoder.java index da4db93e19533..7894d14ccd1db 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/AvroCoder.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/AvroCoder.java @@ -19,11 +19,22 @@ import static org.apache.beam.sdk.util.Structs.addString; -import org.apache.beam.sdk.util.CloudObject; -import org.apache.beam.sdk.values.TypeDescriptor; - import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.io.Serializable; +import java.lang.reflect.Field; +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 java.util.SortedMap; +import java.util.SortedSet; +import javax.annotation.Nullable; import org.apache.avro.Schema; import org.apache.avro.generic.GenericDatumReader; import org.apache.avro.generic.GenericDatumWriter; @@ -45,22 +56,8 @@ import org.apache.avro.specific.SpecificData; import org.apache.avro.util.ClassUtils; import org.apache.avro.util.Utf8; - -import java.io.IOException; -import java.io.InputStream; -import java.io.OutputStream; -import java.io.Serializable; -import java.lang.reflect.Field; -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 java.util.SortedMap; -import java.util.SortedSet; - -import javax.annotation.Nullable; +import org.apache.beam.sdk.util.CloudObject; +import org.apache.beam.sdk.values.TypeDescriptor; /** * A {@link Coder} using Avro binary format. diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/BigDecimalCoder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/BigDecimalCoder.java index c4b7cd1155109..e2628821c96d2 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/BigDecimalCoder.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/BigDecimalCoder.java @@ -20,7 +20,6 @@ import static com.google.common.base.Preconditions.checkNotNull; import com.fasterxml.jackson.annotation.JsonCreator; - import java.io.IOException; import java.io.InputStream; import java.io.OutputStream; diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/BigEndianIntegerCoder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/BigEndianIntegerCoder.java index bd3bcf3800aae..ac8db120a0225 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/BigEndianIntegerCoder.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/BigEndianIntegerCoder.java @@ -18,7 +18,6 @@ package org.apache.beam.sdk.coders; import com.fasterxml.jackson.annotation.JsonCreator; - import java.io.DataInputStream; import java.io.DataOutputStream; import java.io.EOFException; diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/BigEndianLongCoder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/BigEndianLongCoder.java index 5f5793ed7723a..e0053241f8d03 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/BigEndianLongCoder.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/BigEndianLongCoder.java @@ -18,7 +18,6 @@ package org.apache.beam.sdk.coders; import com.fasterxml.jackson.annotation.JsonCreator; - import java.io.DataInputStream; import java.io.DataOutputStream; import java.io.EOFException; diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/BigIntegerCoder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/BigIntegerCoder.java index a41defed15b21..daba983a83b5b 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/BigIntegerCoder.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/BigIntegerCoder.java @@ -20,7 +20,6 @@ import static com.google.common.base.Preconditions.checkNotNull; import com.fasterxml.jackson.annotation.JsonCreator; - import java.io.IOException; import java.io.InputStream; import java.io.OutputStream; diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/ByteArrayCoder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/ByteArrayCoder.java index 4115825bf3d5f..65e24da8c220e 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/ByteArrayCoder.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/ByteArrayCoder.java @@ -17,17 +17,14 @@ */ package org.apache.beam.sdk.coders; -import org.apache.beam.sdk.util.ExposedByteArrayOutputStream; -import org.apache.beam.sdk.util.StreamUtils; -import org.apache.beam.sdk.util.VarInt; - -import com.google.common.io.ByteStreams; - import com.fasterxml.jackson.annotation.JsonCreator; - +import com.google.common.io.ByteStreams; import java.io.IOException; import java.io.InputStream; import java.io.OutputStream; +import org.apache.beam.sdk.util.ExposedByteArrayOutputStream; +import org.apache.beam.sdk.util.StreamUtils; +import org.apache.beam.sdk.util.VarInt; /** * A {@link Coder} for {@code byte[]}. diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/ByteCoder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/ByteCoder.java index e146945c8e16e..c912b355b149e 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/ByteCoder.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/ByteCoder.java @@ -18,7 +18,6 @@ package org.apache.beam.sdk.coders; import com.fasterxml.jackson.annotation.JsonCreator; - import java.io.EOFException; import java.io.IOException; import java.io.InputStream; diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/ByteStringCoder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/ByteStringCoder.java index b03f98a3e174f..c70b9db978e53 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/ByteStringCoder.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/ByteStringCoder.java @@ -17,16 +17,13 @@ */ package org.apache.beam.sdk.coders; -import org.apache.beam.sdk.util.VarInt; - +import com.fasterxml.jackson.annotation.JsonCreator; import com.google.common.io.ByteStreams; import com.google.protobuf.ByteString; - -import com.fasterxml.jackson.annotation.JsonCreator; - import java.io.IOException; import java.io.InputStream; import java.io.OutputStream; +import org.apache.beam.sdk.util.VarInt; /** * A {@link Coder} for {@link ByteString} objects based on their encoded Protocol Buffer form. diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/Coder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/Coder.java index 136c00a21796d..688d1f793ab12 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/Coder.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/Coder.java @@ -19,15 +19,9 @@ import static com.google.common.base.Preconditions.checkArgument; -import org.apache.beam.sdk.annotations.Experimental; -import org.apache.beam.sdk.annotations.Experimental.Kind; -import org.apache.beam.sdk.util.CloudObject; -import org.apache.beam.sdk.util.common.ElementByteSizeObserver; - import com.google.common.base.Joiner; import com.google.common.base.MoreObjects; import com.google.common.base.Objects; - import java.io.IOException; import java.io.InputStream; import java.io.OutputStream; @@ -35,8 +29,11 @@ import java.util.Arrays; import java.util.Collection; import java.util.List; - import javax.annotation.Nullable; +import org.apache.beam.sdk.annotations.Experimental; +import org.apache.beam.sdk.annotations.Experimental.Kind; +import org.apache.beam.sdk.util.CloudObject; +import org.apache.beam.sdk.util.common.ElementByteSizeObserver; /** * A {@link Coder Coder<T>} defines how to encode and decode values of type {@code T} into diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/CoderFactories.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/CoderFactories.java index 480f424182d8c..e1a202a26467a 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/CoderFactories.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/CoderFactories.java @@ -17,8 +17,6 @@ */ package org.apache.beam.sdk.coders; -import org.apache.beam.sdk.values.TypeDescriptor; - import java.lang.reflect.InvocationTargetException; import java.lang.reflect.Method; import java.lang.reflect.Modifier; @@ -26,6 +24,7 @@ import java.util.Arrays; import java.util.Collections; import java.util.List; +import org.apache.beam.sdk.values.TypeDescriptor; /** * Static utility methods for creating and working with {@link Coder}s. diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/CoderProviders.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/CoderProviders.java index 3b852e2ab1f0e..c072008d5f288 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/CoderProviders.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/CoderProviders.java @@ -19,15 +19,13 @@ import static com.google.common.base.Preconditions.checkArgument; -import org.apache.beam.sdk.util.InstanceBuilder; -import org.apache.beam.sdk.values.TypeDescriptor; - import com.google.common.base.Joiner; import com.google.common.collect.ImmutableList; import com.google.common.collect.Lists; - import java.lang.reflect.InvocationTargetException; import java.util.List; +import org.apache.beam.sdk.util.InstanceBuilder; +import org.apache.beam.sdk.values.TypeDescriptor; /** * Static utility methods for working with {@link CoderProvider CoderProviders}. 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 411ebe2bbb589..9110de058af6a 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 @@ -19,24 +19,11 @@ import static com.google.common.base.Preconditions.checkArgument; -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.util.CoderUtils; -import org.apache.beam.sdk.values.KV; -import org.apache.beam.sdk.values.TimestampedValue; -import org.apache.beam.sdk.values.TypeDescriptor; - import com.google.api.services.bigquery.model.TableRow; import com.google.common.annotations.VisibleForTesting; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Maps; import com.google.protobuf.ByteString; - -import org.joda.time.Instant; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - import java.lang.reflect.ParameterizedType; import java.lang.reflect.Type; import java.lang.reflect.TypeVariable; @@ -49,8 +36,17 @@ import java.util.List; import java.util.Map; import java.util.Set; - import javax.annotation.Nullable; +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.util.CoderUtils; +import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.TimestampedValue; +import org.apache.beam.sdk.values.TypeDescriptor; +import org.joda.time.Instant; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * A {@link CoderRegistry} allows registering the default {@link Coder} to use for a Java class, diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/CollectionCoder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/CollectionCoder.java index f40d5f029ffb9..7c61e882c35b4 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/CollectionCoder.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/CollectionCoder.java @@ -19,13 +19,11 @@ import static com.google.common.base.Preconditions.checkArgument; -import org.apache.beam.sdk.util.PropertyNames; - import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; - import java.util.Collection; import java.util.List; +import org.apache.beam.sdk.util.PropertyNames; /** * A {@link CollectionCoder} encodes {@link Collection Collections} in the format diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/CustomCoder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/CustomCoder.java index 69412e5849aaf..2614cc178c029 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/CustomCoder.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/CustomCoder.java @@ -17,23 +17,19 @@ */ package org.apache.beam.sdk.coders; +import static com.google.common.base.Preconditions.checkNotNull; import static org.apache.beam.sdk.util.Structs.addString; import static org.apache.beam.sdk.util.Structs.addStringList; -import static com.google.common.base.Preconditions.checkNotNull; - -import org.apache.beam.sdk.util.CloudObject; -import org.apache.beam.sdk.util.PropertyNames; -import org.apache.beam.sdk.util.SerializableUtils; -import org.apache.beam.sdk.util.StringUtils; - -import com.google.common.collect.Lists; - import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; - +import com.google.common.collect.Lists; import java.io.Serializable; import java.util.Collection; +import org.apache.beam.sdk.util.CloudObject; +import org.apache.beam.sdk.util.PropertyNames; +import org.apache.beam.sdk.util.SerializableUtils; +import org.apache.beam.sdk.util.StringUtils; /** * An abstract base class for writing a {@link Coder} class that encodes itself via Java diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/DefaultCoder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/DefaultCoder.java index 3eb1253c54cb5..9a976f9d4cc43 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/DefaultCoder.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/DefaultCoder.java @@ -17,13 +17,12 @@ */ package org.apache.beam.sdk.coders; -import org.apache.beam.sdk.values.PCollection; - import java.lang.annotation.Documented; import java.lang.annotation.ElementType; import java.lang.annotation.Retention; import java.lang.annotation.RetentionPolicy; import java.lang.annotation.Target; +import org.apache.beam.sdk.values.PCollection; /** * The {@link DefaultCoder} annotation diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/DelegateCoder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/DelegateCoder.java index 385c14996367f..e7f9d4ac11f15 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/DelegateCoder.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/DelegateCoder.java @@ -20,7 +20,6 @@ import com.google.common.base.MoreObjects; import com.google.common.base.Objects; import com.google.common.collect.Lists; - import java.io.IOException; import java.io.InputStream; import java.io.OutputStream; diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/DoubleCoder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/DoubleCoder.java index cb204ecb71ace..4e56914a18ad7 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/DoubleCoder.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/DoubleCoder.java @@ -18,7 +18,6 @@ package org.apache.beam.sdk.coders; import com.fasterxml.jackson.annotation.JsonCreator; - import java.io.DataInputStream; import java.io.DataOutputStream; import java.io.EOFException; diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/DurationCoder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/DurationCoder.java index c34ce66eb0196..a2458f249cf3d 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/DurationCoder.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/DurationCoder.java @@ -17,15 +17,13 @@ */ package org.apache.beam.sdk.coders; -import org.apache.beam.sdk.util.common.ElementByteSizeObserver; - import com.fasterxml.jackson.annotation.JsonCreator; -import org.joda.time.Duration; -import org.joda.time.ReadableDuration; - import java.io.IOException; import java.io.InputStream; import java.io.OutputStream; +import org.apache.beam.sdk.util.common.ElementByteSizeObserver; +import org.joda.time.Duration; +import org.joda.time.ReadableDuration; /** * A {@link Coder} that encodes a joda {@link Duration} as a {@link Long} using the format of diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/InstantCoder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/InstantCoder.java index d41bd1f21bd08..c0409fba46ccb 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/InstantCoder.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/InstantCoder.java @@ -17,18 +17,14 @@ */ package org.apache.beam.sdk.coders; -import org.apache.beam.sdk.util.common.ElementByteSizeObserver; - -import com.google.common.base.Converter; - import com.fasterxml.jackson.annotation.JsonCreator; -import org.joda.time.Instant; - +import com.google.common.base.Converter; +import edu.umd.cs.findbugs.annotations.SuppressFBWarnings; import java.io.IOException; import java.io.InputStream; import java.io.OutputStream; - -import edu.umd.cs.findbugs.annotations.SuppressFBWarnings; +import org.apache.beam.sdk.util.common.ElementByteSizeObserver; +import org.joda.time.Instant; /** * A {@link Coder} for joda {@link Instant} that encodes it as a big endian {@link Long} diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/IterableCoder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/IterableCoder.java index 2e5a8cc361c9a..11fb1720deb75 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/IterableCoder.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/IterableCoder.java @@ -17,17 +17,14 @@ */ package org.apache.beam.sdk.coders; -import static org.apache.beam.sdk.util.Structs.addBoolean; - import static com.google.common.base.Preconditions.checkArgument; - -import org.apache.beam.sdk.util.CloudObject; -import org.apache.beam.sdk.util.PropertyNames; +import static org.apache.beam.sdk.util.Structs.addBoolean; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; - import java.util.List; +import org.apache.beam.sdk.util.CloudObject; +import org.apache.beam.sdk.util.PropertyNames; /** * An {@link IterableCoder} encodes any {@link Iterable} in the format diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/IterableLikeCoder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/IterableLikeCoder.java index 9417d85612015..8680552bddc51 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/IterableLikeCoder.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/IterableLikeCoder.java @@ -19,11 +19,6 @@ import static com.google.common.base.Preconditions.checkArgument; -import org.apache.beam.sdk.util.BufferedElementCountingOutputStream; -import org.apache.beam.sdk.util.VarInt; -import org.apache.beam.sdk.util.common.ElementByteSizeObservableIterable; -import org.apache.beam.sdk.util.common.ElementByteSizeObserver; - import java.io.DataInputStream; import java.io.DataOutputStream; import java.io.IOException; @@ -35,6 +30,10 @@ import java.util.List; import java.util.Observable; import java.util.Observer; +import org.apache.beam.sdk.util.BufferedElementCountingOutputStream; +import org.apache.beam.sdk.util.VarInt; +import org.apache.beam.sdk.util.common.ElementByteSizeObservableIterable; +import org.apache.beam.sdk.util.common.ElementByteSizeObserver; /** * An abstract base class with functionality for assembling a diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/JAXBCoder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/JAXBCoder.java index 689f993a1cdeb..748b07d8f6f9f 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/JAXBCoder.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/JAXBCoder.java @@ -17,25 +17,21 @@ */ package org.apache.beam.sdk.coders; -import org.apache.beam.sdk.util.CloudObject; -import org.apache.beam.sdk.util.Structs; -import org.apache.beam.sdk.util.VarInt; - -import com.google.common.io.ByteStreams; - import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; - +import com.google.common.io.ByteStreams; import java.io.FilterInputStream; import java.io.FilterOutputStream; import java.io.IOException; import java.io.InputStream; import java.io.OutputStream; - import javax.xml.bind.JAXBContext; import javax.xml.bind.JAXBException; import javax.xml.bind.Marshaller; import javax.xml.bind.Unmarshaller; +import org.apache.beam.sdk.util.CloudObject; +import org.apache.beam.sdk.util.Structs; +import org.apache.beam.sdk.util.VarInt; /** * A coder for JAXB annotated objects. This coder uses JAXB marshalling/unmarshalling mechanisms diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/KvCoder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/KvCoder.java index beeb9c5267a66..ad13226b05c43 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/KvCoder.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/KvCoder.java @@ -17,23 +17,20 @@ */ package org.apache.beam.sdk.coders; -import static org.apache.beam.sdk.util.Structs.addBoolean; - import static com.google.common.base.Preconditions.checkArgument; - -import org.apache.beam.sdk.util.CloudObject; -import org.apache.beam.sdk.util.PropertyNames; -import org.apache.beam.sdk.util.common.ElementByteSizeObserver; -import org.apache.beam.sdk.values.KV; +import static org.apache.beam.sdk.util.Structs.addBoolean; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; - import java.io.IOException; import java.io.InputStream; import java.io.OutputStream; import java.util.Arrays; import java.util.List; +import org.apache.beam.sdk.util.CloudObject; +import org.apache.beam.sdk.util.PropertyNames; +import org.apache.beam.sdk.util.common.ElementByteSizeObserver; +import org.apache.beam.sdk.values.KV; /** * A {@code KvCoder} encodes {@link KV}s. diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/ListCoder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/ListCoder.java index f8437a4c929a8..78785017d49cd 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/ListCoder.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/ListCoder.java @@ -19,12 +19,10 @@ import static com.google.common.base.Preconditions.checkArgument; -import org.apache.beam.sdk.util.PropertyNames; - import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; - import java.util.List; +import org.apache.beam.sdk.util.PropertyNames; /** * A {@link Coder} for {@link List}, using the format of {@link IterableLikeCoder}. diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/MapCoder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/MapCoder.java index 044fff1e413aa..ebe705156ffbd 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/MapCoder.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/MapCoder.java @@ -19,14 +19,9 @@ import static com.google.common.base.Preconditions.checkArgument; -import org.apache.beam.sdk.util.PropertyNames; -import org.apache.beam.sdk.util.common.ElementByteSizeObserver; - -import com.google.common.collect.Maps; - import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; - +import com.google.common.collect.Maps; import java.io.DataInputStream; import java.io.DataOutputStream; import java.io.IOException; @@ -36,6 +31,8 @@ import java.util.List; import java.util.Map; import java.util.Map.Entry; +import org.apache.beam.sdk.util.PropertyNames; +import org.apache.beam.sdk.util.common.ElementByteSizeObserver; /** * A {@link Coder} for {@link Map Maps} that encodes them according to provided diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/NullableCoder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/NullableCoder.java index 4001b811db188..44aadbdd88ccb 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/NullableCoder.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/NullableCoder.java @@ -19,21 +19,17 @@ import static com.google.common.base.Preconditions.checkArgument; -import org.apache.beam.sdk.util.PropertyNames; -import org.apache.beam.sdk.util.common.ElementByteSizeObserver; - -import com.google.common.base.Optional; -import com.google.common.collect.ImmutableList; - import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; - +import com.google.common.base.Optional; +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.util.PropertyNames; +import org.apache.beam.sdk.util.common.ElementByteSizeObserver; /** * A {@link NullableCoder} encodes nullable values of type {@code T} using a nested diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/SerializableCoder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/SerializableCoder.java index 0995bdc976e06..46777b9d835ef 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/SerializableCoder.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/SerializableCoder.java @@ -17,12 +17,8 @@ */ package org.apache.beam.sdk.coders; -import org.apache.beam.sdk.util.CloudObject; -import org.apache.beam.sdk.values.TypeDescriptor; - import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; - import java.io.IOException; import java.io.InputStream; import java.io.ObjectInputStream; @@ -30,6 +26,8 @@ import java.io.ObjectStreamClass; import java.io.OutputStream; import java.io.Serializable; +import org.apache.beam.sdk.util.CloudObject; +import org.apache.beam.sdk.values.TypeDescriptor; /** * A {@link Coder} for Java classes that implement {@link Serializable}. diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/SetCoder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/SetCoder.java index fcbb3797a8a29..0d1b0176ae059 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/SetCoder.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/SetCoder.java @@ -19,14 +19,12 @@ import static com.google.common.base.Preconditions.checkArgument; -import org.apache.beam.sdk.util.PropertyNames; - import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; - import java.util.HashSet; import java.util.List; import java.util.Set; +import org.apache.beam.sdk.util.PropertyNames; /** * A {@link SetCoder} encodes any {@link Set} using the format of {@link IterableLikeCoder}. The diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/StandardCoder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/StandardCoder.java index 8b7b78890d4c0..0e57ed22ea370 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/StandardCoder.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/StandardCoder.java @@ -17,26 +17,23 @@ */ package org.apache.beam.sdk.coders; +import static com.google.common.base.Preconditions.checkNotNull; import static org.apache.beam.sdk.util.Structs.addList; import static org.apache.beam.sdk.util.Structs.addString; import static org.apache.beam.sdk.util.Structs.addStringList; -import static com.google.common.base.Preconditions.checkNotNull; - -import org.apache.beam.sdk.util.CloudObject; -import org.apache.beam.sdk.util.PropertyNames; -import org.apache.beam.sdk.util.common.ElementByteSizeObserver; - import com.google.common.collect.Lists; import com.google.common.io.ByteStreams; import com.google.common.io.CountingOutputStream; - import java.io.ByteArrayOutputStream; import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; import java.util.Collections; import java.util.List; +import org.apache.beam.sdk.util.CloudObject; +import org.apache.beam.sdk.util.PropertyNames; +import org.apache.beam.sdk.util.common.ElementByteSizeObserver; /** * An abstract base class to implement a {@link Coder} that defines equality, hashing, and printing diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/StringDelegateCoder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/StringDelegateCoder.java index c498a8aa27150..80bcae30b386a 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/StringDelegateCoder.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/StringDelegateCoder.java @@ -17,14 +17,13 @@ */ package org.apache.beam.sdk.coders; -import org.apache.beam.sdk.coders.DelegateCoder.CodingFunction; -import org.apache.beam.sdk.coders.protobuf.ProtoCoder; - import java.io.IOException; import java.io.InputStream; import java.io.OutputStream; import java.lang.reflect.InvocationTargetException; import java.util.Collection; +import org.apache.beam.sdk.coders.DelegateCoder.CodingFunction; +import org.apache.beam.sdk.coders.protobuf.ProtoCoder; /** * A {@link Coder} that wraps a {@code Coder} diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/StringUtf8Coder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/StringUtf8Coder.java index 807c00190cdb2..e01dfd8a12193 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/StringUtf8Coder.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/StringUtf8Coder.java @@ -17,16 +17,10 @@ */ package org.apache.beam.sdk.coders; -import org.apache.beam.sdk.util.ExposedByteArrayOutputStream; -import org.apache.beam.sdk.util.StreamUtils; -import org.apache.beam.sdk.util.VarInt; - +import com.fasterxml.jackson.annotation.JsonCreator; import com.google.common.base.Utf8; import com.google.common.io.ByteStreams; import com.google.common.io.CountingOutputStream; - -import com.fasterxml.jackson.annotation.JsonCreator; - import java.io.DataInputStream; import java.io.DataOutputStream; import java.io.EOFException; @@ -35,6 +29,9 @@ import java.io.OutputStream; import java.io.UTFDataFormatException; import java.nio.charset.StandardCharsets; +import org.apache.beam.sdk.util.ExposedByteArrayOutputStream; +import org.apache.beam.sdk.util.StreamUtils; +import org.apache.beam.sdk.util.VarInt; /** * A {@link Coder} that encodes {@link String Strings} in UTF-8 encoding. diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/TableRowJsonCoder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/TableRowJsonCoder.java index 42a6dc96cdd9d..a2562f28baf6f 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/TableRowJsonCoder.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/TableRowJsonCoder.java @@ -17,12 +17,10 @@ */ package org.apache.beam.sdk.coders; -import com.google.api.services.bigquery.model.TableRow; - import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.databind.SerializationFeature; - +import com.google.api.services.bigquery.model.TableRow; import java.io.IOException; import java.io.InputStream; import java.io.OutputStream; diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/TextualIntegerCoder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/TextualIntegerCoder.java index 156160eb0e7e9..6258b2101c400 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/TextualIntegerCoder.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/TextualIntegerCoder.java @@ -18,7 +18,6 @@ package org.apache.beam.sdk.coders; import com.fasterxml.jackson.annotation.JsonCreator; - import java.io.IOException; import java.io.InputStream; import java.io.OutputStream; diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/VarIntCoder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/VarIntCoder.java index 20ce7d4c8cadf..baf3be88af59a 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/VarIntCoder.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/VarIntCoder.java @@ -17,15 +17,13 @@ */ package org.apache.beam.sdk.coders; -import org.apache.beam.sdk.util.VarInt; - import com.fasterxml.jackson.annotation.JsonCreator; - import java.io.EOFException; import java.io.IOException; import java.io.InputStream; import java.io.OutputStream; import java.io.UTFDataFormatException; +import org.apache.beam.sdk.util.VarInt; /** * A {@link Coder} that encodes {@link Integer Integers} using between 1 and 5 bytes. Negative diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/VarLongCoder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/VarLongCoder.java index f1a5ea0784119..ee3c501966fe6 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/VarLongCoder.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/VarLongCoder.java @@ -17,15 +17,13 @@ */ package org.apache.beam.sdk.coders; -import org.apache.beam.sdk.util.VarInt; - import com.fasterxml.jackson.annotation.JsonCreator; - import java.io.EOFException; import java.io.IOException; import java.io.InputStream; import java.io.OutputStream; import java.io.UTFDataFormatException; +import org.apache.beam.sdk.util.VarInt; /** * A {@link Coder} that encodes {@link Long Longs} using between 1 and 10 bytes. Negative diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/VoidCoder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/VoidCoder.java index d67d82fbb6a0b..6bd8a0536ebe3 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/VoidCoder.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/VoidCoder.java @@ -18,7 +18,6 @@ package org.apache.beam.sdk.coders; import com.fasterxml.jackson.annotation.JsonCreator; - import java.io.InputStream; import java.io.OutputStream; diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/protobuf/ProtoCoder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/protobuf/ProtoCoder.java index 67cec884b8b69..79fb37305ea62 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/protobuf/ProtoCoder.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/protobuf/ProtoCoder.java @@ -19,27 +19,14 @@ import static com.google.common.base.Preconditions.checkArgument; -import org.apache.beam.sdk.coders.AtomicCoder; -import org.apache.beam.sdk.coders.CannotProvideCoderException; -import org.apache.beam.sdk.coders.Coder; -import org.apache.beam.sdk.coders.CoderException; -import org.apache.beam.sdk.coders.CoderProvider; -import org.apache.beam.sdk.coders.CoderRegistry; -import org.apache.beam.sdk.util.CloudObject; -import org.apache.beam.sdk.util.Structs; -import org.apache.beam.sdk.values.PCollection; -import org.apache.beam.sdk.values.TypeDescriptor; - +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.collect.ImmutableSet; import com.google.common.collect.Lists; import com.google.common.collect.Sets; import com.google.protobuf.ExtensionRegistry; import com.google.protobuf.Message; import com.google.protobuf.Parser; - -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonProperty; - import java.io.IOException; import java.io.InputStream; import java.io.OutputStream; @@ -52,8 +39,17 @@ import java.util.Set; import java.util.SortedSet; import java.util.TreeSet; - import javax.annotation.Nullable; +import org.apache.beam.sdk.coders.AtomicCoder; +import org.apache.beam.sdk.coders.CannotProvideCoderException; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.coders.CoderException; +import org.apache.beam.sdk.coders.CoderProvider; +import org.apache.beam.sdk.coders.CoderRegistry; +import org.apache.beam.sdk.util.CloudObject; +import org.apache.beam.sdk.util.Structs; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.TypeDescriptor; /** * A {@link Coder} using Google Protocol Buffers binary format. {@link ProtoCoder} supports both diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/protobuf/ProtobufUtil.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/protobuf/ProtobufUtil.java index b03b748bfa18e..77afb47db49c0 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/protobuf/ProtobufUtil.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/protobuf/ProtobufUtil.java @@ -19,8 +19,6 @@ import static com.google.common.base.Preconditions.checkArgument; -import org.apache.beam.sdk.coders.Coder.NonDeterministicException; - import com.google.protobuf.Descriptors.Descriptor; import com.google.protobuf.Descriptors.FieldDescriptor; import com.google.protobuf.Descriptors.FileDescriptor.Syntax; @@ -28,10 +26,10 @@ import com.google.protobuf.ExtensionRegistry; import com.google.protobuf.ExtensionRegistry.ExtensionInfo; import com.google.protobuf.Message; - import java.lang.reflect.InvocationTargetException; import java.util.HashSet; import java.util.Set; +import org.apache.beam.sdk.coders.Coder.NonDeterministicException; /** * Utility functions for reflecting and analyzing Protocol Buffers classes. 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 28c283d97dafa..e7c302bed1d70 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 @@ -20,6 +20,16 @@ import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.base.Preconditions.checkState; +import com.google.common.annotations.VisibleForTesting; +import java.io.IOException; +import java.nio.channels.Channels; +import java.nio.channels.WritableByteChannel; +import java.util.regex.Pattern; +import javax.annotation.Nullable; +import org.apache.avro.Schema; +import org.apache.avro.file.DataFileWriter; +import org.apache.avro.generic.GenericRecord; +import org.apache.avro.reflect.ReflectData; import org.apache.beam.sdk.coders.AvroCoder; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.VoidCoder; @@ -34,20 +44,6 @@ import org.apache.beam.sdk.values.PDone; import org.apache.beam.sdk.values.PInput; -import com.google.common.annotations.VisibleForTesting; - -import org.apache.avro.Schema; -import org.apache.avro.file.DataFileWriter; -import org.apache.avro.generic.GenericRecord; -import org.apache.avro.reflect.ReflectData; - -import java.io.IOException; -import java.nio.channels.Channels; -import java.nio.channels.WritableByteChannel; -import java.util.regex.Pattern; - -import javax.annotation.Nullable; - /** * {@link PTransform}s for reading and writing Avro files. * diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/AvroSource.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/AvroSource.java index d1e0c4dc5e39e..6ef02aa2d5011 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/AvroSource.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/AvroSource.java @@ -21,29 +21,6 @@ import static com.google.common.base.Preconditions.checkNotNull; import static com.google.common.base.Preconditions.checkState; -import org.apache.beam.sdk.annotations.Experimental; -import org.apache.beam.sdk.coders.AvroCoder; -import org.apache.beam.sdk.options.PipelineOptions; -import org.apache.beam.sdk.runners.PipelineRunner; -import org.apache.beam.sdk.util.AvroUtils; -import org.apache.beam.sdk.util.AvroUtils.AvroMetadata; -import org.apache.beam.sdk.values.PCollection; - -import org.apache.avro.Schema; -import org.apache.avro.file.CodecFactory; -import org.apache.avro.file.DataFileConstants; -import org.apache.avro.generic.GenericDatumReader; -import org.apache.avro.generic.GenericRecord; -import org.apache.avro.io.BinaryDecoder; -import org.apache.avro.io.DatumReader; -import org.apache.avro.io.DecoderFactory; -import org.apache.avro.reflect.ReflectData; -import org.apache.avro.reflect.ReflectDatumReader; -import org.apache.commons.compress.compressors.bzip2.BZip2CompressorInputStream; -import org.apache.commons.compress.compressors.snappy.SnappyCompressorInputStream; -import org.apache.commons.compress.compressors.xz.XZCompressorInputStream; -import org.apache.commons.compress.utils.CountingInputStream; - import java.io.ByteArrayInputStream; import java.io.EOFException; import java.io.IOException; @@ -57,8 +34,28 @@ import java.util.Collection; import java.util.zip.Inflater; import java.util.zip.InflaterInputStream; - import javax.annotation.concurrent.GuardedBy; +import org.apache.avro.Schema; +import org.apache.avro.file.CodecFactory; +import org.apache.avro.file.DataFileConstants; +import org.apache.avro.generic.GenericDatumReader; +import org.apache.avro.generic.GenericRecord; +import org.apache.avro.io.BinaryDecoder; +import org.apache.avro.io.DatumReader; +import org.apache.avro.io.DecoderFactory; +import org.apache.avro.reflect.ReflectData; +import org.apache.avro.reflect.ReflectDatumReader; +import org.apache.beam.sdk.annotations.Experimental; +import org.apache.beam.sdk.coders.AvroCoder; +import org.apache.beam.sdk.options.PipelineOptions; +import org.apache.beam.sdk.runners.PipelineRunner; +import org.apache.beam.sdk.util.AvroUtils; +import org.apache.beam.sdk.util.AvroUtils.AvroMetadata; +import org.apache.beam.sdk.values.PCollection; +import org.apache.commons.compress.compressors.bzip2.BZip2CompressorInputStream; +import org.apache.commons.compress.compressors.snappy.SnappyCompressorInputStream; +import org.apache.commons.compress.compressors.xz.XZCompressorInputStream; +import org.apache.commons.compress.utils.CountingInputStream; // CHECKSTYLE.OFF: JavadocStyle /** diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/BlockBasedSource.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/BlockBasedSource.java index 997c77a1273f6..83336ff739132 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/BlockBasedSource.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/BlockBasedSource.java @@ -17,13 +17,11 @@ */ package org.apache.beam.sdk.io; -import org.apache.beam.sdk.annotations.Experimental; -import org.apache.beam.sdk.options.PipelineOptions; - import java.io.IOException; import java.util.NoSuchElementException; - import javax.annotation.Nullable; +import org.apache.beam.sdk.annotations.Experimental; +import org.apache.beam.sdk.options.PipelineOptions; /** * A {@code BlockBasedSource} is a {@link FileBasedSource} where a file consists of blocks of 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 cfdd5814e2735..ede65a96cc998 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 @@ -19,6 +19,13 @@ import static org.apache.beam.sdk.util.StringUtils.approximateSimpleName; +import com.google.api.client.util.BackOff; +import com.google.common.util.concurrent.Uninterruptibles; +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.NoSuchElementException; +import java.util.concurrent.TimeUnit; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.options.PipelineOptions; @@ -31,19 +38,9 @@ import org.apache.beam.sdk.util.ValueWithRecordId; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PInput; - -import com.google.api.client.util.BackOff; -import com.google.common.util.concurrent.Uninterruptibles; - import org.joda.time.Duration; import org.joda.time.Instant; -import java.io.IOException; -import java.util.ArrayList; -import java.util.List; -import java.util.NoSuchElementException; -import java.util.concurrent.TimeUnit; - /** * {@link PTransform} that reads a bounded amount of data from an {@link UnboundedSource}, diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/BoundedSource.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/BoundedSource.java index 394afa4bb3c8c..5fd7b8adc948d 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/BoundedSource.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/BoundedSource.java @@ -17,20 +17,17 @@ */ package org.apache.beam.sdk.io; +import java.io.IOException; +import java.util.List; +import java.util.NoSuchElementException; +import javax.annotation.Nullable; import org.apache.beam.sdk.annotations.Experimental; import org.apache.beam.sdk.io.range.OffsetRangeTracker; import org.apache.beam.sdk.io.range.RangeTracker; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; - import org.joda.time.Instant; -import java.io.IOException; -import java.util.List; -import java.util.NoSuchElementException; - -import javax.annotation.Nullable; - /** * A {@link Source} that reads a finite amount of input and, because of that, supports * some additional operations. diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/CompressedSource.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/CompressedSource.java index 11ff90f66af9e..3cd097ca507bf 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/CompressedSource.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/CompressedSource.java @@ -20,17 +20,8 @@ import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.base.Preconditions.checkNotNull; -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.display.DisplayData; - import com.google.common.io.ByteStreams; import com.google.common.primitives.Ints; - -import org.apache.commons.compress.compressors.bzip2.BZip2CompressorInputStream; -import org.apache.commons.compress.compressors.gzip.GzipCompressorInputStream; - import java.io.IOException; import java.io.InputStream; import java.io.PushbackInputStream; @@ -42,8 +33,13 @@ import java.util.zip.GZIPInputStream; import java.util.zip.ZipEntry; import java.util.zip.ZipInputStream; - import javax.annotation.concurrent.GuardedBy; +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.display.DisplayData; +import org.apache.commons.compress.compressors.bzip2.BZip2CompressorInputStream; +import org.apache.commons.compress.compressors.gzip.GzipCompressorInputStream; /** * A Source that reads from compressed files. A {@code CompressedSources} wraps a delegate 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 a21bc2418fa32..f479215296442 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 @@ -20,6 +20,7 @@ import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.base.Preconditions.checkNotNull; +import com.google.common.base.Optional; import org.apache.beam.sdk.io.CountingSource.NowTimestampFn; import org.apache.beam.sdk.io.Read.Unbounded; import org.apache.beam.sdk.transforms.PTransform; @@ -28,9 +29,6 @@ import org.apache.beam.sdk.values.PBegin; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollection.IsBounded; - -import com.google.common.base.Optional; - import org.joda.time.Duration; import org.joda.time.Instant; 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 403d22eba3195..59a8df8ac9d9f 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 @@ -20,6 +20,10 @@ import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.base.Preconditions.checkNotNull; +import com.google.common.collect.ImmutableList; +import java.io.IOException; +import java.util.List; +import java.util.NoSuchElementException; import org.apache.beam.sdk.coders.AvroCoder; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.DefaultCoder; @@ -29,16 +33,9 @@ import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.SerializableFunction; import org.apache.beam.sdk.values.PCollection; - -import com.google.common.collect.ImmutableList; - import org.joda.time.Duration; import org.joda.time.Instant; -import java.io.IOException; -import java.util.List; -import java.util.NoSuchElementException; - /** * A source that produces longs. When used as a {@link BoundedSource}, {@link CountingSource} * starts at {@code 0} and counts up to a specified maximum. When used as an 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 761d49c9f23b4..ea95f2f5629c1 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 @@ -21,23 +21,7 @@ import static com.google.common.base.Preconditions.checkNotNull; import static com.google.common.base.Preconditions.checkState; -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.transforms.display.DisplayData; -import org.apache.beam.sdk.util.FileIOChannelFactory; -import org.apache.beam.sdk.util.GcsIOChannelFactory; -import org.apache.beam.sdk.util.GcsUtil; -import org.apache.beam.sdk.util.GcsUtil.GcsUtilFactory; -import org.apache.beam.sdk.util.IOChannelFactory; -import org.apache.beam.sdk.util.IOChannelUtils; -import org.apache.beam.sdk.util.MimeTypes; - import com.google.common.collect.Ordering; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - import java.io.File; import java.io.IOException; import java.io.Serializable; @@ -50,6 +34,19 @@ import java.util.Collection; import java.util.HashSet; import java.util.List; +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.transforms.display.DisplayData; +import org.apache.beam.sdk.util.FileIOChannelFactory; +import org.apache.beam.sdk.util.GcsIOChannelFactory; +import org.apache.beam.sdk.util.GcsUtil; +import org.apache.beam.sdk.util.GcsUtil.GcsUtilFactory; +import org.apache.beam.sdk.util.IOChannelFactory; +import org.apache.beam.sdk.util.IOChannelUtils; +import org.apache.beam.sdk.util.MimeTypes; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * Abstract {@link Sink} for file-based output. An implementation of FileBasedSink writes file-based 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 f000f6a71eda9..b073236605360 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 @@ -20,22 +20,12 @@ import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.base.Preconditions.checkState; -import org.apache.beam.sdk.options.PipelineOptions; -import org.apache.beam.sdk.transforms.display.DisplayData; -import org.apache.beam.sdk.util.IOChannelFactory; -import org.apache.beam.sdk.util.IOChannelUtils; - import com.google.common.collect.ImmutableList; import com.google.common.collect.Iterables; import com.google.common.util.concurrent.Futures; import com.google.common.util.concurrent.ListenableFuture; import com.google.common.util.concurrent.ListeningExecutorService; import com.google.common.util.concurrent.MoreExecutors; - -import org.joda.time.Instant; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - import java.io.IOException; import java.nio.channels.ReadableByteChannel; import java.nio.channels.SeekableByteChannel; @@ -48,6 +38,13 @@ import java.util.concurrent.Callable; import java.util.concurrent.ExecutionException; import java.util.concurrent.Executors; +import org.apache.beam.sdk.options.PipelineOptions; +import org.apache.beam.sdk.transforms.display.DisplayData; +import org.apache.beam.sdk.util.IOChannelFactory; +import org.apache.beam.sdk.util.IOChannelUtils; +import org.joda.time.Instant; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * A common base class for all file-based {@link Source}s. Extend this class to implement your own diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/OffsetBasedSource.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/OffsetBasedSource.java index 8cbcd1f868da3..6c685ffc89a0a 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/OffsetBasedSource.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/OffsetBasedSource.java @@ -19,19 +19,17 @@ import static com.google.common.base.Preconditions.checkArgument; +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.NoSuchElementException; import org.apache.beam.sdk.io.range.OffsetRangeTracker; import org.apache.beam.sdk.io.range.RangeTracker; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.display.DisplayData; - import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.io.IOException; -import java.util.ArrayList; -import java.util.List; -import java.util.NoSuchElementException; - /** * A {@link BoundedSource} that uses offsets to define starting and ending positions. * 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 2b2717574e551..b137f1551010a 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 @@ -19,6 +19,14 @@ import static com.google.common.base.Preconditions.checkState; +import com.google.common.base.Strings; +import java.io.IOException; +import java.io.Serializable; +import java.util.ArrayList; +import java.util.List; +import java.util.regex.Matcher; +import java.util.regex.Pattern; +import javax.annotation.Nullable; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.StringUtf8Coder; import org.apache.beam.sdk.coders.VoidCoder; @@ -41,23 +49,11 @@ import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PDone; import org.apache.beam.sdk.values.PInput; - -import com.google.common.base.Strings; - import org.joda.time.Duration; import org.joda.time.Instant; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.io.IOException; -import java.io.Serializable; -import java.util.ArrayList; -import java.util.List; -import java.util.regex.Matcher; -import java.util.regex.Pattern; - -import javax.annotation.Nullable; - /** * Read and Write {@link PTransform}s for Cloud Pub/Sub streams. These transforms create * and consume unbounded {@link PCollection PCollections}. 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 301475198b02f..179abf6da4f6d 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 @@ -20,6 +20,16 @@ import static com.google.common.base.Preconditions.checkState; +import com.google.common.annotations.VisibleForTesting; +import com.google.common.hash.Hashing; +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.util.ArrayList; +import java.util.List; +import java.util.UUID; +import java.util.concurrent.ThreadLocalRandom; +import javax.annotation.Nullable; import org.apache.beam.sdk.coders.AtomicCoder; import org.apache.beam.sdk.coders.BigEndianLongCoder; import org.apache.beam.sdk.coders.ByteArrayCoder; @@ -52,21 +62,8 @@ import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PDone; - -import com.google.common.annotations.VisibleForTesting; -import com.google.common.hash.Hashing; - import org.joda.time.Duration; -import java.io.IOException; -import java.io.InputStream; -import java.io.OutputStream; -import java.util.ArrayList; -import java.util.List; -import java.util.UUID; -import java.util.concurrent.ThreadLocalRandom; -import javax.annotation.Nullable; - /** * A PTransform which streams messages to Pubsub. *
      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 f99b47142425a..36f154f9c9874 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 @@ -22,6 +22,28 @@ import static com.google.common.base.Preconditions.checkNotNull; import static com.google.common.base.Preconditions.checkState; +import com.google.api.client.util.Clock; +import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Charsets; +import com.google.common.collect.Iterables; +import com.google.common.collect.Lists; +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.security.GeneralSecurityException; +import java.util.ArrayDeque; +import java.util.ArrayList; +import java.util.Collection; +import java.util.HashSet; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.NoSuchElementException; +import java.util.Queue; +import java.util.Set; +import java.util.concurrent.ConcurrentLinkedQueue; +import java.util.concurrent.atomic.AtomicInteger; +import javax.annotation.Nullable; import org.apache.beam.sdk.coders.AtomicCoder; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.CoderException; @@ -49,36 +71,11 @@ import org.apache.beam.sdk.util.PubsubClient.TopicPath; import org.apache.beam.sdk.values.PBegin; import org.apache.beam.sdk.values.PCollection; - -import com.google.api.client.util.Clock; -import com.google.common.annotations.VisibleForTesting; -import com.google.common.base.Charsets; -import com.google.common.collect.Iterables; -import com.google.common.collect.Lists; - import org.joda.time.Duration; import org.joda.time.Instant; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.io.IOException; -import java.io.InputStream; -import java.io.OutputStream; -import java.security.GeneralSecurityException; -import java.util.ArrayDeque; -import java.util.ArrayList; -import java.util.Collection; -import java.util.HashSet; -import java.util.LinkedHashMap; -import java.util.List; -import java.util.Map; -import java.util.NoSuchElementException; -import java.util.Queue; -import java.util.Set; -import java.util.concurrent.ConcurrentLinkedQueue; -import java.util.concurrent.atomic.AtomicInteger; -import javax.annotation.Nullable; - /** * A PTransform which streams messages from Pubsub. *
        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 e13ff06dc39db..f99877d84d29a 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 @@ -19,6 +19,7 @@ import static org.apache.beam.sdk.util.StringUtils.approximateSimpleName; +import javax.annotation.Nullable; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.display.DisplayData; @@ -27,11 +28,8 @@ import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollection.IsBounded; import org.apache.beam.sdk.values.PInput; - import org.joda.time.Duration; -import javax.annotation.Nullable; - /** * A {@link PTransform} for reading from a {@link Source}. * diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/Sink.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/Sink.java index 20b1631306d53..1abcc3d1166d5 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/Sink.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/Sink.java @@ -17,6 +17,7 @@ */ package org.apache.beam.sdk.io; +import java.io.Serializable; import org.apache.beam.sdk.annotations.Experimental; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.options.PipelineOptions; @@ -24,8 +25,6 @@ import org.apache.beam.sdk.transforms.display.HasDisplayData; import org.apache.beam.sdk.values.PCollection; -import java.io.Serializable; - /** * A {@code Sink} represents a resource that can be written to using the {@link Write} transform. * diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/Source.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/Source.java index de000351538b7..542d91ca791ea 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/Source.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/Source.java @@ -17,17 +17,15 @@ */ package org.apache.beam.sdk.io; +import java.io.IOException; +import java.io.Serializable; +import java.util.NoSuchElementException; import org.apache.beam.sdk.annotations.Experimental; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.transforms.display.DisplayData; import org.apache.beam.sdk.transforms.display.HasDisplayData; - import org.joda.time.Instant; -import java.io.IOException; -import java.io.Serializable; -import java.util.NoSuchElementException; - /** * Base class for defining input formats and creating a {@code Source} for reading the input. * 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 64db3f76311cd..ed9a62790afe7 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 @@ -20,23 +20,8 @@ import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.base.Preconditions.checkState; -import org.apache.beam.sdk.coders.Coder; -import org.apache.beam.sdk.coders.Coder.Context; -import org.apache.beam.sdk.coders.StringUtf8Coder; -import org.apache.beam.sdk.coders.VoidCoder; -import org.apache.beam.sdk.io.Read.Bounded; -import org.apache.beam.sdk.options.PipelineOptions; -import org.apache.beam.sdk.transforms.PTransform; -import org.apache.beam.sdk.transforms.display.DisplayData; -import org.apache.beam.sdk.util.IOChannelUtils; -import org.apache.beam.sdk.util.MimeTypes; -import org.apache.beam.sdk.values.PCollection; -import org.apache.beam.sdk.values.PDone; -import org.apache.beam.sdk.values.PInput; - import com.google.common.annotations.VisibleForTesting; import com.google.protobuf.ByteString; - import java.io.IOException; import java.io.OutputStream; import java.nio.ByteBuffer; @@ -47,8 +32,20 @@ import java.nio.charset.StandardCharsets; import java.util.NoSuchElementException; import java.util.regex.Pattern; - import javax.annotation.Nullable; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.coders.Coder.Context; +import org.apache.beam.sdk.coders.StringUtf8Coder; +import org.apache.beam.sdk.coders.VoidCoder; +import org.apache.beam.sdk.io.Read.Bounded; +import org.apache.beam.sdk.options.PipelineOptions; +import org.apache.beam.sdk.transforms.PTransform; +import org.apache.beam.sdk.transforms.display.DisplayData; +import org.apache.beam.sdk.util.IOChannelUtils; +import org.apache.beam.sdk.util.MimeTypes; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.PDone; +import org.apache.beam.sdk.values.PInput; /** * {@link PTransform}s for reading and writing text files. diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/UnboundedSource.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/UnboundedSource.java index dded8e2493b4c..043f2fcefb19b 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/UnboundedSource.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/UnboundedSource.java @@ -17,17 +17,14 @@ */ package org.apache.beam.sdk.io; -import org.apache.beam.sdk.annotations.Experimental; -import org.apache.beam.sdk.coders.Coder; -import org.apache.beam.sdk.options.PipelineOptions; - -import org.joda.time.Instant; - import java.io.IOException; import java.util.List; import java.util.NoSuchElementException; - import javax.annotation.Nullable; +import org.apache.beam.sdk.annotations.Experimental; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.options.PipelineOptions; +import org.joda.time.Instant; /** * A {@link Source} that reads an unbounded amount of input and, because of that, supports 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 a474528e04e6f..9d0beb769c444 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 @@ -20,6 +20,10 @@ import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.base.Preconditions.checkNotNull; +import com.google.common.collect.Lists; +import java.util.List; +import java.util.UUID; +import java.util.concurrent.ThreadLocalRandom; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.annotations.Experimental; import org.apache.beam.sdk.coders.Coder; @@ -44,16 +48,9 @@ import org.apache.beam.sdk.values.PCollection.IsBounded; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.PDone; - -import com.google.common.collect.Lists; - import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.util.List; -import java.util.UUID; -import java.util.concurrent.ThreadLocalRandom; - /** * A {@link PTransform} that writes to a {@link Sink}. A write begins with a sequential global * initialization of a sink, followed by a parallel write, and ends with a sequential finalization 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 ef50a1c322f58..983eed2468a57 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 @@ -19,6 +19,12 @@ import static com.google.common.base.Preconditions.checkNotNull; +import java.io.OutputStream; +import java.nio.channels.Channels; +import java.nio.channels.WritableByteChannel; +import javax.xml.bind.JAXBContext; +import javax.xml.bind.JAXBException; +import javax.xml.bind.Marshaller; import org.apache.beam.sdk.coders.StringUtf8Coder; import org.apache.beam.sdk.io.FileBasedSink.FileBasedWriteOperation; import org.apache.beam.sdk.io.FileBasedSink.FileBasedWriter; @@ -27,14 +33,6 @@ import org.apache.beam.sdk.util.CoderUtils; import org.apache.beam.sdk.values.PCollection; -import java.io.OutputStream; -import java.nio.channels.Channels; -import java.nio.channels.WritableByteChannel; - -import javax.xml.bind.JAXBContext; -import javax.xml.bind.JAXBException; -import javax.xml.bind.Marshaller; - // CHECKSTYLE.OFF: JavadocStyle /** * A {@link Sink} that outputs records as XML-formatted elements. Writes a {@link PCollection} of diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/XmlSource.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/XmlSource.java index 83443a32107e2..e00857ea2bf82 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/XmlSource.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/XmlSource.java @@ -19,14 +19,6 @@ import static com.google.common.base.Preconditions.checkNotNull; -import org.apache.beam.sdk.coders.Coder; -import org.apache.beam.sdk.coders.JAXBCoder; -import org.apache.beam.sdk.options.PipelineOptions; -import org.apache.beam.sdk.runners.PipelineRunner; -import org.apache.beam.sdk.transforms.display.DisplayData; - -import org.codehaus.stax2.XMLInputFactory2; - import java.io.ByteArrayInputStream; import java.io.ByteArrayOutputStream; import java.io.IOException; @@ -39,7 +31,6 @@ import java.nio.channels.ReadableByteChannel; import java.nio.charset.StandardCharsets; import java.util.NoSuchElementException; - import javax.xml.bind.JAXBContext; import javax.xml.bind.JAXBElement; import javax.xml.bind.JAXBException; @@ -51,6 +42,12 @@ import javax.xml.stream.XMLStreamConstants; import javax.xml.stream.XMLStreamException; import javax.xml.stream.XMLStreamReader; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.coders.JAXBCoder; +import org.apache.beam.sdk.options.PipelineOptions; +import org.apache.beam.sdk.runners.PipelineRunner; +import org.apache.beam.sdk.transforms.display.DisplayData; +import org.codehaus.stax2.XMLInputFactory2; // CHECKSTYLE.OFF: JavadocStyle /** diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/range/ByteKey.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/range/ByteKey.java index f82a11d532cd4..e4129ff56512b 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/range/ByteKey.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/range/ByteKey.java @@ -21,7 +21,6 @@ import com.google.protobuf.ByteString; import com.google.protobuf.ByteString.ByteIterator; - import java.io.Serializable; /** 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 cfd924d3841e8..288124b310ecc 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 @@ -24,16 +24,14 @@ import com.google.common.base.MoreObjects; import com.google.common.collect.ImmutableList; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - import java.io.Serializable; import java.math.BigDecimal; import java.math.BigInteger; import java.util.Arrays; import java.util.List; import java.util.Objects; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * A class representing a range of {@link ByteKey ByteKeys}. 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 24bf6e474b4da..7c0f1c0a6d388 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 @@ -20,13 +20,11 @@ import static com.google.common.base.MoreObjects.toStringHelper; import static com.google.common.base.Preconditions.checkState; +import javax.annotation.Nullable; import org.apache.beam.sdk.io.BoundedSource.BoundedReader; - import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import javax.annotation.Nullable; - /** * A {@link RangeTracker} for {@link ByteKey ByteKeys} in {@link ByteKeyRange ByteKeyRanges}. * diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/range/OffsetRangeTracker.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/range/OffsetRangeTracker.java index a8d00ee1d16bb..51e2b1ac2a1ce 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/range/OffsetRangeTracker.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/range/OffsetRangeTracker.java @@ -19,10 +19,8 @@ import static com.google.common.base.Preconditions.checkState; -import org.apache.beam.sdk.io.BoundedSource.BoundedReader; - import com.google.common.annotations.VisibleForTesting; - +import org.apache.beam.sdk.io.BoundedSource.BoundedReader; import org.slf4j.Logger; import org.slf4j.LoggerFactory; 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 b2df96e7474f2..b4ae31406448c 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,21 +17,12 @@ */ package org.apache.beam.sdk.options; -import org.apache.beam.sdk.util.CredentialFactory; -import org.apache.beam.sdk.util.GcpCredentialFactory; -import org.apache.beam.sdk.util.InstanceBuilder; -import org.apache.beam.sdk.util.PathValidator; - +import com.fasterxml.jackson.annotation.JsonIgnore; import com.google.api.client.auth.oauth2.Credential; import com.google.api.client.googleapis.auth.oauth2.GoogleOAuthConstants; import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Strings; import com.google.common.io.Files; - -import com.fasterxml.jackson.annotation.JsonIgnore; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - import java.io.File; import java.io.IOException; import java.nio.charset.StandardCharsets; @@ -40,8 +31,13 @@ import java.util.Map; import java.util.regex.Matcher; import java.util.regex.Pattern; - import javax.annotation.Nullable; +import org.apache.beam.sdk.util.CredentialFactory; +import org.apache.beam.sdk.util.GcpCredentialFactory; +import org.apache.beam.sdk.util.InstanceBuilder; +import org.apache.beam.sdk.util.PathValidator; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * Options used to configure Google Cloud Platform project and credentials. diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/options/GcsOptions.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/options/GcsOptions.java index 1b3436bf31187..29a29d9f4acda 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/options/GcsOptions.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/options/GcsOptions.java @@ -17,22 +17,19 @@ */ package org.apache.beam.sdk.options; -import org.apache.beam.sdk.util.AppEngineEnvironment; -import org.apache.beam.sdk.util.GcsPathValidator; -import org.apache.beam.sdk.util.GcsUtil; -import org.apache.beam.sdk.util.InstanceBuilder; -import org.apache.beam.sdk.util.PathValidator; - +import com.fasterxml.jackson.annotation.JsonIgnore; import com.google.cloud.hadoop.util.AbstractGoogleAsyncWriteChannel; import com.google.common.util.concurrent.MoreExecutors; import com.google.common.util.concurrent.ThreadFactoryBuilder; - -import com.fasterxml.jackson.annotation.JsonIgnore; - import java.util.concurrent.ExecutorService; import java.util.concurrent.SynchronousQueue; import java.util.concurrent.ThreadPoolExecutor; import java.util.concurrent.TimeUnit; +import org.apache.beam.sdk.util.AppEngineEnvironment; +import org.apache.beam.sdk.util.GcsPathValidator; +import org.apache.beam.sdk.util.GcsUtil; +import org.apache.beam.sdk.util.InstanceBuilder; +import org.apache.beam.sdk.util.PathValidator; /** * Options used to configure Google Cloud Storage. diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/options/GoogleApiDebugOptions.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/options/GoogleApiDebugOptions.java index 39cd40e715f81..4932b16ef49f7 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/options/GoogleApiDebugOptions.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/options/GoogleApiDebugOptions.java @@ -20,7 +20,6 @@ import com.google.api.client.googleapis.services.AbstractGoogleClient; import com.google.api.client.googleapis.services.AbstractGoogleClientRequest; import com.google.api.client.googleapis.services.GoogleClientRequestInitializer; - import java.io.IOException; import java.util.HashMap; import java.util.Map; diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/options/PipelineOptionSpec.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/options/PipelineOptionSpec.java index 9a88f70762328..1220e6bc14e27 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/options/PipelineOptionSpec.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/options/PipelineOptionSpec.java @@ -17,10 +17,8 @@ */ package org.apache.beam.sdk.options; -import com.google.auto.value.AutoValue; - import com.fasterxml.jackson.annotation.JsonIgnore; - +import com.google.auto.value.AutoValue; import java.lang.reflect.Method; /** 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 4595fc871d225..db54d0ac76b67 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 @@ -17,6 +17,14 @@ */ package org.apache.beam.sdk.options; +import com.fasterxml.jackson.annotation.JsonIgnore; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.fasterxml.jackson.databind.annotation.JsonDeserialize; +import com.fasterxml.jackson.databind.annotation.JsonSerialize; +import com.google.auto.service.AutoService; +import java.lang.reflect.Proxy; +import java.util.ServiceLoader; +import javax.annotation.concurrent.ThreadSafe; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.options.GoogleApiDebugOptions.GoogleApiTracer; import org.apache.beam.sdk.options.ProxyInvocationHandler.Deserializer; @@ -26,17 +34,6 @@ import org.apache.beam.sdk.transforms.DoFn.Context; import org.apache.beam.sdk.transforms.display.HasDisplayData; -import com.google.auto.service.AutoService; - -import com.fasterxml.jackson.annotation.JsonIgnore; -import com.fasterxml.jackson.databind.ObjectMapper; -import com.fasterxml.jackson.databind.annotation.JsonDeserialize; -import com.fasterxml.jackson.databind.annotation.JsonSerialize; - -import java.lang.reflect.Proxy; -import java.util.ServiceLoader; -import javax.annotation.concurrent.ThreadSafe; - /** * PipelineOptions are used to configure Pipelines. You can extend {@link PipelineOptions} * to create custom configuration options specific to your {@link Pipeline}, 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 a795fcd024ffc..43927bc716217 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 @@ -20,13 +20,9 @@ import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.base.Preconditions.checkNotNull; -import org.apache.beam.sdk.options.Validation.Required; -import org.apache.beam.sdk.runners.PipelineRunner; -import org.apache.beam.sdk.runners.PipelineRunnerRegistrar; -import org.apache.beam.sdk.transforms.display.DisplayData; -import org.apache.beam.sdk.util.StringUtils; -import org.apache.beam.sdk.util.common.ReflectHelpers; - +import com.fasterxml.jackson.annotation.JsonIgnore; +import com.fasterxml.jackson.databind.JavaType; +import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Function; import com.google.common.base.Joiner; @@ -49,13 +45,6 @@ import com.google.common.collect.SortedSetMultimap; import com.google.common.collect.TreeBasedTable; import com.google.common.collect.TreeMultimap; - -import com.fasterxml.jackson.annotation.JsonIgnore; -import com.fasterxml.jackson.databind.JavaType; -import com.fasterxml.jackson.databind.ObjectMapper; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - import java.beans.BeanInfo; import java.beans.IntrospectionException; import java.beans.Introspector; @@ -81,8 +70,15 @@ import java.util.SortedSet; import java.util.TreeMap; import java.util.TreeSet; - import javax.annotation.Nonnull; +import org.apache.beam.sdk.options.Validation.Required; +import org.apache.beam.sdk.runners.PipelineRunner; +import org.apache.beam.sdk.runners.PipelineRunnerRegistrar; +import org.apache.beam.sdk.transforms.display.DisplayData; +import org.apache.beam.sdk.util.StringUtils; +import org.apache.beam.sdk.util.common.ReflectHelpers; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * Constructs a {@link PipelineOptions} or any derived interface that is composable to any other diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/options/PipelineOptionsReflector.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/options/PipelineOptionsReflector.java index 607bddae6578c..26c65aec890d3 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/options/PipelineOptionsReflector.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/options/PipelineOptionsReflector.java @@ -17,16 +17,14 @@ */ package org.apache.beam.sdk.options; -import org.apache.beam.sdk.util.common.ReflectHelpers; - import com.google.common.collect.HashMultimap; import com.google.common.collect.ImmutableSet; import com.google.common.collect.Multimap; - import java.beans.Introspector; import java.lang.reflect.Method; import java.util.Map; import java.util.Set; +import org.apache.beam.sdk.util.common.ReflectHelpers; /** * Utilities to reflect over {@link PipelineOptions}. diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/options/PipelineOptionsRegistrar.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/options/PipelineOptionsRegistrar.java index 16cf7cd730022..2ddff42f8eb43 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/options/PipelineOptionsRegistrar.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/options/PipelineOptionsRegistrar.java @@ -18,7 +18,6 @@ package org.apache.beam.sdk.options; import com.google.auto.service.AutoService; - import java.util.ServiceLoader; /** diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/options/PipelineOptionsValidator.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/options/PipelineOptionsValidator.java index 70be65fa4acdb..bd54ec39bd74b 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/options/PipelineOptionsValidator.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/options/PipelineOptionsValidator.java @@ -20,17 +20,15 @@ import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.base.Preconditions.checkNotNull; -import org.apache.beam.sdk.options.Validation.Required; -import org.apache.beam.sdk.util.common.ReflectHelpers; - import com.google.common.collect.Collections2; import com.google.common.collect.Ordering; import com.google.common.collect.SortedSetMultimap; import com.google.common.collect.TreeMultimap; - import java.lang.reflect.Method; import java.lang.reflect.Proxy; import java.util.Collection; +import org.apache.beam.sdk.options.Validation.Required; +import org.apache.beam.sdk.util.common.ReflectHelpers; /** * Validates that the {@link PipelineOptions} conforms to all the {@link Validation} criteria. 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 fe67f1685da3c..204ad97e7184b 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 @@ -20,25 +20,6 @@ import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.base.Preconditions.checkNotNull; -import org.apache.beam.sdk.options.PipelineOptionsFactory.JsonIgnorePredicate; -import org.apache.beam.sdk.options.PipelineOptionsFactory.Registration; -import org.apache.beam.sdk.transforms.display.DisplayData; -import org.apache.beam.sdk.transforms.display.HasDisplayData; -import org.apache.beam.sdk.util.InstanceBuilder; -import org.apache.beam.sdk.util.common.ReflectHelpers; - -import com.google.auto.value.AutoValue; -import com.google.common.base.Defaults; -import com.google.common.base.Function; -import com.google.common.collect.ClassToInstanceMap; -import com.google.common.collect.FluentIterable; -import com.google.common.collect.HashMultimap; -import com.google.common.collect.ImmutableMap; -import com.google.common.collect.Lists; -import com.google.common.collect.Maps; -import com.google.common.collect.Multimap; -import com.google.common.collect.MutableClassToInstanceMap; - import com.fasterxml.jackson.annotation.JsonIgnore; import com.fasterxml.jackson.core.JsonGenerator; import com.fasterxml.jackson.core.JsonParser; @@ -51,7 +32,17 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.databind.SerializerProvider; import com.fasterxml.jackson.databind.node.ObjectNode; - +import com.google.auto.value.AutoValue; +import com.google.common.base.Defaults; +import com.google.common.base.Function; +import com.google.common.collect.ClassToInstanceMap; +import com.google.common.collect.FluentIterable; +import com.google.common.collect.HashMultimap; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; +import com.google.common.collect.Multimap; +import com.google.common.collect.MutableClassToInstanceMap; import java.beans.PropertyDescriptor; import java.io.IOException; import java.lang.annotation.Annotation; @@ -70,9 +61,14 @@ import java.util.SortedMap; import java.util.TreeMap; import java.util.concurrent.ThreadLocalRandom; - import javax.annotation.Nullable; import javax.annotation.concurrent.ThreadSafe; +import org.apache.beam.sdk.options.PipelineOptionsFactory.JsonIgnorePredicate; +import org.apache.beam.sdk.options.PipelineOptionsFactory.Registration; +import org.apache.beam.sdk.transforms.display.DisplayData; +import org.apache.beam.sdk.transforms.display.HasDisplayData; +import org.apache.beam.sdk.util.InstanceBuilder; +import org.apache.beam.sdk.util.common.ReflectHelpers; /** * Represents and {@link InvocationHandler} for a {@link Proxy}. The invocation handler uses bean diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/PipelineRunnerRegistrar.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/PipelineRunnerRegistrar.java index 949f5daf9ec47..d72c44f9a37db 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/PipelineRunnerRegistrar.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/runners/PipelineRunnerRegistrar.java @@ -18,7 +18,6 @@ package org.apache.beam.sdk.runners; import com.google.auto.service.AutoService; - import java.util.ServiceLoader; /** 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 6b93bbd8a2ad7..0a4bb08a315de 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 @@ -19,16 +19,15 @@ import static com.google.common.base.Preconditions.checkState; -import org.apache.beam.sdk.Pipeline; -import org.apache.beam.sdk.values.PInput; -import org.apache.beam.sdk.values.POutput; -import org.apache.beam.sdk.values.PValue; - import java.util.Deque; import java.util.HashMap; import java.util.LinkedList; import java.util.Map; import java.util.Set; +import org.apache.beam.sdk.Pipeline; +import org.apache.beam.sdk.values.PInput; +import org.apache.beam.sdk.values.POutput; +import org.apache.beam.sdk.values.PValue; /** * Captures information about a collection of transformations and their 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 b5a250b57f1f3..d16b8281dc66c 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 @@ -20,21 +20,19 @@ import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.base.Preconditions.checkState; -import org.apache.beam.sdk.Pipeline.PipelineVisitor; -import org.apache.beam.sdk.Pipeline.PipelineVisitor.CompositeBehavior; -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; - import java.util.ArrayList; import java.util.Collection; import java.util.Collections; import java.util.HashMap; 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.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 diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/CoderProperties.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/CoderProperties.java index e56f01ff7a3e0..910b9395aae85 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/CoderProperties.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/CoderProperties.java @@ -26,24 +26,12 @@ import static org.junit.Assert.assertThat; import static org.junit.Assert.fail; -import org.apache.beam.sdk.coders.Coder; -import org.apache.beam.sdk.coders.Coder.NonDeterministicException; -import org.apache.beam.sdk.coders.CoderException; -import org.apache.beam.sdk.util.CoderUtils; -import org.apache.beam.sdk.util.PropertyNames; -import org.apache.beam.sdk.util.SerializableUtils; -import org.apache.beam.sdk.util.Serializer; -import org.apache.beam.sdk.util.Structs; -import org.apache.beam.sdk.util.UnownedInputStream; -import org.apache.beam.sdk.util.UnownedOutputStream; -import org.apache.beam.sdk.util.common.ElementByteSizeObserver; import com.google.common.annotations.VisibleForTesting; import com.google.common.collect.ImmutableList; import com.google.common.collect.Iterables; import com.google.common.io.ByteStreams; import com.google.common.io.CountingInputStream; import com.google.common.io.CountingOutputStream; - import java.io.ByteArrayInputStream; import java.io.ByteArrayOutputStream; import java.io.IOException; @@ -51,6 +39,17 @@ import java.util.Collection; import java.util.Collections; import java.util.List; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.coders.Coder.NonDeterministicException; +import org.apache.beam.sdk.coders.CoderException; +import org.apache.beam.sdk.util.CoderUtils; +import org.apache.beam.sdk.util.PropertyNames; +import org.apache.beam.sdk.util.SerializableUtils; +import org.apache.beam.sdk.util.Serializer; +import org.apache.beam.sdk.util.Structs; +import org.apache.beam.sdk.util.UnownedInputStream; +import org.apache.beam.sdk.util.UnownedOutputStream; +import org.apache.beam.sdk.util.common.ElementByteSizeObserver; /** * Properties for use in {@link Coder} tests. These are implemented with junit assertions 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 303efcb392b8a..de6cea383e8c7 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 @@ -20,20 +20,10 @@ import static com.google.common.base.Preconditions.checkArgument; -import org.apache.beam.sdk.PipelineResult; -import org.apache.beam.sdk.util.IOChannelFactory; -import org.apache.beam.sdk.util.IOChannelUtils; - import com.google.common.base.Strings; import com.google.common.hash.HashCode; import com.google.common.hash.Hashing; import com.google.common.io.CharStreams; - -import org.hamcrest.Description; -import org.hamcrest.TypeSafeMatcher; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - import java.io.IOException; import java.io.Reader; import java.nio.channels.Channels; @@ -41,6 +31,13 @@ import java.util.ArrayList; import java.util.Collection; import java.util.List; +import org.apache.beam.sdk.PipelineResult; +import org.apache.beam.sdk.util.IOChannelFactory; +import org.apache.beam.sdk.util.IOChannelUtils; +import org.hamcrest.Description; +import org.hamcrest.TypeSafeMatcher; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * Matcher to verify file checksum in E2E test. diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/MatcherDeserializer.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/MatcherDeserializer.java index 84984709bc64a..6ca07badac929 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/MatcherDeserializer.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/MatcherDeserializer.java @@ -17,17 +17,14 @@ */ package org.apache.beam.sdk.testing; -import org.apache.beam.sdk.util.SerializableUtils; - -import com.google.api.client.util.Base64; - import com.fasterxml.jackson.core.JsonParser; import com.fasterxml.jackson.core.JsonProcessingException; import com.fasterxml.jackson.databind.DeserializationContext; import com.fasterxml.jackson.databind.JsonDeserializer; import com.fasterxml.jackson.databind.node.ObjectNode; - +import com.google.api.client.util.Base64; import java.io.IOException; +import org.apache.beam.sdk.util.SerializableUtils; /** * MatcherDeserializer is used with Jackson to enable deserialization of SerializableMatchers. diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/MatcherSerializer.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/MatcherSerializer.java index 845248604e2b3..2b4584c8cdaca 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/MatcherSerializer.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/MatcherSerializer.java @@ -17,16 +17,13 @@ */ package org.apache.beam.sdk.testing; -import org.apache.beam.sdk.util.SerializableUtils; - -import com.google.api.client.util.Base64; - import com.fasterxml.jackson.core.JsonGenerator; import com.fasterxml.jackson.core.JsonProcessingException; import com.fasterxml.jackson.databind.JsonSerializer; import com.fasterxml.jackson.databind.SerializerProvider; - +import com.google.api.client.util.Base64; import java.io.IOException; +import org.apache.beam.sdk.util.SerializableUtils; /** * MatcherSerializer is used with Jackson to enable serialization of SerializableMatchers. 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 f02bbe0ed4939..56a1b7c418dae 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 @@ -22,6 +22,16 @@ import static org.hamcrest.Matchers.not; import static org.junit.Assert.assertThat; +import com.google.common.collect.Iterables; +import com.google.common.collect.Lists; +import edu.umd.cs.findbugs.annotations.SuppressFBWarnings; +import java.io.IOException; +import java.io.Serializable; +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.Map; +import java.util.NoSuchElementException; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.CoderException; @@ -63,24 +73,10 @@ import org.apache.beam.sdk.values.PCollectionList; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.PDone; - -import com.google.common.collect.Iterables; -import com.google.common.collect.Lists; - import org.joda.time.Duration; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.io.IOException; -import java.io.Serializable; -import java.util.Arrays; -import java.util.Collection; -import java.util.Collections; -import java.util.Map; -import java.util.NoSuchElementException; - -import edu.umd.cs.findbugs.annotations.SuppressFBWarnings; - /** * An assertion on the contents of a {@link PCollection} incorporated into the pipeline. Such an * assertion can be checked no matter what kind of {@link PipelineRunner} is used. 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 899612b3ef2a1..db72a0cd4c229 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 @@ -19,6 +19,8 @@ import static com.google.common.base.Preconditions.checkState; +import java.util.ArrayList; +import java.util.List; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.SimpleFunction; import org.apache.beam.sdk.transforms.windowing.PaneInfo; @@ -27,9 +29,6 @@ import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.TypeDescriptor; -import java.util.ArrayList; -import java.util.List; - /** * {@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 diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/SerializableMatcher.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/SerializableMatcher.java index a465bbec32aa8..4e4299d13b2bc 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/SerializableMatcher.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/SerializableMatcher.java @@ -19,9 +19,8 @@ import com.fasterxml.jackson.databind.annotation.JsonDeserialize; import com.fasterxml.jackson.databind.annotation.JsonSerialize; -import org.hamcrest.Matcher; - import java.io.Serializable; +import org.hamcrest.Matcher; /** * A {@link Matcher} that is also {@link Serializable}. diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/SerializableMatchers.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/SerializableMatchers.java index 62a42e4176e89..bd44c4801dcbd 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/SerializableMatchers.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/SerializableMatchers.java @@ -17,28 +17,24 @@ */ package org.apache.beam.sdk.testing; +import com.google.common.base.MoreObjects; +import edu.umd.cs.findbugs.annotations.SuppressFBWarnings; +import java.io.Serializable; +import java.util.Arrays; +import java.util.Collection; +import java.util.List; +import javax.annotation.Nullable; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.CoderException; import org.apache.beam.sdk.coders.ListCoder; import org.apache.beam.sdk.util.CoderUtils; import org.apache.beam.sdk.util.UserCodeException; import org.apache.beam.sdk.values.KV; - -import com.google.common.base.MoreObjects; - import org.hamcrest.BaseMatcher; import org.hamcrest.Description; import org.hamcrest.Matcher; import org.hamcrest.Matchers; -import java.io.Serializable; -import java.util.Arrays; -import java.util.Collection; -import java.util.List; -import javax.annotation.Nullable; - -import edu.umd.cs.findbugs.annotations.SuppressFBWarnings; - /** * Static class for building and using {@link SerializableMatcher} instances. * diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/SourceTestUtils.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/SourceTestUtils.java index 9ce9c5e518da8..e38e1af456639 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/SourceTestUtils.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/SourceTestUtils.java @@ -18,7 +18,6 @@ package org.apache.beam.sdk.testing; import static com.google.common.base.Preconditions.checkNotNull; - import static org.hamcrest.Matchers.containsInAnyOrder; import static org.hamcrest.Matchers.equalTo; import static org.junit.Assert.assertEquals; @@ -27,21 +26,7 @@ import static org.junit.Assert.assertThat; import static org.junit.Assert.assertTrue; -import org.apache.beam.sdk.coders.Coder; -import org.apache.beam.sdk.io.BoundedSource; -import org.apache.beam.sdk.io.BoundedSource.BoundedReader; -import org.apache.beam.sdk.io.Source; -import org.apache.beam.sdk.options.PipelineOptions; -import org.apache.beam.sdk.transforms.display.DisplayData; -import org.apache.beam.sdk.values.KV; - import com.google.common.collect.ImmutableList; - -import org.joda.time.Instant; -import org.junit.Assert; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - import java.io.IOException; import java.util.ArrayList; import java.util.List; @@ -52,8 +37,18 @@ import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.Future; - import javax.annotation.Nullable; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.io.BoundedSource; +import org.apache.beam.sdk.io.BoundedSource.BoundedReader; +import org.apache.beam.sdk.io.Source; +import org.apache.beam.sdk.options.PipelineOptions; +import org.apache.beam.sdk.transforms.display.DisplayData; +import org.apache.beam.sdk.values.KV; +import org.joda.time.Instant; +import org.junit.Assert; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * Helper functions and test harnesses for checking correctness of {@link Source} diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/StaticWindows.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/StaticWindows.java index 08d23559bb500..949ecac43c813 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/StaticWindows.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/StaticWindows.java @@ -19,17 +19,15 @@ import static com.google.common.base.Preconditions.checkArgument; -import org.apache.beam.sdk.coders.Coder; -import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.transforms.windowing.NonMergingWindowFn; -import org.apache.beam.sdk.transforms.windowing.WindowFn; - import com.google.common.base.Supplier; import com.google.common.collect.Iterables; - import java.util.Collection; import java.util.Collections; import java.util.Objects; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.transforms.windowing.BoundedWindow; +import org.apache.beam.sdk.transforms.windowing.NonMergingWindowFn; +import org.apache.beam.sdk.transforms.windowing.WindowFn; /** * A {@link WindowFn} that assigns all elements to a static collection of 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 2e64a5e1beb3b..3202000224cd2 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 @@ -17,35 +17,31 @@ */ package org.apache.beam.sdk.testing; -import org.apache.beam.sdk.Pipeline; -import org.apache.beam.sdk.PipelineResult; -import org.apache.beam.sdk.options.ApplicationNameOptions; -import org.apache.beam.sdk.options.GcpOptions; -import org.apache.beam.sdk.options.PipelineOptions; -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.util.IOChannelUtils; -import org.apache.beam.sdk.util.TestCredential; - -import com.google.common.base.Optional; -import com.google.common.base.Strings; -import com.google.common.collect.Iterators; - import com.fasterxml.jackson.core.JsonParser; import com.fasterxml.jackson.core.TreeNode; import com.fasterxml.jackson.databind.JsonNode; import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.databind.node.ObjectNode; -import org.junit.experimental.categories.Category; - +import com.google.common.base.Optional; +import com.google.common.base.Strings; +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.Map.Entry; - import javax.annotation.Nullable; +import org.apache.beam.sdk.Pipeline; +import org.apache.beam.sdk.PipelineResult; +import org.apache.beam.sdk.options.ApplicationNameOptions; +import org.apache.beam.sdk.options.GcpOptions; +import org.apache.beam.sdk.options.PipelineOptions; +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.util.IOChannelUtils; +import org.apache.beam.sdk.util.TestCredential; +import org.junit.experimental.categories.Category; /** * A creator of test pipelines that can be used inside of tests that can be diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/TestPipelineOptions.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/TestPipelineOptions.java index c4596c152c5a3..ff553bafa85e6 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/TestPipelineOptions.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/TestPipelineOptions.java @@ -21,7 +21,6 @@ import org.apache.beam.sdk.options.Default; import org.apache.beam.sdk.options.DefaultValueFactory; import org.apache.beam.sdk.options.PipelineOptions; - import org.hamcrest.BaseMatcher; import org.hamcrest.Description; 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 e2eda32228556..e2730edfe6f4c 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 @@ -21,6 +21,16 @@ 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.auto.value.AutoValue; +import com.google.common.annotations.VisibleForTesting; +import com.google.common.collect.ImmutableList; +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.util.Collections; +import java.util.List; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.DurationCoder; @@ -36,24 +46,10 @@ import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.TimestampedValue; import org.apache.beam.sdk.values.TimestampedValue.TimestampedValueCoder; - -import com.google.auto.value.AutoValue; -import com.google.common.annotations.VisibleForTesting; -import com.google.common.collect.ImmutableList; - -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonProperty; - import org.joda.time.Duration; import org.joda.time.Instant; import org.joda.time.ReadableDuration; -import java.io.IOException; -import java.io.InputStream; -import java.io.OutputStream; -import java.util.Collections; -import java.util.List; - /** * A testing input that generates an unbounded {@link PCollection} of elements, advancing the * watermark and processing time as elements are emitted. After all of the specified elements are diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/WindowFnTestUtils.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/WindowFnTestUtils.java index 127721affcddc..63e7903b8e648 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/WindowFnTestUtils.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/WindowFnTestUtils.java @@ -22,16 +22,6 @@ import static org.junit.Assert.assertThat; import static org.junit.Assert.assertTrue; -import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.transforms.windowing.GlobalWindow; -import org.apache.beam.sdk.transforms.windowing.IntervalWindow; -import org.apache.beam.sdk.transforms.windowing.OutputTimeFn; -import org.apache.beam.sdk.transforms.windowing.OutputTimeFns; -import org.apache.beam.sdk.transforms.windowing.WindowFn; - -import org.joda.time.Instant; -import org.joda.time.ReadableInstant; - import java.util.ArrayList; import java.util.Collection; import java.util.Collections; @@ -41,8 +31,15 @@ import java.util.List; import java.util.Map; import java.util.Set; - import javax.annotation.Nullable; +import org.apache.beam.sdk.transforms.windowing.BoundedWindow; +import org.apache.beam.sdk.transforms.windowing.GlobalWindow; +import org.apache.beam.sdk.transforms.windowing.IntervalWindow; +import org.apache.beam.sdk.transforms.windowing.OutputTimeFn; +import org.apache.beam.sdk.transforms.windowing.OutputTimeFns; +import org.apache.beam.sdk.transforms.windowing.WindowFn; +import org.joda.time.Instant; +import org.joda.time.ReadableInstant; /** * A utility class for testing {@link WindowFn}s. diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/WindowSupplier.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/WindowSupplier.java index 62bc09ffc275b..96091ef87fde6 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/WindowSupplier.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/WindowSupplier.java @@ -17,17 +17,15 @@ */ package org.apache.beam.sdk.testing; -import org.apache.beam.sdk.coders.Coder; -import org.apache.beam.sdk.coders.CoderException; -import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.CoderUtils; - import com.google.common.base.Supplier; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableSet; - import java.io.Serializable; import java.util.Collection; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.coders.CoderException; +import org.apache.beam.sdk.transforms.windowing.BoundedWindow; +import org.apache.beam.sdk.util.CoderUtils; /** * A {@link Supplier} that returns a static set of {@link BoundedWindow BoundedWindows}. The 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 fdbde74e4b13a..d80c116c175c8 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,11 +17,10 @@ */ package org.apache.beam.sdk.transforms; -import org.apache.beam.sdk.values.PInput; -import org.apache.beam.sdk.values.POutput; - import com.google.common.base.MoreObjects; import com.google.common.base.Objects; +import org.apache.beam.sdk.values.PInput; +import org.apache.beam.sdk.values.POutput; /** * Represents the application of a {@link PTransform} to a specific input to produce diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/ApproximateQuantiles.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/ApproximateQuantiles.java index fa87ed89fb0c4..656bd7bbac361 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/ApproximateQuantiles.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/ApproximateQuantiles.java @@ -19,24 +19,9 @@ import static com.google.common.base.Preconditions.checkArgument; -import org.apache.beam.sdk.coders.BigEndianIntegerCoder; -import org.apache.beam.sdk.coders.Coder; -import org.apache.beam.sdk.coders.CoderException; -import org.apache.beam.sdk.coders.CoderRegistry; -import org.apache.beam.sdk.coders.CustomCoder; -import org.apache.beam.sdk.coders.ListCoder; -import org.apache.beam.sdk.transforms.Combine.AccumulatingCombineFn; -import org.apache.beam.sdk.transforms.Combine.AccumulatingCombineFn.Accumulator; -import org.apache.beam.sdk.transforms.display.DisplayData; -import org.apache.beam.sdk.util.WeightedValue; -import org.apache.beam.sdk.util.common.ElementByteSizeObserver; -import org.apache.beam.sdk.values.KV; -import org.apache.beam.sdk.values.PCollection; - import com.google.common.collect.Iterators; import com.google.common.collect.Lists; import com.google.common.collect.UnmodifiableIterator; - import java.io.DataInputStream; import java.io.DataOutputStream; import java.io.IOException; @@ -50,8 +35,20 @@ import java.util.Iterator; import java.util.List; import java.util.PriorityQueue; - import javax.annotation.Nullable; +import org.apache.beam.sdk.coders.BigEndianIntegerCoder; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.coders.CoderException; +import org.apache.beam.sdk.coders.CoderRegistry; +import org.apache.beam.sdk.coders.CustomCoder; +import org.apache.beam.sdk.coders.ListCoder; +import org.apache.beam.sdk.transforms.Combine.AccumulatingCombineFn; +import org.apache.beam.sdk.transforms.Combine.AccumulatingCombineFn.Accumulator; +import org.apache.beam.sdk.transforms.display.DisplayData; +import org.apache.beam.sdk.util.WeightedValue; +import org.apache.beam.sdk.util.common.ElementByteSizeObserver; +import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.PCollection; /** * {@code PTransform}s for getting an idea of a {@code PCollection}'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 2fa24277661cc..71c2158fa42cb 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 @@ -17,6 +17,16 @@ */ package org.apache.beam.sdk.transforms; +import com.google.common.hash.Hashing; +import com.google.common.hash.HashingOutputStream; +import com.google.common.io.ByteStreams; +import java.io.IOException; +import java.io.Serializable; +import java.util.Arrays; +import java.util.Iterator; +import java.util.List; +import java.util.PriorityQueue; +import org.apache.avro.reflect.Nullable; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.Coder.Context; import org.apache.beam.sdk.coders.CoderException; @@ -28,19 +38,6 @@ import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; -import com.google.common.hash.Hashing; -import com.google.common.hash.HashingOutputStream; -import com.google.common.io.ByteStreams; - -import org.apache.avro.reflect.Nullable; - -import java.io.IOException; -import java.io.Serializable; -import java.util.Arrays; -import java.util.Iterator; -import java.util.List; -import java.util.PriorityQueue; - /** * {@code PTransform}s for estimating the number of distinct elements * in a {@code PCollection}, or the number of distinct values 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 56c0bc4e5c6e2..26f0f660f0743 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 @@ -19,6 +19,18 @@ import static com.google.common.base.Preconditions.checkState; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.Iterables; +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Iterator; +import java.util.List; +import java.util.concurrent.ThreadLocalRandom; import org.apache.beam.sdk.coders.CannotProvideCoderException; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.CoderException; @@ -58,21 +70,6 @@ import org.apache.beam.sdk.values.TupleTagList; import org.apache.beam.sdk.values.TypeDescriptor; -import com.google.common.collect.ImmutableList; -import com.google.common.collect.Iterables; - -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonProperty; - -import java.io.IOException; -import java.io.InputStream; -import java.io.OutputStream; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Iterator; -import java.util.List; -import java.util.concurrent.ThreadLocalRandom; - /** * {@code PTransform}s for combining {@code PCollection} elements * globally and per-key. diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/CombineFnBase.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/CombineFnBase.java index c73ba542c3591..f98ec96b782e2 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/CombineFnBase.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/CombineFnBase.java @@ -17,6 +17,10 @@ */ package org.apache.beam.sdk.transforms; +import com.google.common.collect.ImmutableMap; +import java.io.Serializable; +import java.lang.reflect.Type; +import java.lang.reflect.TypeVariable; import org.apache.beam.sdk.coders.CannotProvideCoderException; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.CoderRegistry; @@ -28,12 +32,6 @@ import org.apache.beam.sdk.transforms.display.HasDisplayData; import org.apache.beam.sdk.values.TypeDescriptor; -import com.google.common.collect.ImmutableMap; - -import java.io.Serializable; -import java.lang.reflect.Type; -import java.lang.reflect.TypeVariable; - /** * This class contains the shared interfaces and abstract classes for different types of combine * functions. diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/CombineFns.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/CombineFns.java index 61f4888ebb2d9..9fa8ded6b2409 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/CombineFns.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/CombineFns.java @@ -19,6 +19,24 @@ import static com.google.common.base.Preconditions.checkArgument; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.collect.ArrayListMultimap; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Iterables; +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; +import com.google.common.collect.Multimap; +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.io.Serializable; +import java.util.Collection; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; import org.apache.beam.sdk.coders.CannotProvideCoderException; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.CoderException; @@ -37,27 +55,6 @@ import org.apache.beam.sdk.util.PropertyNames; import org.apache.beam.sdk.values.TupleTag; -import com.google.common.collect.ArrayListMultimap; -import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableMap; -import com.google.common.collect.Iterables; -import com.google.common.collect.Lists; -import com.google.common.collect.Maps; -import com.google.common.collect.Multimap; - -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonProperty; - -import java.io.IOException; -import java.io.InputStream; -import java.io.OutputStream; -import java.io.Serializable; -import java.util.Collection; -import java.util.Iterator; -import java.util.List; -import java.util.Map; -import java.util.Map.Entry; - /** * Static utility methods that create combine function instances. */ 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 195c5d17ed88a..b393a303951d7 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 @@ -17,6 +17,12 @@ */ package org.apache.beam.sdk.transforms; +import java.io.EOFException; +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.io.UTFDataFormatException; +import java.util.Iterator; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.CoderException; import org.apache.beam.sdk.coders.CoderRegistry; @@ -26,13 +32,6 @@ import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; -import java.io.EOFException; -import java.io.IOException; -import java.io.InputStream; -import java.io.OutputStream; -import java.io.UTFDataFormatException; -import java.util.Iterator; - /** * {@code PTransorm}s to count the elements in a {@link PCollection}. 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 08d0a7ae356ba..e261db2b30f91 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 @@ -19,6 +19,20 @@ import static com.google.common.base.Preconditions.checkArgument; +import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Function; +import com.google.common.base.Optional; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.Iterables; +import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.NoSuchElementException; +import java.util.Objects; +import javax.annotation.Nullable; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.coders.CannotProvideCoderException; import org.apache.beam.sdk.coders.Coder; @@ -37,26 +51,8 @@ import org.apache.beam.sdk.values.TimestampedValue; import org.apache.beam.sdk.values.TimestampedValue.TimestampedValueCoder; import org.apache.beam.sdk.values.TypeDescriptor; - -import com.google.common.annotations.VisibleForTesting; -import com.google.common.base.Function; -import com.google.common.base.Optional; -import com.google.common.collect.ImmutableList; -import com.google.common.collect.Iterables; - import org.joda.time.Instant; -import java.io.IOException; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Iterator; -import java.util.List; -import java.util.Map; -import java.util.NoSuchElementException; -import java.util.Objects; - -import javax.annotation.Nullable; - /** * {@code Create} takes a collection of elements of type {@code T} * known when the pipeline is constructed and returns a 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 2348783f9f616..9f898261c7bb3 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 @@ -21,6 +21,14 @@ import static com.google.common.base.Preconditions.checkNotNull; import static com.google.common.base.Preconditions.checkState; +import java.io.Serializable; +import java.lang.annotation.Documented; +import java.lang.annotation.ElementType; +import java.lang.annotation.Retention; +import java.lang.annotation.RetentionPolicy; +import java.lang.annotation.Target; +import java.util.HashMap; +import java.util.Map; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.Combine.CombineFn; import org.apache.beam.sdk.transforms.OldDoFn.DelegatingAggregator; @@ -32,19 +40,9 @@ 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; -import java.io.Serializable; -import java.lang.annotation.Documented; -import java.lang.annotation.ElementType; -import java.lang.annotation.Retention; -import java.lang.annotation.RetentionPolicy; -import java.lang.annotation.Target; -import java.util.HashMap; -import java.util.Map; - /** * The argument to {@link ParDo} providing the code to use to process * elements of the input 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 71a148ffafe90..4803d77f57f10 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 @@ -17,6 +17,7 @@ */ package org.apache.beam.sdk.transforms; +import java.io.IOException; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.display.DisplayData; import org.apache.beam.sdk.transforms.reflect.DoFnInvoker; @@ -28,12 +29,9 @@ 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; -import java.io.IOException; - /** * Utility class containing adapters for running a {@link DoFn} as an {@link OldDoFn}. * 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 4cd410a195973..82c12938fa684 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 @@ -17,6 +17,19 @@ */ package org.apache.beam.sdk.transforms; +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.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +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; @@ -34,20 +47,7 @@ import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TimestampedValue; import org.apache.beam.sdk.values.TupleTag; -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 org.joda.time.Instant; -import java.io.IOException; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collection; -import java.util.Collections; -import java.util.HashMap; -import java.util.List; -import java.util.Map; /** * A harness for unit-testing a {@link OldDoFn}. 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 2837c40cc3a6a..b590d459f9e1f 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 @@ -17,13 +17,12 @@ */ package org.apache.beam.sdk.transforms; +import java.lang.reflect.ParameterizedType; import org.apache.beam.sdk.transforms.display.DisplayData; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.TypeDescriptor; import org.apache.beam.sdk.values.TypeDescriptors; -import java.lang.reflect.ParameterizedType; - /** * {@code PTransform}s for mapping a simple function that returns iterables over the elements of a * {@link PCollection} and merging the results. diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/IntraBundleParallelization.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/IntraBundleParallelization.java index b5fe60f330d01..1eef0e1b06477 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/IntraBundleParallelization.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/IntraBundleParallelization.java @@ -19,6 +19,10 @@ import static com.google.common.base.Preconditions.checkArgument; +import com.google.common.base.Throwables; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Semaphore; +import java.util.concurrent.atomic.AtomicReference; import org.apache.beam.sdk.options.GcsOptions; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.Combine.CombineFn; @@ -30,15 +34,8 @@ import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TupleTag; import org.apache.beam.sdk.values.TypeDescriptor; - -import com.google.common.base.Throwables; - import org.joda.time.Instant; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.Semaphore; -import java.util.concurrent.atomic.AtomicReference; - /** * Provides multi-threading of {@link OldDoFn}s, using threaded execution to * process multiple elements concurrently within a bundle. diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Max.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Max.java index eed13fbb70b4e..c44d9b634671b 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Max.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Max.java @@ -17,11 +17,10 @@ */ package org.apache.beam.sdk.transforms; -import org.apache.beam.sdk.transforms.Combine.BinaryCombineFn; -import org.apache.beam.sdk.transforms.display.DisplayData; - import java.io.Serializable; import java.util.Comparator; +import org.apache.beam.sdk.transforms.Combine.BinaryCombineFn; +import org.apache.beam.sdk.transforms.display.DisplayData; /** * {@code PTransform}s for computing the maximum of the elements in a {@code PCollection}, or the diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Mean.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Mean.java index 5005f6bafb573..1a0791f993094 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Mean.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Mean.java @@ -17,6 +17,11 @@ */ package org.apache.beam.sdk.transforms; +import com.google.common.base.MoreObjects; +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.util.Objects; import org.apache.beam.sdk.coders.AtomicCoder; import org.apache.beam.sdk.coders.BigEndianLongCoder; import org.apache.beam.sdk.coders.Coder; @@ -25,13 +30,6 @@ import org.apache.beam.sdk.coders.DoubleCoder; import org.apache.beam.sdk.transforms.Combine.AccumulatingCombineFn.Accumulator; -import com.google.common.base.MoreObjects; - -import java.io.IOException; -import java.io.InputStream; -import java.io.OutputStream; -import java.util.Objects; - /** * {@code PTransform}s for computing the arithmetic mean * (a.k.a. average) of the elements in a {@code PCollection}, or the diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Min.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Min.java index 9c9d14f3ad01a..f046779038337 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Min.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Min.java @@ -17,11 +17,10 @@ */ package org.apache.beam.sdk.transforms; -import org.apache.beam.sdk.transforms.Combine.BinaryCombineFn; -import org.apache.beam.sdk.transforms.display.DisplayData; - import java.io.Serializable; import java.util.Comparator; +import org.apache.beam.sdk.transforms.Combine.BinaryCombineFn; +import org.apache.beam.sdk.transforms.display.DisplayData; /** * {@code PTransform}s for computing the minimum of the elements in a {@code PCollection}, or the 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 84cd9978c6212..474efef2c3d36 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 @@ -21,6 +21,14 @@ import static com.google.common.base.Preconditions.checkNotNull; import static com.google.common.base.Preconditions.checkState; +import com.google.common.base.MoreObjects; +import java.io.Serializable; +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; +import java.util.Objects; +import java.util.UUID; import org.apache.beam.sdk.annotations.Experimental; import org.apache.beam.sdk.annotations.Experimental.Kind; import org.apache.beam.sdk.options.PipelineOptions; @@ -33,20 +41,9 @@ import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TupleTag; import org.apache.beam.sdk.values.TypeDescriptor; - -import com.google.common.base.MoreObjects; - import org.joda.time.Duration; import org.joda.time.Instant; -import java.io.Serializable; -import java.util.Collection; -import java.util.Collections; -import java.util.HashMap; -import java.util.Map; -import java.util.Objects; -import java.util.UUID; - /** * The argument to {@link ParDo} providing the code to use to process * elements of the input 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 12ab54d0e3704..19abef90cea1b 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 @@ -17,6 +17,9 @@ */ package org.apache.beam.sdk.transforms; +import java.io.ObjectInputStream; +import java.io.ObjectOutputStream; +import java.io.Serializable; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.coders.CannotProvideCoderException; import org.apache.beam.sdk.coders.Coder; @@ -27,10 +30,6 @@ import org.apache.beam.sdk.values.POutput; import org.apache.beam.sdk.values.TypedPValue; -import java.io.ObjectInputStream; -import java.io.ObjectOutputStream; -import java.io.Serializable; - /** * A {@code PTransform} is an operation that takes an * {@code InputT} (some subtype of {@link PInput}) and produces an 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 af500ba0c9c39..5efbe9f190240 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 @@ -17,6 +17,10 @@ */ package org.apache.beam.sdk.transforms; +import com.google.common.collect.ImmutableList; +import java.io.Serializable; +import java.util.Arrays; +import java.util.List; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.annotations.Experimental; import org.apache.beam.sdk.coders.CannotProvideCoderException; @@ -34,12 +38,6 @@ import org.apache.beam.sdk.values.TupleTagList; import org.apache.beam.sdk.values.TypedPValue; -import com.google.common.collect.ImmutableList; - -import java.io.Serializable; -import java.util.Arrays; -import java.util.List; - /** * {@link ParDo} is the core element-wise transform in Google Cloud * Dataflow, invoking a user-specified function on each of the elements of the input 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 05c94700245a9..9247942f36dec 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 @@ -17,6 +17,7 @@ */ package org.apache.beam.sdk.transforms; +import java.io.Serializable; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.transforms.display.DisplayData; import org.apache.beam.sdk.values.PCollection; @@ -25,8 +26,6 @@ import org.apache.beam.sdk.values.TupleTag; import org.apache.beam.sdk.values.TupleTagList; -import java.io.Serializable; - /** * {@code Partition} takes a {@code PCollection} and a * {@code PartitionFn}, uses the {@code PartitionFn} to split the 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 12ff2b90b4952..eca987a942279 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 @@ -19,6 +19,9 @@ import static com.google.common.base.Preconditions.checkArgument; +import java.util.ArrayList; +import java.util.List; +import java.util.Random; import org.apache.beam.sdk.coders.BigEndianIntegerCoder; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.CoderRegistry; @@ -31,10 +34,6 @@ import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionView; -import java.util.ArrayList; -import java.util.List; -import java.util.Random; - /** * {@code PTransform}s for taking samples of the elements in a * {@code PCollection}, or samples of the values associated with each diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Top.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Top.java index 04cf9e33f6f98..992a341059a6e 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Top.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Top.java @@ -19,6 +19,15 @@ import static com.google.common.base.Preconditions.checkArgument; +import com.google.common.collect.Lists; +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.io.Serializable; +import java.util.ArrayList; +import java.util.Comparator; +import java.util.List; +import java.util.PriorityQueue; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.CoderException; import org.apache.beam.sdk.coders.CoderRegistry; @@ -34,17 +43,6 @@ import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; -import com.google.common.collect.Lists; - -import java.io.IOException; -import java.io.InputStream; -import java.io.OutputStream; -import java.io.Serializable; -import java.util.ArrayList; -import java.util.Comparator; -import java.util.List; -import java.util.PriorityQueue; - /** * {@code PTransform}s for finding the largest (or smallest) set * of elements in a {@code PCollection}, or the largest (or smallest) 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 7a97c13d336cc..d750efc9fedc4 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 @@ -17,15 +17,14 @@ */ package org.apache.beam.sdk.transforms; +import java.util.List; +import java.util.Map; import org.apache.beam.sdk.runners.PipelineRunner; import org.apache.beam.sdk.util.PCollectionViews; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionView; -import java.util.List; -import java.util.Map; - /** * Transforms for creating {@link PCollectionView PCollectionViews} from * {@link PCollection PCollections} (to read them as side inputs). diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/ViewFn.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/ViewFn.java index aa3cb0d628e64..767e58e243d44 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/ViewFn.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/ViewFn.java @@ -17,11 +17,10 @@ */ package org.apache.beam.sdk.transforms; +import java.io.Serializable; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionView; -import java.io.Serializable; - /** * A function to adapt a primitive "view" of a {@link PCollection} - some materialization * specified in the Beam model and implemented by the runner - to a user-facing view type 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 7b395f5e8dbf4..00ac8e4a392db 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 @@ -24,7 +24,6 @@ import org.apache.beam.sdk.transforms.windowing.Window; import org.apache.beam.sdk.transforms.windowing.WindowFn; import org.apache.beam.sdk.values.PCollection; - import org.joda.time.Duration; import org.joda.time.Instant; 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 5dcaec892379d..0b92d9fce3947 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 @@ -20,27 +20,24 @@ import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.base.Preconditions.checkNotNull; -import org.apache.beam.sdk.transforms.PTransform; - +import com.fasterxml.jackson.annotation.JsonGetter; +import com.fasterxml.jackson.annotation.JsonInclude; +import com.fasterxml.jackson.annotation.JsonValue; import com.google.auto.value.AutoValue; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Maps; import com.google.common.collect.Sets; - -import com.fasterxml.jackson.annotation.JsonGetter; -import com.fasterxml.jackson.annotation.JsonInclude; -import com.fasterxml.jackson.annotation.JsonValue; -import org.apache.avro.reflect.Nullable; -import org.joda.time.Duration; -import org.joda.time.Instant; -import org.joda.time.format.DateTimeFormatter; -import org.joda.time.format.ISODateTimeFormat; - import java.io.Serializable; import java.util.Collection; import java.util.Map; import java.util.Objects; import java.util.Set; +import org.apache.avro.reflect.Nullable; +import org.apache.beam.sdk.transforms.PTransform; +import org.joda.time.Duration; +import org.joda.time.Instant; +import org.joda.time.format.DateTimeFormatter; +import org.joda.time.format.ISODateTimeFormat; /** * Static display data associated with a pipeline component. Display data is useful for diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/join/CoGbkResult.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/join/CoGbkResult.java index aa26cbbf68e24..10ba3c9489da7 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/join/CoGbkResult.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/join/CoGbkResult.java @@ -17,10 +17,21 @@ */ package org.apache.beam.sdk.transforms.join; -import static org.apache.beam.sdk.util.Structs.addObject; - import static com.google.common.base.Preconditions.checkArgument; +import static org.apache.beam.sdk.util.Structs.addObject; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.collect.Iterators; +import com.google.common.collect.PeekingIterator; +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Iterator; +import java.util.List; +import java.util.Objects; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.CoderException; import org.apache.beam.sdk.coders.IterableCoder; @@ -31,24 +42,9 @@ import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.TupleTag; import org.apache.beam.sdk.values.TupleTagList; - -import com.google.common.collect.Iterators; -import com.google.common.collect.PeekingIterator; - -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonProperty; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.io.IOException; -import java.io.InputStream; -import java.io.OutputStream; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Iterator; -import java.util.List; -import java.util.Objects; - /** * A row result of a {@link CoGroupByKey}. This is a tuple of {@link Iterable}s produced for * a given key, and these can be accessed in different ways. diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/join/CoGbkResultSchema.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/join/CoGbkResultSchema.java index 29a0c7d731d4b..276cccb3a0531 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/join/CoGbkResultSchema.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/join/CoGbkResultSchema.java @@ -19,18 +19,16 @@ import static org.apache.beam.sdk.util.Structs.addList; -import org.apache.beam.sdk.util.CloudObject; -import org.apache.beam.sdk.util.PropertyNames; -import org.apache.beam.sdk.values.TupleTag; -import org.apache.beam.sdk.values.TupleTagList; - import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; - import java.io.Serializable; import java.util.ArrayList; import java.util.HashMap; import java.util.List; +import org.apache.beam.sdk.util.CloudObject; +import org.apache.beam.sdk.util.PropertyNames; +import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.TupleTagList; /** * A schema for the results of a {@link CoGroupByKey}. This maintains the full 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 cb06f95354fe5..9546b291d4fc5 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 @@ -17,6 +17,8 @@ */ package org.apache.beam.sdk.transforms.join; +import java.util.ArrayList; +import java.util.List; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.KvCoder; import org.apache.beam.sdk.transforms.DoFn; @@ -30,9 +32,6 @@ import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionList; -import java.util.ArrayList; -import java.util.List; - /** * A {@link PTransform} that performs a {@link CoGroupByKey} on a tuple * of tables. A {@link CoGroupByKey} groups results from all 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 0852c9c25b29b..67b819fddc2cc 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 @@ -17,6 +17,9 @@ */ 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; import org.apache.beam.sdk.coders.KvCoder; @@ -29,10 +32,6 @@ import org.apache.beam.sdk.values.TupleTag; import org.apache.beam.sdk.values.TupleTagList; -import java.util.ArrayList; -import java.util.Collection; -import java.util.List; - /** * An immutable tuple of keyed {@link PCollection PCollections} * with key type K. diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/join/UnionCoder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/join/UnionCoder.java index 29240e7bb8638..72ac6e881ffff 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/join/UnionCoder.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/join/UnionCoder.java @@ -17,20 +17,18 @@ */ package org.apache.beam.sdk.transforms.join; -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.util.PropertyNames; -import org.apache.beam.sdk.util.VarInt; -import org.apache.beam.sdk.util.common.ElementByteSizeObserver; - import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; - import java.io.IOException; import java.io.InputStream; import java.io.OutputStream; import java.util.List; +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.util.PropertyNames; +import org.apache.beam.sdk.util.VarInt; +import org.apache.beam.sdk.util.common.ElementByteSizeObserver; /** * A UnionCoder encodes RawUnionValues. 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 73874d767d3c1..68e2ca99886e1 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 @@ -20,14 +20,14 @@ import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.base.Preconditions.checkNotNull; -import org.apache.beam.sdk.transforms.DoFn; -import org.apache.beam.sdk.transforms.DoFn.FinishBundle; -import org.apache.beam.sdk.transforms.DoFn.ProcessElement; -import org.apache.beam.sdk.transforms.DoFn.Setup; -import org.apache.beam.sdk.transforms.DoFn.StartBundle; -import org.apache.beam.sdk.transforms.DoFn.Teardown; -import org.apache.beam.sdk.util.UserCodeException; - +import java.lang.reflect.Constructor; +import java.lang.reflect.InvocationTargetException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.EnumMap; +import java.util.LinkedHashMap; +import java.util.Map; +import javax.annotation.Nullable; import net.bytebuddy.ByteBuddy; import net.bytebuddy.NamingStrategy; import net.bytebuddy.description.field.FieldDescription; @@ -53,16 +53,13 @@ import net.bytebuddy.jar.asm.MethodVisitor; import net.bytebuddy.jar.asm.Opcodes; import net.bytebuddy.matcher.ElementMatchers; - -import java.lang.reflect.Constructor; -import java.lang.reflect.InvocationTargetException; -import java.util.ArrayList; -import java.util.Collections; -import java.util.EnumMap; -import java.util.LinkedHashMap; -import java.util.Map; - -import javax.annotation.Nullable; +import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.transforms.DoFn.FinishBundle; +import org.apache.beam.sdk.transforms.DoFn.ProcessElement; +import org.apache.beam.sdk.transforms.DoFn.Setup; +import org.apache.beam.sdk.transforms.DoFn.StartBundle; +import org.apache.beam.sdk.transforms.DoFn.Teardown; +import org.apache.beam.sdk.util.UserCodeException; /** Dynamically generates {@link DoFnInvoker} instances for invoking a {@link DoFn}. */ public class DoFnInvokers { 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 6730140ac309d..181c088820496 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 @@ -17,15 +17,12 @@ */ package org.apache.beam.sdk.transforms.reflect; -import org.apache.beam.sdk.transforms.DoFn; - import com.google.auto.value.AutoValue; - import java.lang.reflect.Method; import java.util.Collections; import java.util.List; - import javax.annotation.Nullable; +import org.apache.beam.sdk.transforms.DoFn; /** * Describes the signature of a {@link DoFn}, in particular, which features it uses, which extra 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 80b3b4f5b825a..7e482d5df6e32 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 @@ -20,14 +20,9 @@ import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.base.Preconditions.checkNotNull; -import org.apache.beam.sdk.transforms.DoFn; -import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.common.ReflectHelpers; - import com.google.common.annotations.VisibleForTesting; import com.google.common.reflect.TypeParameter; import com.google.common.reflect.TypeToken; - import java.lang.annotation.Annotation; import java.lang.reflect.Method; import java.lang.reflect.Modifier; @@ -41,6 +36,9 @@ import java.util.LinkedHashSet; import java.util.List; import java.util.Map; +import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.transforms.windowing.BoundedWindow; +import org.apache.beam.sdk.util.common.ReflectHelpers; /** * Parses a {@link DoFn} and computes its {@link DoFnSignature}. See {@link #getOrParseSignature}. diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterAll.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterAll.java index 27dd1293ad32c..cc8c97fa14bf6 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterAll.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterAll.java @@ -19,17 +19,14 @@ import static com.google.common.base.Preconditions.checkArgument; +import com.google.common.base.Joiner; +import java.util.Arrays; +import java.util.List; import org.apache.beam.sdk.annotations.Experimental; import org.apache.beam.sdk.transforms.windowing.Trigger.OnceTrigger; import org.apache.beam.sdk.util.ExecutableTrigger; - -import com.google.common.base.Joiner; - import org.joda.time.Instant; -import java.util.Arrays; -import java.util.List; - /** * Create a {@link Trigger} that fires and finishes once after all of its sub-triggers have fired. */ diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterDelayFromFirstElement.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterDelayFromFirstElement.java index a0fc04d64288a..c4bc94615dc3d 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterDelayFromFirstElement.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterDelayFromFirstElement.java @@ -17,6 +17,12 @@ */ package org.apache.beam.sdk.transforms.windowing; +import com.google.common.collect.ImmutableList; +import edu.umd.cs.findbugs.annotations.SuppressFBWarnings; +import java.util.List; +import java.util.Locale; +import java.util.Objects; +import javax.annotation.Nullable; import org.apache.beam.sdk.annotations.Experimental; import org.apache.beam.sdk.coders.InstantCoder; import org.apache.beam.sdk.transforms.Combine; @@ -31,21 +37,11 @@ import org.apache.beam.sdk.util.state.StateMerging; import org.apache.beam.sdk.util.state.StateTag; import org.apache.beam.sdk.util.state.StateTags; - -import com.google.common.collect.ImmutableList; - import org.joda.time.Duration; import org.joda.time.Instant; import org.joda.time.format.PeriodFormat; import org.joda.time.format.PeriodFormatter; -import java.util.List; -import java.util.Locale; -import java.util.Objects; -import javax.annotation.Nullable; - -import edu.umd.cs.findbugs.annotations.SuppressFBWarnings; - /** * A base class for triggers that happen after a processing time delay from the arrival * of the first element in a pane. diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterEach.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterEach.java index dc1e74b099581..629c640eabddf 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterEach.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterEach.java @@ -19,15 +19,12 @@ import static com.google.common.base.Preconditions.checkArgument; -import org.apache.beam.sdk.annotations.Experimental; -import org.apache.beam.sdk.util.ExecutableTrigger; - import com.google.common.base.Joiner; - -import org.joda.time.Instant; - import java.util.Arrays; import java.util.List; +import org.apache.beam.sdk.annotations.Experimental; +import org.apache.beam.sdk.util.ExecutableTrigger; +import org.joda.time.Instant; /** * A composite {@link Trigger} that executes its sub-triggers in order. diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterFirst.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterFirst.java index 1034cac7d8b9a..6b06cfa09eb3f 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterFirst.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterFirst.java @@ -19,17 +19,14 @@ import static com.google.common.base.Preconditions.checkArgument; +import com.google.common.base.Joiner; +import java.util.Arrays; +import java.util.List; import org.apache.beam.sdk.annotations.Experimental; import org.apache.beam.sdk.transforms.windowing.Trigger.OnceTrigger; import org.apache.beam.sdk.util.ExecutableTrigger; - -import com.google.common.base.Joiner; - import org.joda.time.Instant; -import java.util.Arrays; -import java.util.List; - /** * Create a composite {@link Trigger} that fires once after at least one of its sub-triggers have * fired. diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterPane.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterPane.java index 73289877c00da..8c128dd32bc22 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterPane.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterPane.java @@ -17,6 +17,9 @@ */ package org.apache.beam.sdk.transforms.windowing; +import edu.umd.cs.findbugs.annotations.SuppressFBWarnings; +import java.util.List; +import java.util.Objects; import org.apache.beam.sdk.annotations.Experimental; import org.apache.beam.sdk.coders.VarLongCoder; import org.apache.beam.sdk.transforms.Sum; @@ -27,14 +30,8 @@ import org.apache.beam.sdk.util.state.StateMerging; import org.apache.beam.sdk.util.state.StateTag; import org.apache.beam.sdk.util.state.StateTags; - import org.joda.time.Instant; -import java.util.List; -import java.util.Objects; - -import edu.umd.cs.findbugs.annotations.SuppressFBWarnings; - /** * {@link Trigger}s that fire based on properties of the elements in the current pane. */ diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterProcessingTime.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterProcessingTime.java index 4c792df518293..f551118b22c05 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterProcessingTime.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterProcessingTime.java @@ -17,17 +17,14 @@ */ package org.apache.beam.sdk.transforms.windowing; +import java.util.List; +import java.util.Objects; +import javax.annotation.Nullable; import org.apache.beam.sdk.annotations.Experimental; import org.apache.beam.sdk.transforms.SerializableFunction; import org.apache.beam.sdk.util.TimeDomain; - import org.joda.time.Instant; -import java.util.List; -import java.util.Objects; - -import javax.annotation.Nullable; - /** * {@code AfterProcessingTime} triggers fire based on the current processing time. They operate in * the real-time domain. diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterSynchronizedProcessingTime.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterSynchronizedProcessingTime.java index e8e0fb6f707e5..59ece1073c20f 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterSynchronizedProcessingTime.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterSynchronizedProcessingTime.java @@ -17,17 +17,13 @@ */ package org.apache.beam.sdk.transforms.windowing; -import org.apache.beam.sdk.transforms.SerializableFunction; -import org.apache.beam.sdk.util.TimeDomain; - import com.google.common.base.Objects; - -import org.joda.time.Instant; - import java.util.Collections; import java.util.List; - import javax.annotation.Nullable; +import org.apache.beam.sdk.transforms.SerializableFunction; +import org.apache.beam.sdk.util.TimeDomain; +import org.joda.time.Instant; class AfterSynchronizedProcessingTime extends AfterDelayFromFirstElement { diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterWatermark.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterWatermark.java index 0d2a878e56b21..9690be8385265 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterWatermark.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/AfterWatermark.java @@ -19,20 +19,16 @@ import static com.google.common.base.Preconditions.checkNotNull; +import com.google.common.collect.ImmutableList; +import java.util.List; +import java.util.Objects; +import javax.annotation.Nullable; import org.apache.beam.sdk.annotations.Experimental; import org.apache.beam.sdk.transforms.windowing.Trigger.OnceTrigger; import org.apache.beam.sdk.util.ExecutableTrigger; import org.apache.beam.sdk.util.TimeDomain; - -import com.google.common.collect.ImmutableList; - import org.joda.time.Instant; -import java.util.List; -import java.util.Objects; - -import javax.annotation.Nullable; - /** *

        {@code AfterWatermark} triggers fire based on progress of the system watermark. This time is a * lower-bound, sometimes heuristically established, on event times that have been fully processed 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 50ee9b5552c43..3654074174c17 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 @@ -17,9 +17,8 @@ */ package org.apache.beam.sdk.transforms.windowing; -import org.joda.time.Instant; - import java.util.concurrent.TimeUnit; +import org.joda.time.Instant; /** * A {@code BoundedWindow} represents a finite grouping of elements, with an diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/CalendarWindows.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/CalendarWindows.java index 9653ae84ea944..fada50ad5a4d5 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/CalendarWindows.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/CalendarWindows.java @@ -19,7 +19,6 @@ import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.transforms.display.DisplayData; - import org.joda.time.DateTime; import org.joda.time.DateTimeZone; import org.joda.time.Days; diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/DefaultTrigger.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/DefaultTrigger.java index fcea3337b8a4f..d6b72ef603520 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/DefaultTrigger.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/DefaultTrigger.java @@ -17,13 +17,11 @@ */ package org.apache.beam.sdk.transforms.windowing; +import java.util.List; import org.apache.beam.sdk.annotations.Experimental; import org.apache.beam.sdk.util.TimeDomain; - import org.joda.time.Instant; -import java.util.List; - /** * A trigger that is equivalent to {@code Repeatedly.forever(AfterWatermark.pastEndOfWindow())}. * See {@link Repeatedly#forever} and {@link AfterWatermark#pastEndOfWindow} for more details. diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/FixedWindows.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/FixedWindows.java index 1e439ff389b39..8683a600e2045 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/FixedWindows.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/FixedWindows.java @@ -17,14 +17,12 @@ */ package org.apache.beam.sdk.transforms.windowing; +import java.util.Objects; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.transforms.display.DisplayData; - import org.joda.time.Duration; import org.joda.time.Instant; -import java.util.Objects; - /** * A {@link WindowFn} that windows values into fixed-size timestamp-based windows. * diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/GlobalWindow.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/GlobalWindow.java index a4712fef76367..58b059aab67ce 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/GlobalWindow.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/GlobalWindow.java @@ -17,14 +17,12 @@ */ package org.apache.beam.sdk.transforms.windowing; +import java.io.InputStream; +import java.io.OutputStream; import org.apache.beam.sdk.coders.AtomicCoder; - import org.joda.time.Duration; import org.joda.time.Instant; -import java.io.InputStream; -import java.io.OutputStream; - /** * The default window into which all data is placed (via {@link GlobalWindows}). */ diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/GlobalWindows.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/GlobalWindows.java index 002bf2e0b419a..aba00a3122911 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/GlobalWindows.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/GlobalWindows.java @@ -17,12 +17,10 @@ */ package org.apache.beam.sdk.transforms.windowing; -import org.apache.beam.sdk.coders.Coder; - -import org.joda.time.Instant; - import java.util.Collection; import java.util.Collections; +import org.apache.beam.sdk.coders.Coder; +import org.joda.time.Instant; /** * Default {@link WindowFn} that assigns all data to the same window. diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/IntervalWindow.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/IntervalWindow.java index 45898e00d26b9..af987966be922 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/IntervalWindow.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/IntervalWindow.java @@ -17,21 +17,19 @@ */ package org.apache.beam.sdk.transforms.windowing; +import com.fasterxml.jackson.annotation.JsonCreator; +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; 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.DurationCoder; import org.apache.beam.sdk.coders.InstantCoder; - -import com.fasterxml.jackson.annotation.JsonCreator; import org.joda.time.Duration; import org.joda.time.Instant; import org.joda.time.ReadableDuration; -import java.io.IOException; -import java.io.InputStream; -import java.io.OutputStream; - /** * An implementation of {@link BoundedWindow} that represents an interval from * {@link #start} (inclusive) to {@link #end} (exclusive). diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/InvalidWindows.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/InvalidWindows.java index e0d65e5e98e5c..cbbc937eecb6b 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/InvalidWindows.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/InvalidWindows.java @@ -17,12 +17,10 @@ */ package org.apache.beam.sdk.transforms.windowing; +import java.util.Collection; import org.apache.beam.sdk.coders.Coder; - import org.joda.time.Instant; -import java.util.Collection; - /** * A {@link WindowFn} that represents an invalid pipeline state. * diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/Never.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/Never.java index 7267d002715bd..e1f5d4da0c821 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/Never.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/Never.java @@ -17,13 +17,11 @@ */ package org.apache.beam.sdk.transforms.windowing; +import java.util.List; import org.apache.beam.sdk.transforms.GroupByKey; import org.apache.beam.sdk.transforms.windowing.Trigger.OnceTrigger; - import org.joda.time.Instant; -import java.util.List; - /** * A trigger which never fires. * diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/OrFinallyTrigger.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/OrFinallyTrigger.java index c48f5f47dacad..25b7b34b520b4 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/OrFinallyTrigger.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/OrFinallyTrigger.java @@ -17,14 +17,11 @@ */ package org.apache.beam.sdk.transforms.windowing; -import org.apache.beam.sdk.util.ExecutableTrigger; - import com.google.common.annotations.VisibleForTesting; - -import org.joda.time.Instant; - import java.util.Arrays; import java.util.List; +import org.apache.beam.sdk.util.ExecutableTrigger; +import org.joda.time.Instant; /** * Executes the {@code actual} trigger until it finishes or until the {@code until} trigger fires. diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/OutputTimeFn.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/OutputTimeFn.java index 497a6fbd0c98c..0efd278a644a3 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/OutputTimeFn.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/OutputTimeFn.java @@ -17,14 +17,11 @@ */ package org.apache.beam.sdk.transforms.windowing; -import org.apache.beam.sdk.annotations.Experimental; - import com.google.common.collect.Ordering; - -import org.joda.time.Instant; - import java.io.Serializable; import java.util.Objects; +import org.apache.beam.sdk.annotations.Experimental; +import org.joda.time.Instant; /** * (Experimental) A function from timestamps of input values to the timestamp for a diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/OutputTimeFns.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/OutputTimeFns.java index 363181b22df21..2bcd3192f5ea7 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/OutputTimeFns.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/OutputTimeFns.java @@ -19,14 +19,11 @@ import static com.google.common.base.Preconditions.checkArgument; -import org.apache.beam.sdk.annotations.Experimental; - import com.google.common.collect.Iterables; import com.google.common.collect.Ordering; - -import org.joda.time.Instant; - import javax.annotation.Nullable; +import org.apache.beam.sdk.annotations.Experimental; +import org.joda.time.Instant; /** * (Experimental) Static utility methods and provided implementations for diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/PaneInfo.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/PaneInfo.java index 6ec17f991c0c7..0c87e2271ec92 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/PaneInfo.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/PaneInfo.java @@ -20,20 +20,18 @@ import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.base.Preconditions.checkNotNull; -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.transforms.GroupByKey; -import org.apache.beam.sdk.transforms.OldDoFn; -import org.apache.beam.sdk.util.VarInt; - import com.google.common.base.MoreObjects; import com.google.common.collect.ImmutableMap; - import java.io.IOException; import java.io.InputStream; import java.io.OutputStream; import java.util.Objects; +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.transforms.GroupByKey; +import org.apache.beam.sdk.transforms.OldDoFn; +import org.apache.beam.sdk.util.VarInt; /** * Provides information about the pane an element belongs to. Every pane is implicitly associated diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/PartitioningWindowFn.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/PartitioningWindowFn.java index da2f38ce6cdcf..b84f891464bed 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/PartitioningWindowFn.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/PartitioningWindowFn.java @@ -17,10 +17,9 @@ */ package org.apache.beam.sdk.transforms.windowing; -import org.joda.time.Instant; - import java.util.Arrays; import java.util.Collection; +import org.joda.time.Instant; /** * A {@link WindowFn} that places each value into exactly one window based on its timestamp and diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/Repeatedly.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/Repeatedly.java index 591bbf06a0ea9..88587983dfc20 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/Repeatedly.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/Repeatedly.java @@ -17,12 +17,10 @@ */ package org.apache.beam.sdk.transforms.windowing; -import org.apache.beam.sdk.util.ExecutableTrigger; - -import org.joda.time.Instant; - import java.util.Arrays; import java.util.List; +import org.apache.beam.sdk.util.ExecutableTrigger; +import org.joda.time.Instant; /** * Repeat a trigger, either until some condition is met or forever. diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/Sessions.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/Sessions.java index 875db12918ae3..14d68f972381b 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/Sessions.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/Sessions.java @@ -17,14 +17,12 @@ */ package org.apache.beam.sdk.transforms.windowing; -import org.apache.beam.sdk.coders.Coder; -import org.apache.beam.sdk.transforms.display.DisplayData; - -import org.joda.time.Duration; - import java.util.Arrays; import java.util.Collection; import java.util.Objects; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.transforms.display.DisplayData; +import org.joda.time.Duration; /** * A {@link WindowFn} windowing values into sessions separated by {@link #gapDuration}-long diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/SlidingWindows.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/SlidingWindows.java index 99367f5557f4a..ec21723384f4b 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/SlidingWindows.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/SlidingWindows.java @@ -17,19 +17,17 @@ */ package org.apache.beam.sdk.transforms.windowing; +import java.util.ArrayList; +import java.util.Collection; +import java.util.List; +import java.util.Objects; import org.apache.beam.sdk.annotations.Experimental; import org.apache.beam.sdk.annotations.Experimental.Kind; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.transforms.display.DisplayData; - import org.joda.time.Duration; import org.joda.time.Instant; -import java.util.ArrayList; -import java.util.Collection; -import java.util.List; -import java.util.Objects; - /** * A {@link WindowFn} that windows values into possibly overlapping fixed-size * timestamp-based windows. diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/Trigger.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/Trigger.java index 86801e817799e..a960aa4b7a009 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/Trigger.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/Trigger.java @@ -17,22 +17,18 @@ */ package org.apache.beam.sdk.transforms.windowing; -import org.apache.beam.sdk.annotations.Experimental; -import org.apache.beam.sdk.util.ExecutableTrigger; -import org.apache.beam.sdk.util.TimeDomain; -import org.apache.beam.sdk.util.state.MergingStateAccessor; -import org.apache.beam.sdk.util.state.StateAccessor; - import com.google.common.base.Joiner; - -import org.joda.time.Instant; - import java.io.Serializable; import java.util.ArrayList; import java.util.List; import java.util.Objects; - import javax.annotation.Nullable; +import org.apache.beam.sdk.annotations.Experimental; +import org.apache.beam.sdk.util.ExecutableTrigger; +import org.apache.beam.sdk.util.TimeDomain; +import org.apache.beam.sdk.util.state.MergingStateAccessor; +import org.apache.beam.sdk.util.state.StateAccessor; +import org.joda.time.Instant; /** * {@code Trigger}s control when the elements for a specific key and window are output. As elements 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 9dd069cf99521..52b78584fcb40 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 @@ -17,6 +17,7 @@ */ package org.apache.beam.sdk.transforms.windowing; +import javax.annotation.Nullable; import org.apache.beam.sdk.annotations.Experimental; import org.apache.beam.sdk.annotations.Experimental.Kind; import org.apache.beam.sdk.coders.Coder; @@ -29,11 +30,8 @@ import org.apache.beam.sdk.util.WindowingStrategy; import org.apache.beam.sdk.util.WindowingStrategy.AccumulationMode; import org.apache.beam.sdk.values.PCollection; - import org.joda.time.Duration; -import javax.annotation.Nullable; - /** * {@code Window} logically divides up or groups the elements of a * {@link PCollection} into finite windows according to a {@link WindowFn}. diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/WindowFn.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/WindowFn.java index d84866b6fb728..abb4ee083f203 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/WindowFn.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/WindowFn.java @@ -17,17 +17,15 @@ */ package org.apache.beam.sdk.transforms.windowing; +import java.io.Serializable; +import java.util.Collection; import org.apache.beam.sdk.annotations.Experimental; import org.apache.beam.sdk.annotations.Experimental.Kind; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.transforms.display.DisplayData; import org.apache.beam.sdk.transforms.display.HasDisplayData; - import org.joda.time.Instant; -import java.io.Serializable; -import java.util.Collection; - /** * The argument to the {@link Window} transform used to assign elements into * windows and to determine how windows are merged. See {@link Window} for more diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ActiveWindowSet.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ActiveWindowSet.java index 02c12c040b2d8..2e0af29732a10 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ActiveWindowSet.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ActiveWindowSet.java @@ -17,13 +17,11 @@ */ package org.apache.beam.sdk.util; -import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.transforms.windowing.WindowFn; - import com.google.common.annotations.VisibleForTesting; - import java.util.Collection; import java.util.Set; +import org.apache.beam.sdk.transforms.windowing.BoundedWindow; +import org.apache.beam.sdk.transforms.windowing.WindowFn; /** * Track which windows are active, and the state address window(s) under which their 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 0d6874026cb28..3914bb0e10a10 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 @@ -29,10 +29,6 @@ import com.google.common.reflect.Invokable; import com.google.common.reflect.Parameter; import com.google.common.reflect.TypeToken; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - import java.io.IOException; import java.lang.annotation.Annotation; import java.lang.reflect.Constructor; @@ -49,6 +45,8 @@ import java.util.List; import java.util.Set; import java.util.regex.Pattern; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * Represents the API surface of a package prefix. Used for accessing public classes, diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/AppliedCombineFn.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/AppliedCombineFn.java index 1e7d7c002e191..30b302c9953d2 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/AppliedCombineFn.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/AppliedCombineFn.java @@ -17,6 +17,8 @@ */ package org.apache.beam.sdk.util; +import com.google.common.annotations.VisibleForTesting; +import java.io.Serializable; import org.apache.beam.sdk.coders.CannotProvideCoderException; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.CoderRegistry; @@ -25,10 +27,6 @@ import org.apache.beam.sdk.transforms.CombineWithContext.KeyedCombineFnWithContext; import org.apache.beam.sdk.values.PCollectionView; -import com.google.common.annotations.VisibleForTesting; - -import java.io.Serializable; - /** * A {@link KeyedCombineFnWithContext} with a fixed accumulator coder. This is created from a * specific application of the {@link KeyedCombineFnWithContext}. diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/AttemptAndTimeBoundedExponentialBackOff.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/AttemptAndTimeBoundedExponentialBackOff.java index df2897ee91e0e..d8050e0a32198 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/AttemptAndTimeBoundedExponentialBackOff.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/AttemptAndTimeBoundedExponentialBackOff.java @@ -21,10 +21,8 @@ import com.google.api.client.util.BackOff; import com.google.api.client.util.NanoClock; - -import java.util.concurrent.TimeUnit; - import edu.umd.cs.findbugs.annotations.SuppressFBWarnings; +import java.util.concurrent.TimeUnit; /** * Extension of {@link AttemptBoundedExponentialBackOff} that bounds the total time that the backoff diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/AvroUtils.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/AvroUtils.java index 16fc6fa5b3fbd..d855b344fe10d 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/AvroUtils.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/AvroUtils.java @@ -17,15 +17,14 @@ */ package org.apache.beam.sdk.util; -import org.apache.avro.file.DataFileConstants; -import org.apache.avro.io.BinaryDecoder; -import org.apache.avro.io.DecoderFactory; - import java.io.IOException; import java.io.InputStream; import java.nio.ByteBuffer; import java.nio.channels.Channels; import java.util.Arrays; +import org.apache.avro.file.DataFileConstants; +import org.apache.avro.io.BinaryDecoder; +import org.apache.avro.io.DecoderFactory; /** * A set of utilities for working with Avro files. diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/BaseExecutionContext.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/BaseExecutionContext.java index dd36367f75461..dead76eb3d001 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/BaseExecutionContext.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/BaseExecutionContext.java @@ -17,16 +17,15 @@ */ package org.apache.beam.sdk.util; -import org.apache.beam.sdk.coders.Coder; -import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.state.StateInternals; -import org.apache.beam.sdk.values.TupleTag; - import java.io.IOException; import java.util.Collection; import java.util.Collections; import java.util.HashMap; 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.state.StateInternals; +import org.apache.beam.sdk.values.TupleTag; /** * Base class for implementations of {@link ExecutionContext}. diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/BitSetCoder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/BitSetCoder.java index b406b1a56cc52..fde90af683697 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/BitSetCoder.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/BitSetCoder.java @@ -17,14 +17,13 @@ */ package org.apache.beam.sdk.util; -import org.apache.beam.sdk.coders.AtomicCoder; -import org.apache.beam.sdk.coders.ByteArrayCoder; -import org.apache.beam.sdk.coders.CoderException; - import java.io.IOException; import java.io.InputStream; import java.io.OutputStream; import java.util.BitSet; +import org.apache.beam.sdk.coders.AtomicCoder; +import org.apache.beam.sdk.coders.ByteArrayCoder; +import org.apache.beam.sdk.coders.CoderException; /** * Coder for the BitSet used to track child-trigger finished states. diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/BucketingFunction.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/BucketingFunction.java index e14aec862a0ea..8740b22710772 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/BucketingFunction.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/BucketingFunction.java @@ -20,10 +20,9 @@ import static com.google.common.base.Preconditions.checkState; -import org.apache.beam.sdk.transforms.Combine; - import java.util.HashMap; import java.util.Map; +import org.apache.beam.sdk.transforms.Combine; /** * Keep track of the minimum/maximum/sum of a set of timestamped long values. diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/BufferedElementCountingOutputStream.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/BufferedElementCountingOutputStream.java index 6335de47378db..c17d92d41f8a8 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/BufferedElementCountingOutputStream.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/BufferedElementCountingOutputStream.java @@ -17,13 +17,11 @@ */ package org.apache.beam.sdk.util; -import org.apache.beam.sdk.coders.Coder.Context; - import java.io.IOException; import java.io.OutputStream; import java.nio.ByteBuffer; - import javax.annotation.concurrent.NotThreadSafe; +import org.apache.beam.sdk.coders.Coder.Context; /** * Provides an efficient encoding for {@link Iterable}s containing small values by diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/CloudKnownType.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/CloudKnownType.java index ad79b989d9239..b22b3c01286d8 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/CloudKnownType.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/CloudKnownType.java @@ -20,7 +20,6 @@ import java.util.Collections; import java.util.HashMap; import java.util.Map; - import javax.annotation.Nullable; /** A utility for manipulating well-known cloud types. */ diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/CloudObject.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/CloudObject.java index 27722255e9164..9cab453bdf89e 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/CloudObject.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/CloudObject.java @@ -22,9 +22,7 @@ import com.google.api.client.json.GenericJson; import com.google.api.client.util.Key; - import java.util.Map; - import javax.annotation.Nullable; /** diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/CoderUtils.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/CoderUtils.java index 520ba4a9a6111..36bf78980e546 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/CoderUtils.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/CoderUtils.java @@ -19,15 +19,6 @@ import static org.apache.beam.sdk.util.Structs.addList; -import org.apache.beam.sdk.coders.Coder; -import org.apache.beam.sdk.coders.CoderException; -import org.apache.beam.sdk.coders.IterableCoder; -import org.apache.beam.sdk.coders.KvCoder; -import org.apache.beam.sdk.values.TypeDescriptor; - -import com.google.api.client.util.Base64; -import com.google.common.base.Throwables; - import com.fasterxml.jackson.annotation.JsonTypeInfo; import com.fasterxml.jackson.annotation.JsonTypeInfo.As; import com.fasterxml.jackson.annotation.JsonTypeInfo.Id; @@ -38,7 +29,8 @@ import com.fasterxml.jackson.databind.jsontype.impl.TypeIdResolverBase; import com.fasterxml.jackson.databind.module.SimpleModule; import com.fasterxml.jackson.databind.type.TypeFactory; - +import com.google.api.client.util.Base64; +import com.google.common.base.Throwables; import java.io.ByteArrayInputStream; import java.io.ByteArrayOutputStream; import java.io.IOException; @@ -47,6 +39,11 @@ import java.lang.ref.SoftReference; import java.lang.reflect.ParameterizedType; import java.lang.reflect.TypeVariable; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.coders.CoderException; +import org.apache.beam.sdk.coders.IterableCoder; +import org.apache.beam.sdk.coders.KvCoder; +import org.apache.beam.sdk.values.TypeDescriptor; /** * Utilities for working with Coders. diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/CombineFnUtil.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/CombineFnUtil.java index 351a158c1edfa..a9a0178edf4e6 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/CombineFnUtil.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/CombineFnUtil.java @@ -17,6 +17,9 @@ */ package org.apache.beam.sdk.util; +import java.io.IOException; +import java.io.NotSerializableException; +import java.io.ObjectOutputStream; import org.apache.beam.sdk.coders.CannotProvideCoderException; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.CoderRegistry; @@ -30,10 +33,6 @@ import org.apache.beam.sdk.transforms.display.DisplayData; import org.apache.beam.sdk.util.state.StateContext; -import java.io.IOException; -import java.io.NotSerializableException; -import java.io.ObjectOutputStream; - /** * Static utility methods that create combine function instances. */ diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/CredentialFactory.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/CredentialFactory.java index 3283c8e0c5288..ce29d9192aed8 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/CredentialFactory.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/CredentialFactory.java @@ -18,7 +18,6 @@ package org.apache.beam.sdk.util; import com.google.api.client.auth.oauth2.Credential; - import java.io.IOException; import java.security.GeneralSecurityException; diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/Credentials.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/Credentials.java index 41065cde41143..1e77f4dc324af 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/Credentials.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/Credentials.java @@ -19,8 +19,6 @@ import static com.google.common.base.Preconditions.checkArgument; -import org.apache.beam.sdk.options.GcpOptions; - import com.google.api.client.auth.oauth2.Credential; import com.google.api.client.extensions.java6.auth.oauth2.AbstractPromptReceiver; import com.google.api.client.extensions.java6.auth.oauth2.AuthorizationCodeInstalledApp; @@ -34,10 +32,6 @@ import com.google.api.client.json.JsonFactory; import com.google.api.client.json.jackson2.JacksonFactory; import com.google.api.client.util.store.FileDataStoreFactory; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - import java.io.File; import java.io.FileReader; import java.io.IOException; @@ -45,6 +39,9 @@ import java.util.Arrays; import java.util.Collection; import java.util.List; +import org.apache.beam.sdk.options.GcpOptions; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * Provides support for loading credentials. diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/DirectSideInputReader.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/DirectSideInputReader.java index c8d360c09a8f4..30e371656396c 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/DirectSideInputReader.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/DirectSideInputReader.java @@ -17,14 +17,13 @@ */ package org.apache.beam.sdk.util; +import com.google.common.base.Predicate; +import com.google.common.collect.Iterables; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.GlobalWindows; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TupleTag; -import com.google.common.base.Predicate; -import com.google.common.collect.Iterables; - /** * Basic side input reader wrapping a {@link PTuple} of side input iterables. Encapsulates * conversion according to the {@link PCollectionView} and projection to a particular diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ExecutableTrigger.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ExecutableTrigger.java index bb71fccabddae..088c499daa0b4 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ExecutableTrigger.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ExecutableTrigger.java @@ -20,13 +20,12 @@ import static com.google.common.base.Preconditions.checkNotNull; import static com.google.common.base.Preconditions.checkState; -import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.transforms.windowing.Trigger; -import org.apache.beam.sdk.transforms.windowing.Trigger.OnceTrigger; - import java.io.Serializable; import java.util.ArrayList; import java.util.List; +import org.apache.beam.sdk.transforms.windowing.BoundedWindow; +import org.apache.beam.sdk.transforms.windowing.Trigger; +import org.apache.beam.sdk.transforms.windowing.Trigger.OnceTrigger; /** * A wrapper around a trigger used during execution. While an actual trigger may appear multiple 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 1c2f554559e81..82d900c530374 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 @@ -17,14 +17,13 @@ */ 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.windowing.BoundedWindow; import org.apache.beam.sdk.util.state.StateInternals; import org.apache.beam.sdk.values.TupleTag; -import java.io.IOException; -import java.util.Collection; - /** * Context for the current execution. This is guaranteed to exist during processing, * but does not necessarily persist between different batches of work. diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ExposedByteArrayInputStream.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ExposedByteArrayInputStream.java index eae3e40ad1a7b..020c45308f97d 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ExposedByteArrayInputStream.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ExposedByteArrayInputStream.java @@ -17,11 +17,10 @@ */ package org.apache.beam.sdk.util; +import edu.umd.cs.findbugs.annotations.SuppressFBWarnings; import java.io.ByteArrayInputStream; import java.io.IOException; -import edu.umd.cs.findbugs.annotations.SuppressFBWarnings; - /** * {@link ByteArrayInputStream} that allows accessing the entire internal buffer without copying. */ diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ExposedByteArrayOutputStream.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ExposedByteArrayOutputStream.java index 48006ea22f4d9..5a98f8419c3cd 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ExposedByteArrayOutputStream.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ExposedByteArrayOutputStream.java @@ -17,11 +17,10 @@ */ package org.apache.beam.sdk.util; +import edu.umd.cs.findbugs.annotations.SuppressFBWarnings; import java.io.ByteArrayOutputStream; import java.io.IOException; -import edu.umd.cs.findbugs.annotations.SuppressFBWarnings; - /** * {@link ByteArrayOutputStream} special cased to treat writes of a single byte-array specially. * When calling {@link #toByteArray()} after writing only one {@code byte[]} using diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/FileIOChannelFactory.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/FileIOChannelFactory.java index 92f351bb3af20..a11231bae2f1f 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/FileIOChannelFactory.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/FileIOChannelFactory.java @@ -20,10 +20,6 @@ import com.google.common.base.Predicate; import com.google.common.base.Predicates; import com.google.common.collect.Iterables; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - import java.io.BufferedOutputStream; import java.io.File; import java.io.FileInputStream; @@ -44,6 +40,8 @@ import java.util.LinkedList; import java.util.List; import java.util.regex.Matcher; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * Implements {@link IOChannelFactory} for local files. diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/FinishedTriggersSet.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/FinishedTriggersSet.java index 23aaa5467767a..a9feb7300a17c 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/FinishedTriggersSet.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/FinishedTriggersSet.java @@ -18,7 +18,6 @@ package org.apache.beam.sdk.util; import com.google.common.collect.Sets; - import java.util.Set; /** diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/GcpCredentialFactory.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/GcpCredentialFactory.java index 98b82e2d39aa5..0497e750bbe16 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/GcpCredentialFactory.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/GcpCredentialFactory.java @@ -17,13 +17,11 @@ */ package org.apache.beam.sdk.util; -import org.apache.beam.sdk.options.GcpOptions; -import org.apache.beam.sdk.options.PipelineOptions; - import com.google.api.client.auth.oauth2.Credential; - import java.io.IOException; import java.security.GeneralSecurityException; +import org.apache.beam.sdk.options.GcpOptions; +import org.apache.beam.sdk.options.PipelineOptions; /** * Construct an oauth credential to be used by the SDK and the SDK workers. diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/GcsIOChannelFactory.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/GcsIOChannelFactory.java index 2122c6427bd0d..14090e31c38ed 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/GcsIOChannelFactory.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/GcsIOChannelFactory.java @@ -17,15 +17,14 @@ */ package org.apache.beam.sdk.util; -import org.apache.beam.sdk.options.GcsOptions; -import org.apache.beam.sdk.util.gcsfs.GcsPath; - import java.io.IOException; import java.nio.channels.ReadableByteChannel; import java.nio.channels.WritableByteChannel; import java.util.Collection; import java.util.LinkedList; import java.util.List; +import org.apache.beam.sdk.options.GcsOptions; +import org.apache.beam.sdk.util.gcsfs.GcsPath; /** * Implements IOChannelFactory for GCS. 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 87f9181090597..89363cec4cb23 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 @@ -19,12 +19,11 @@ import static com.google.common.base.Preconditions.checkArgument; +import java.io.IOException; import org.apache.beam.sdk.options.GcsOptions; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.util.gcsfs.GcsPath; -import java.io.IOException; - /** * GCP implementation of {@link PathValidator}. Only GCS paths are allowed. */ 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 06685e577bf93..44a182eeb3d8f 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 @@ -20,11 +20,6 @@ import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.base.Preconditions.checkNotNull; -import org.apache.beam.sdk.options.DefaultValueFactory; -import org.apache.beam.sdk.options.GcsOptions; -import org.apache.beam.sdk.options.PipelineOptions; -import org.apache.beam.sdk.util.gcsfs.GcsPath; - import com.google.api.client.googleapis.batch.BatchRequest; import com.google.api.client.googleapis.batch.json.JsonBatchCallback; import com.google.api.client.googleapis.json.GoogleJsonError; @@ -50,10 +45,6 @@ import com.google.common.util.concurrent.ListenableFuture; import com.google.common.util.concurrent.ListeningExecutorService; import com.google.common.util.concurrent.MoreExecutors; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - import java.io.FileNotFoundException; import java.io.IOException; import java.nio.channels.SeekableByteChannel; @@ -70,8 +61,13 @@ import java.util.concurrent.TimeUnit; import java.util.regex.Matcher; import java.util.regex.Pattern; - import javax.annotation.Nullable; +import org.apache.beam.sdk.options.DefaultValueFactory; +import org.apache.beam.sdk.options.GcsOptions; +import org.apache.beam.sdk.options.PipelineOptions; +import org.apache.beam.sdk.util.gcsfs.GcsPath; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * Provides operations on GCS. diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/IOChannelUtils.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/IOChannelUtils.java index 913824d821a89..16a6e952f6c49 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/IOChannelUtils.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/IOChannelUtils.java @@ -17,9 +17,6 @@ */ package org.apache.beam.sdk.util; -import org.apache.beam.sdk.options.GcsOptions; -import org.apache.beam.sdk.options.PipelineOptions; - import java.io.FileNotFoundException; import java.io.IOException; import java.nio.channels.WritableByteChannel; @@ -32,6 +29,8 @@ import java.util.Set; import java.util.regex.Matcher; import java.util.regex.Pattern; +import org.apache.beam.sdk.options.GcsOptions; +import org.apache.beam.sdk.options.PipelineOptions; /** * Provides utilities for creating read and write channels. 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 a3477e9f4f120..8ca1bfd77b5b7 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 @@ -17,6 +17,8 @@ */ package org.apache.beam.sdk.util; +import java.util.Collection; +import java.util.Collections; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.transforms.GroupByKey; @@ -26,12 +28,8 @@ import org.apache.beam.sdk.transforms.windowing.Window; import org.apache.beam.sdk.transforms.windowing.WindowFn; import org.apache.beam.sdk.values.PCollection; - import org.joda.time.Instant; -import java.util.Collection; -import java.util.Collections; - /** * A {@link WindowFn} that leaves all associations between elements and windows unchanged. * diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/InstanceBuilder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/InstanceBuilder.java index 08e07ce4e6407..e9e542c0853ce 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/InstanceBuilder.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/InstanceBuilder.java @@ -20,18 +20,15 @@ import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.base.Preconditions.checkState; -import org.apache.beam.sdk.values.TypeDescriptor; - import com.google.common.base.Joiner; - import java.lang.reflect.Constructor; import java.lang.reflect.InvocationTargetException; import java.lang.reflect.Method; import java.lang.reflect.Modifier; import java.util.LinkedList; import java.util.List; - import javax.annotation.Nullable; +import org.apache.beam.sdk.values.TypeDescriptor; /** * Utility for creating objects dynamically. diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/KeyedWorkItemCoder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/KeyedWorkItemCoder.java index ec5d821382695..a6e3d6c10b0d6 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/KeyedWorkItemCoder.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/KeyedWorkItemCoder.java @@ -19,6 +19,13 @@ import static com.google.common.base.Preconditions.checkArgument; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.collect.ImmutableList; +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.util.List; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.CoderException; import org.apache.beam.sdk.coders.IterableCoder; @@ -28,16 +35,6 @@ import org.apache.beam.sdk.util.TimerInternals.TimerDataCoder; import org.apache.beam.sdk.util.WindowedValue.FullWindowedValueCoder; -import com.google.common.collect.ImmutableList; - -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonProperty; - -import java.io.IOException; -import java.io.InputStream; -import java.io.OutputStream; -import java.util.List; - /** * A {@link Coder} for {@link KeyedWorkItem KeyedWorkItems}. */ diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/KeyedWorkItems.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/KeyedWorkItems.java index 11d84d083ba17..74348428331f9 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/KeyedWorkItems.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/KeyedWorkItems.java @@ -17,13 +17,11 @@ */ package org.apache.beam.sdk.util; -import org.apache.beam.sdk.util.TimerInternals.TimerData; - 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; /** * Static utility methods that provide {@link KeyedWorkItem} implementations. diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/MergingActiveWindowSet.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/MergingActiveWindowSet.java index 4702fe20460b0..6ad63b0f1a0eb 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/MergingActiveWindowSet.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/MergingActiveWindowSet.java @@ -20,20 +20,9 @@ import static com.google.common.base.Preconditions.checkNotNull; import static com.google.common.base.Preconditions.checkState; -import org.apache.beam.sdk.coders.MapCoder; -import org.apache.beam.sdk.coders.SetCoder; -import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.transforms.windowing.WindowFn; -import org.apache.beam.sdk.util.state.StateInternals; -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 com.google.common.annotations.VisibleForTesting; import com.google.common.collect.Iterables; import com.google.common.collect.Sets; - import java.util.ArrayList; import java.util.Collection; import java.util.HashMap; @@ -44,8 +33,16 @@ import java.util.Map; import java.util.Objects; import java.util.Set; - import javax.annotation.Nullable; +import org.apache.beam.sdk.coders.MapCoder; +import org.apache.beam.sdk.coders.SetCoder; +import org.apache.beam.sdk.transforms.windowing.BoundedWindow; +import org.apache.beam.sdk.transforms.windowing.WindowFn; +import org.apache.beam.sdk.util.state.StateInternals; +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; /** * An {@link ActiveWindowSet} for merging {@link WindowFn} implementations. diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/MovingFunction.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/MovingFunction.java index eb0a91af6f066..7b30d346024ef 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/MovingFunction.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/MovingFunction.java @@ -20,9 +20,8 @@ import static com.google.common.base.Preconditions.checkArgument; -import org.apache.beam.sdk.transforms.Combine; - import java.util.Arrays; +import org.apache.beam.sdk.transforms.Combine; /** * Keep track of the moving minimum/maximum/sum of sampled long values. The minimum/maximum/sum diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/MutationDetectors.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/MutationDetectors.java index ee84c45957c2f..3b593bf0e9448 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/MutationDetectors.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/MutationDetectors.java @@ -17,11 +17,10 @@ */ package org.apache.beam.sdk.util; -import org.apache.beam.sdk.coders.Coder; -import org.apache.beam.sdk.coders.CoderException; - import java.util.Arrays; import java.util.Objects; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.coders.CoderException; /** * Static methods for creating and working with {@link MutationDetector}. diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/NonMergingActiveWindowSet.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/NonMergingActiveWindowSet.java index 15a4ebe0f25e1..99d591bb00b00 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/NonMergingActiveWindowSet.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/NonMergingActiveWindowSet.java @@ -17,14 +17,12 @@ */ package org.apache.beam.sdk.util; -import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.transforms.windowing.WindowFn; - import com.google.common.annotations.VisibleForTesting; import com.google.common.collect.ImmutableSet; - import java.util.Collection; import java.util.Set; +import org.apache.beam.sdk.transforms.windowing.BoundedWindow; +import org.apache.beam.sdk.transforms.windowing.WindowFn; /** * Implementation of {@link ActiveWindowSet} used with {@link WindowFn WindowFns} that don't diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/NoopCredentialFactory.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/NoopCredentialFactory.java index 09f1f9000f0b8..5d9255286a777 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/NoopCredentialFactory.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/NoopCredentialFactory.java @@ -17,12 +17,10 @@ */ package org.apache.beam.sdk.util; -import org.apache.beam.sdk.options.PipelineOptions; - import com.google.api.client.auth.oauth2.Credential; - import java.io.IOException; import java.security.GeneralSecurityException; +import org.apache.beam.sdk.options.PipelineOptions; /** * Construct an oauth credential to be used by the SDK and the SDK workers. diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/NullSideInputReader.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/NullSideInputReader.java index 99319ed142342..3da2dc17b401b 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/NullSideInputReader.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/NullSideInputReader.java @@ -17,13 +17,11 @@ */ package org.apache.beam.sdk.util; -import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.values.PCollectionView; - import com.google.common.collect.Sets; - import java.util.Collections; import java.util.Set; +import org.apache.beam.sdk.transforms.windowing.BoundedWindow; +import org.apache.beam.sdk.values.PCollectionView; /** * A {@link SideInputReader} representing a well-defined set of views, but not storing diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/PCollectionViewWindow.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/PCollectionViewWindow.java index a7311e3767bc0..410c8cea68322 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/PCollectionViewWindow.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/PCollectionViewWindow.java @@ -17,11 +17,10 @@ */ package org.apache.beam.sdk.util; +import java.util.Objects; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.values.PCollectionView; -import java.util.Objects; - /** * A pair of a {@link PCollectionView} and a {@link BoundedWindow}, which can * be thought of as window "of" the view. This is a value class for use e.g. diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/PCollectionViews.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/PCollectionViews.java index 581a98a1a745e..14ae5c8dfcfc9 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/PCollectionViews.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/PCollectionViews.java @@ -17,25 +17,12 @@ */ package org.apache.beam.sdk.util; -import org.apache.beam.sdk.Pipeline; -import org.apache.beam.sdk.coders.Coder; -import org.apache.beam.sdk.coders.IterableCoder; -import org.apache.beam.sdk.transforms.ViewFn; -import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.transforms.windowing.InvalidWindows; -import org.apache.beam.sdk.values.KV; -import org.apache.beam.sdk.values.PCollection; -import org.apache.beam.sdk.values.PCollectionView; -import org.apache.beam.sdk.values.PValueBase; -import org.apache.beam.sdk.values.TupleTag; - import com.google.common.base.Function; import com.google.common.base.MoreObjects; import com.google.common.collect.HashMultimap; import com.google.common.collect.ImmutableList; import com.google.common.collect.Iterables; import com.google.common.collect.Multimap; - import java.io.IOException; import java.util.Collections; import java.util.HashMap; @@ -43,8 +30,18 @@ import java.util.Map; import java.util.NoSuchElementException; import java.util.Objects; - import javax.annotation.Nullable; +import org.apache.beam.sdk.Pipeline; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.coders.IterableCoder; +import org.apache.beam.sdk.transforms.ViewFn; +import org.apache.beam.sdk.transforms.windowing.BoundedWindow; +import org.apache.beam.sdk.transforms.windowing.InvalidWindows; +import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.PCollectionView; +import org.apache.beam.sdk.values.PValueBase; +import org.apache.beam.sdk.values.TupleTag; /** * Implementations of {@link PCollectionView} shared across the SDK. diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/PTuple.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/PTuple.java index eea7bcdc0f471..d03803fd6d490 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/PTuple.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/PTuple.java @@ -17,11 +17,10 @@ */ package org.apache.beam.sdk.util; -import org.apache.beam.sdk.values.TupleTag; - import java.util.Collections; import java.util.LinkedHashMap; import java.util.Map; +import org.apache.beam.sdk.values.TupleTag; /** * A {@code PTuple} is an immutable tuple of diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/PerKeyCombineFnRunner.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/PerKeyCombineFnRunner.java index ae3d391bd8094..e8fa4f7e0c23f 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/PerKeyCombineFnRunner.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/PerKeyCombineFnRunner.java @@ -17,14 +17,13 @@ */ package org.apache.beam.sdk.util; +import java.io.Serializable; +import java.util.Collection; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.CombineFnBase.PerKeyCombineFn; import org.apache.beam.sdk.transforms.OldDoFn; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import java.io.Serializable; -import java.util.Collection; - /** * An interface that runs a {@link PerKeyCombineFn} with unified APIs. * diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/PerKeyCombineFnRunners.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/PerKeyCombineFnRunners.java index 87870a83f3b41..c537eb36a4594 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/PerKeyCombineFnRunners.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/PerKeyCombineFnRunners.java @@ -17,6 +17,8 @@ */ package org.apache.beam.sdk.util; +import com.google.common.collect.Iterables; +import java.util.Collection; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.Combine.KeyedCombineFn; import org.apache.beam.sdk.transforms.CombineFnBase.PerKeyCombineFn; @@ -26,10 +28,6 @@ import org.apache.beam.sdk.transforms.OldDoFn; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import com.google.common.collect.Iterables; - -import java.util.Collection; - /** * Static utility methods that provide {@link PerKeyCombineFnRunner} implementations * for different keyed combine functions. diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/PubsubClient.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/PubsubClient.java index 9e6c7d2249837..fdcee16b69add 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/PubsubClient.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/PubsubClient.java @@ -21,12 +21,9 @@ import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.base.Preconditions.checkState; -import org.apache.beam.sdk.options.PubsubOptions; - import com.google.api.client.util.DateTime; import com.google.common.base.Objects; import com.google.common.base.Strings; - import java.io.Closeable; import java.io.IOException; import java.io.Serializable; @@ -34,8 +31,8 @@ import java.util.List; import java.util.Map; import java.util.concurrent.ThreadLocalRandom; - import javax.annotation.Nullable; +import org.apache.beam.sdk.options.PubsubOptions; /** * An (abstract) helper class for talking to Pubsub via an underlying transport. diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/PubsubGrpcClient.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/PubsubGrpcClient.java index ac157fb803098..988b90fec63cb 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/PubsubGrpcClient.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/PubsubGrpcClient.java @@ -20,9 +20,6 @@ import static com.google.common.base.Preconditions.checkState; -import org.apache.beam.sdk.options.GcpOptions; -import org.apache.beam.sdk.options.PubsubOptions; - import com.google.auth.oauth2.GoogleCredentials; import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Strings; @@ -50,7 +47,6 @@ import com.google.pubsub.v1.SubscriberGrpc.SubscriberBlockingStub; import com.google.pubsub.v1.Subscription; import com.google.pubsub.v1.Topic; - import io.grpc.Channel; import io.grpc.ClientInterceptors; import io.grpc.ManagedChannel; @@ -58,7 +54,6 @@ import io.grpc.netty.GrpcSslContexts; import io.grpc.netty.NegotiationType; import io.grpc.netty.NettyChannelBuilder; - import java.io.IOException; import java.util.ArrayList; import java.util.Collections; @@ -66,8 +61,9 @@ import java.util.Map; import java.util.concurrent.Executors; import java.util.concurrent.TimeUnit; - import javax.annotation.Nullable; +import org.apache.beam.sdk.options.GcpOptions; +import org.apache.beam.sdk.options.PubsubOptions; /** * A helper class for talking to Pubsub via grpc. diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/PubsubJsonClient.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/PubsubJsonClient.java index 69c5128df8e07..bdb5c04f8cbf9 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/PubsubJsonClient.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/PubsubJsonClient.java @@ -20,8 +20,6 @@ import static com.google.common.base.Preconditions.checkState; -import org.apache.beam.sdk.options.PubsubOptions; - import com.google.api.client.auth.oauth2.Credential; import com.google.api.client.http.HttpRequestInitializer; import com.google.api.services.pubsub.Pubsub; @@ -42,14 +40,13 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Strings; import com.google.common.collect.ImmutableList; - import java.io.IOException; import java.util.ArrayList; import java.util.List; import java.util.Map; import java.util.TreeMap; - import javax.annotation.Nullable; +import org.apache.beam.sdk.options.PubsubOptions; /** * A Pubsub client using JSON transport. diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/PubsubTestClient.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/PubsubTestClient.java index 88ae6cc8cf101..6e5ba46a7f0cc 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/PubsubTestClient.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/PubsubTestClient.java @@ -20,12 +20,9 @@ import static com.google.common.base.Preconditions.checkState; -import org.apache.beam.sdk.options.PubsubOptions; - import com.google.api.client.util.Clock; import com.google.common.collect.Lists; import com.google.common.collect.Sets; - import java.io.Closeable; import java.io.IOException; import java.util.ArrayList; @@ -34,8 +31,8 @@ import java.util.List; import java.util.Map; import java.util.Set; - import javax.annotation.Nullable; +import org.apache.beam.sdk.options.PubsubOptions; /** * A (partial) implementation of {@link PubsubClient} for use by unit tests. Only suitable for 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 ad03c79e01c82..2c0ae4099c804 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 @@ -19,13 +19,11 @@ import com.google.api.client.json.GenericJson; import com.google.api.client.util.Key; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - import java.io.IOException; import java.io.InputStream; import java.util.Properties; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * Utilities for working with release information. 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 ad33a252e0a73..ebd44bf4ab03e 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 @@ -25,7 +25,6 @@ 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; /** diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ReshuffleTrigger.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ReshuffleTrigger.java index f529ce160e2f4..9e2c27d1bc7f1 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ReshuffleTrigger.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ReshuffleTrigger.java @@ -17,13 +17,11 @@ */ package org.apache.beam.sdk.util; +import java.util.List; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.Trigger; - import org.joda.time.Instant; -import java.util.List; - /** * The trigger used with {@link Reshuffle} which triggers on every element * and never buffers state. diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/RetryHttpRequestInitializer.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/RetryHttpRequestInitializer.java index 3857435b08f9f..fa6e9136cd280 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/RetryHttpRequestInitializer.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/RetryHttpRequestInitializer.java @@ -28,18 +28,15 @@ import com.google.api.client.util.ExponentialBackOff; import com.google.api.client.util.NanoClock; import com.google.api.client.util.Sleeper; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - import java.io.IOException; import java.util.Arrays; import java.util.Collection; import java.util.Collections; import java.util.HashSet; import java.util.Set; - import javax.annotation.Nullable; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * Implements a request initializer that adds retry handlers to all diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/SerializableUtils.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/SerializableUtils.java index 1e70aaf021b70..354aa5d91182d 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/SerializableUtils.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/SerializableUtils.java @@ -17,17 +17,10 @@ */ package org.apache.beam.sdk.util; +import static com.google.common.base.Preconditions.checkState; import static org.apache.beam.sdk.util.CoderUtils.decodeFromByteArray; import static org.apache.beam.sdk.util.CoderUtils.encodeToByteArray; -import static com.google.common.base.Preconditions.checkState; - -import org.apache.beam.sdk.coders.Coder; -import org.apache.beam.sdk.coders.CoderException; - -import org.xerial.snappy.SnappyInputStream; -import org.xerial.snappy.SnappyOutputStream; - import java.io.ByteArrayInputStream; import java.io.ByteArrayOutputStream; import java.io.IOException; @@ -35,6 +28,10 @@ import java.io.ObjectOutputStream; import java.io.Serializable; import java.util.Arrays; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.coders.CoderException; +import org.xerial.snappy.SnappyInputStream; +import org.xerial.snappy.SnappyOutputStream; /** * Utilities for working with Serializables. diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/Serializer.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/Serializer.java index 738577d17340b..86a3b8ecc9bf6 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/Serializer.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/Serializer.java @@ -20,12 +20,10 @@ import com.fasterxml.jackson.core.JsonProcessingException; import com.fasterxml.jackson.databind.DeserializationFeature; import com.fasterxml.jackson.databind.ObjectMapper; - import java.util.ArrayList; import java.util.HashMap; import java.util.List; import java.util.Map; - import javax.annotation.Nullable; /** diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/SideInputReader.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/SideInputReader.java index e99eb57d9a38e..e81c704cf1967 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/SideInputReader.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/SideInputReader.java @@ -17,11 +17,10 @@ */ package org.apache.beam.sdk.util; +import javax.annotation.Nullable; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.values.PCollectionView; -import javax.annotation.Nullable; - /** * The interface to objects that provide side inputs. Particular implementations * may read a side input directly or use appropriate sorts of caching, etc. diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/StringUtils.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/StringUtils.java index bb59373571578..4f81eef4fc7c4 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/StringUtils.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/StringUtils.java @@ -20,14 +20,12 @@ import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.base.Preconditions.checkNotNull; -import org.apache.beam.sdk.transforms.PTransform; - import com.google.common.base.Joiner; - import java.util.ArrayList; import java.util.List; import java.util.regex.Matcher; import java.util.regex.Pattern; +import org.apache.beam.sdk.transforms.PTransform; /** * Utilities for working with JSON and other human-readable string formats. diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/Structs.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/Structs.java index d6b704bf902e8..d50b74a0526f0 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/Structs.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/Structs.java @@ -18,13 +18,11 @@ package org.apache.beam.sdk.util; import com.google.api.client.util.Data; - import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; import java.util.List; import java.util.Map; - import javax.annotation.Nullable; /** diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/SystemDoFnInternal.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/SystemDoFnInternal.java index b8a5cd4eb51d3..e9904b2d7660a 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/SystemDoFnInternal.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/SystemDoFnInternal.java @@ -17,13 +17,12 @@ */ package org.apache.beam.sdk.util; -import org.apache.beam.sdk.transforms.OldDoFn; - import java.lang.annotation.Documented; import java.lang.annotation.ElementType; import java.lang.annotation.Retention; import java.lang.annotation.RetentionPolicy; import java.lang.annotation.Target; +import org.apache.beam.sdk.transforms.OldDoFn; /** * Annotation to mark {@link OldDoFn DoFns} as an internal component of the Dataflow SDK. diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/TestCredential.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/TestCredential.java index 554ac1b8cb5f2..4b81a0ef54f21 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/TestCredential.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/TestCredential.java @@ -21,7 +21,6 @@ import com.google.api.client.auth.oauth2.Credential; import com.google.api.client.auth.oauth2.TokenResponse; import com.google.api.client.testing.http.MockHttpTransport; - import java.io.IOException; /** 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 eb49b9dd6facd..dd3b773312347 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 @@ -20,30 +20,26 @@ import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.base.Preconditions.checkNotNull; -import org.apache.beam.sdk.coders.Coder; -import org.apache.beam.sdk.coders.CoderException; -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.windowing.BoundedWindow; -import org.apache.beam.sdk.util.state.StateNamespace; -import org.apache.beam.sdk.util.state.StateNamespaces; - -import com.google.common.base.MoreObjects; -import com.google.common.collect.ComparisonChain; - import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; -import org.joda.time.Instant; - +import com.google.common.base.MoreObjects; +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; +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.windowing.BoundedWindow; +import org.apache.beam.sdk.util.state.StateNamespace; +import org.apache.beam.sdk.util.state.StateNamespaces; +import org.joda.time.Instant; /** * Encapsulate interaction with time within the execution environment. diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/Timers.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/Timers.java index e8404bda26f58..beb80ec3d51e5 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/Timers.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/Timers.java @@ -17,12 +17,10 @@ */ package org.apache.beam.sdk.util; +import javax.annotation.Nullable; import org.apache.beam.sdk.annotations.Experimental; - import org.joda.time.Instant; -import javax.annotation.Nullable; - /** * Interface for interacting with time. */ diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/Transport.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/Transport.java index 2c68dfcd3b7a7..d824207a72339 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/Transport.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/Transport.java @@ -17,10 +17,6 @@ */ package org.apache.beam.sdk.util; -import org.apache.beam.sdk.options.BigQueryOptions; -import org.apache.beam.sdk.options.GcsOptions; -import org.apache.beam.sdk.options.PubsubOptions; - import com.google.api.client.auth.oauth2.Credential; import com.google.api.client.googleapis.javanet.GoogleNetHttpTransport; import com.google.api.client.http.HttpRequestInitializer; @@ -32,11 +28,13 @@ import com.google.api.services.storage.Storage; import com.google.cloud.hadoop.util.ChainingHttpRequestInitializer; import com.google.common.collect.ImmutableList; - import java.io.IOException; import java.net.MalformedURLException; import java.net.URL; import java.security.GeneralSecurityException; +import org.apache.beam.sdk.options.BigQueryOptions; +import org.apache.beam.sdk.options.GcsOptions; +import org.apache.beam.sdk.options.PubsubOptions; /** * Helpers for cloud communication. diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/TriggerContextFactory.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/TriggerContextFactory.java index 03f1baa8b4939..e09aac2eee751 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/TriggerContextFactory.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/TriggerContextFactory.java @@ -17,6 +17,13 @@ */ package org.apache.beam.sdk.util; +import com.google.common.base.Predicate; +import com.google.common.collect.FluentIterable; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Iterables; +import java.util.Collection; +import java.util.Map; +import javax.annotation.Nullable; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.Trigger; @@ -30,19 +37,8 @@ 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 com.google.common.base.Predicate; -import com.google.common.collect.FluentIterable; -import com.google.common.collect.ImmutableMap; -import com.google.common.collect.Iterables; - import org.joda.time.Instant; -import java.util.Collection; -import java.util.Map; - -import javax.annotation.Nullable; - /** * Factory for creating instances of the various {@link Trigger} contexts. * diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/UnownedInputStream.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/UnownedInputStream.java index 58c6617e818ac..e7e7bbd20359d 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/UnownedInputStream.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/UnownedInputStream.java @@ -18,7 +18,6 @@ package org.apache.beam.sdk.util; import com.google.common.base.MoreObjects; - import java.io.FilterInputStream; import java.io.IOException; import java.io.InputStream; diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/UnownedOutputStream.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/UnownedOutputStream.java index c2f30c6957945..3c9b3bc91c6d4 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/UnownedOutputStream.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/UnownedOutputStream.java @@ -18,7 +18,6 @@ package org.apache.beam.sdk.util; import com.google.common.base.MoreObjects; - import java.io.FilterOutputStream; import java.io.IOException; import java.io.OutputStream; diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/UploadIdResponseInterceptor.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/UploadIdResponseInterceptor.java index caa0759fddbb0..f685b6974441f 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/UploadIdResponseInterceptor.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/UploadIdResponseInterceptor.java @@ -20,12 +20,10 @@ import com.google.api.client.http.GenericUrl; import com.google.api.client.http.HttpResponse; import com.google.api.client.http.HttpResponseInterceptor; - +import java.io.IOException; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.io.IOException; - /** * Implements a response intercepter that logs the upload id if the upload * id header exists and it is the first request (does not have upload_id parameter in the request). diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ValueWithRecordId.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ValueWithRecordId.java index f0e48124425e2..d6b5fe3b4b382 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ValueWithRecordId.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ValueWithRecordId.java @@ -19,22 +19,19 @@ import static com.google.common.base.Preconditions.checkArgument; -import org.apache.beam.sdk.coders.ByteArrayCoder; -import org.apache.beam.sdk.coders.Coder; -import org.apache.beam.sdk.coders.StandardCoder; -import org.apache.beam.sdk.transforms.OldDoFn; - -import com.google.common.base.MoreObjects; - import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; - +import com.google.common.base.MoreObjects; 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 org.apache.beam.sdk.coders.ByteArrayCoder; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.coders.StandardCoder; +import org.apache.beam.sdk.transforms.OldDoFn; /** * Immutable struct containing a value as well as a unique id identifying the value. diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/Values.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/Values.java index 71110e1cbe2e2..d8aa046e07901 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/Values.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/Values.java @@ -18,7 +18,6 @@ package org.apache.beam.sdk.util; import java.util.Map; - import javax.annotation.Nullable; /** 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 9d341a1d611e9..a0b4cf5260962 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 @@ -17,29 +17,14 @@ */ package org.apache.beam.sdk.util; -import static org.apache.beam.sdk.util.Structs.addBoolean; - import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.base.Preconditions.checkNotNull; - -import org.apache.beam.sdk.coders.Coder; -import org.apache.beam.sdk.coders.CoderException; -import org.apache.beam.sdk.coders.CollectionCoder; -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.GlobalWindow; -import org.apache.beam.sdk.transforms.windowing.PaneInfo; -import org.apache.beam.sdk.transforms.windowing.PaneInfo.PaneInfoCoder; -import org.apache.beam.sdk.util.common.ElementByteSizeObserver; - -import com.google.common.base.MoreObjects; -import com.google.common.collect.ImmutableList; +import static org.apache.beam.sdk.util.Structs.addBoolean; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; -import org.joda.time.Instant; - +import com.google.common.base.MoreObjects; +import com.google.common.collect.ImmutableList; import java.io.IOException; import java.io.InputStream; import java.io.OutputStream; @@ -50,6 +35,17 @@ import java.util.List; import java.util.Objects; import java.util.Set; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.coders.CoderException; +import org.apache.beam.sdk.coders.CollectionCoder; +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.GlobalWindow; +import org.apache.beam.sdk.transforms.windowing.PaneInfo; +import org.apache.beam.sdk.transforms.windowing.PaneInfo.PaneInfoCoder; +import org.apache.beam.sdk.util.common.ElementByteSizeObserver; +import org.joda.time.Instant; /** * An immutable triple of value, timestamp, and windows. 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 3a1b6543d513d..54158d24e5208 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,18 +17,16 @@ */ 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.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.util.state.StateInternals; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TupleTag; - import org.joda.time.Instant; -import java.io.IOException; -import java.util.Collection; - /** * Interface that may be required by some (internal) {@code OldDoFn}s to implement windowing. It * should not be necessary for general user code to interact with this at all. diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/WindowingStrategy.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/WindowingStrategy.java index f5ae812636204..b332ed7c8e0ce 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/WindowingStrategy.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/WindowingStrategy.java @@ -17,6 +17,10 @@ */ package org.apache.beam.sdk.util; +import com.google.common.base.MoreObjects; +import java.io.Serializable; +import java.util.Collections; +import java.util.Objects; import org.apache.beam.sdk.annotations.Experimental; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.DefaultTrigger; @@ -27,16 +31,9 @@ import org.apache.beam.sdk.transforms.windowing.Trigger; import org.apache.beam.sdk.transforms.windowing.Window.ClosingBehavior; import org.apache.beam.sdk.transforms.windowing.WindowFn; - -import com.google.common.base.MoreObjects; - import org.joda.time.Duration; import org.joda.time.Instant; -import java.io.Serializable; -import java.util.Collections; -import java.util.Objects; - /** * A {@code WindowingStrategy} describes the windowing behavior for a specific collection of values. * It has both a {@link WindowFn} describing how elements are assigned to windows and a diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ZipFiles.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ZipFiles.java index c93e18d7706d8..c17de3722e7ce 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ZipFiles.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ZipFiles.java @@ -26,7 +26,7 @@ import com.google.common.io.CharSource; import com.google.common.io.Closer; import com.google.common.io.Files; - +import edu.umd.cs.findbugs.annotations.SuppressFBWarnings; import java.io.BufferedOutputStream; import java.io.File; import java.io.FileOutputStream; @@ -40,8 +40,6 @@ import java.util.zip.ZipFile; import java.util.zip.ZipOutputStream; -import edu.umd.cs.findbugs.annotations.SuppressFBWarnings; - /** * Functions for zipping a directory (including a subdirectory) into a ZIP-file * or unzipping it again. 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 6db532e5811f2..2034ebaef6c84 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 @@ -19,7 +19,6 @@ import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.base.Preconditions.checkNotNull; - import static java.util.Arrays.asList; import com.google.common.base.Function; @@ -27,7 +26,6 @@ import com.google.common.collect.FluentIterable; import com.google.common.collect.ImmutableSet; import com.google.common.collect.Queues; - import java.lang.reflect.GenericArrayType; import java.lang.reflect.Method; import java.lang.reflect.ParameterizedType; @@ -38,7 +36,6 @@ import java.util.Collections; import java.util.LinkedHashSet; import java.util.Queue; - import javax.annotation.Nonnull; import javax.annotation.Nullable; diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/gcsfs/GcsPath.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/gcsfs/GcsPath.java index e629d643715aa..bfcd6da681410 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/gcsfs/GcsPath.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/gcsfs/GcsPath.java @@ -21,7 +21,6 @@ import static com.google.common.base.Strings.isNullOrEmpty; import com.google.api.services.storage.model.StorageObject; - import java.io.File; import java.io.IOException; import java.net.URI; @@ -35,7 +34,6 @@ import java.util.Iterator; import java.util.regex.Matcher; import java.util.regex.Pattern; - import javax.annotation.Nonnull; import javax.annotation.Nullable; diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/CopyOnAccessInMemoryStateInternals.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/CopyOnAccessInMemoryStateInternals.java index 3cc34a694077f..da9aa3ef5ebd1 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/CopyOnAccessInMemoryStateInternals.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/CopyOnAccessInMemoryStateInternals.java @@ -19,6 +19,12 @@ import static com.google.common.base.Preconditions.checkState; +import com.google.common.base.Optional; +import com.google.common.collect.Iterables; +import java.util.Collection; +import java.util.HashSet; +import java.util.Map; +import javax.annotation.Nullable; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.transforms.Combine.CombineFn; import org.apache.beam.sdk.transforms.Combine.KeyedCombineFn; @@ -28,18 +34,8 @@ import org.apache.beam.sdk.util.CombineFnUtil; import org.apache.beam.sdk.util.state.InMemoryStateInternals.InMemoryState; import org.apache.beam.sdk.util.state.StateTag.StateBinder; - -import com.google.common.base.Optional; -import com.google.common.collect.Iterables; - import org.joda.time.Instant; -import java.util.Collection; -import java.util.HashSet; -import java.util.Map; - -import javax.annotation.Nullable; - /** * {@link StateInternals} built on top of an underlying {@link StateTable} that contains instances * of {@link InMemoryState}. Whenever state that exists in the underlying {@link StateTable} is 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 1d5d4325dad27..efb270cfa9ec5 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 @@ -17,6 +17,11 @@ */ package org.apache.beam.sdk.util.state; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.Objects; +import javax.annotation.Nullable; import org.apache.beam.sdk.annotations.Experimental; import org.apache.beam.sdk.annotations.Experimental.Kind; import org.apache.beam.sdk.coders.Coder; @@ -27,16 +32,8 @@ import org.apache.beam.sdk.transforms.windowing.OutputTimeFn; import org.apache.beam.sdk.util.CombineFnUtil; import org.apache.beam.sdk.util.state.StateTag.StateBinder; - import org.joda.time.Instant; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.List; -import java.util.Objects; - -import javax.annotation.Nullable; - /** * In-memory implementation of {@link StateInternals}. Used in {@code BatchModeExecutionContext} * and for running tests that need state. diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/MergingStateAccessor.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/MergingStateAccessor.java index 26af7a324dcc2..fb877cad78f9d 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/MergingStateAccessor.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/MergingStateAccessor.java @@ -17,12 +17,11 @@ */ package org.apache.beam.sdk.util.state; +import java.util.Map; import org.apache.beam.sdk.annotations.Experimental; import org.apache.beam.sdk.annotations.Experimental.Kind; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import java.util.Map; - /** * Interface for accessing persistent state while windows are merging. * diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/StateContexts.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/StateContexts.java index 77bb4eb9c4a6d..d0c566d7accf5 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/StateContexts.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/StateContexts.java @@ -17,13 +17,12 @@ */ package org.apache.beam.sdk.util.state; +import javax.annotation.Nullable; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.util.WindowingInternals; import org.apache.beam.sdk.values.PCollectionView; -import javax.annotation.Nullable; - /** * Factory that produces {@link StateContext} based on different inputs. */ diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/StateInternalsFactory.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/StateInternalsFactory.java index 05c3b77c13847..019b69d5fd29a 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/StateInternalsFactory.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/StateInternalsFactory.java @@ -17,11 +17,10 @@ */ 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 java.io.Serializable; - /** * A factory for providing {@link StateInternals} for a particular key. * diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/StateMerging.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/StateMerging.java index c41a23e7115b9..12baff9312c0e 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/StateMerging.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/StateMerging.java @@ -19,16 +19,13 @@ import static com.google.common.base.Preconditions.checkState; -import org.apache.beam.sdk.transforms.windowing.BoundedWindow; - -import org.joda.time.Instant; - +import edu.umd.cs.findbugs.annotations.SuppressFBWarnings; import java.util.ArrayList; import java.util.Collection; import java.util.List; import java.util.Map; - -import edu.umd.cs.findbugs.annotations.SuppressFBWarnings; +import org.apache.beam.sdk.transforms.windowing.BoundedWindow; +import org.joda.time.Instant; /** * Helpers for merging state. diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/StateNamespaces.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/StateNamespaces.java index bfee9765b6610..75f555cfe73a5 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/StateNamespaces.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/StateNamespaces.java @@ -17,16 +17,14 @@ */ package org.apache.beam.sdk.util.state; -import org.apache.beam.sdk.coders.Coder; -import org.apache.beam.sdk.coders.CoderException; -import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.CoderUtils; - import com.google.common.base.Splitter; - import java.io.IOException; import java.util.List; import java.util.Objects; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.coders.CoderException; +import org.apache.beam.sdk.transforms.windowing.BoundedWindow; +import org.apache.beam.sdk.util.CoderUtils; /** * Factory methods for creating the {@link StateNamespace StateNamespaces}. diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/StateTable.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/StateTable.java index 2ae651679d9eb..dcd2a7eb00db5 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/StateTable.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/StateTable.java @@ -17,13 +17,11 @@ */ package org.apache.beam.sdk.util.state; -import org.apache.beam.sdk.util.state.StateTag.StateBinder; - import com.google.common.collect.HashBasedTable; import com.google.common.collect.Table; - import java.util.Map; import java.util.Set; +import org.apache.beam.sdk.util.state.StateTag.StateBinder; /** * Table mapping {@code StateNamespace} and {@code StateTag} to a {@code State} instance. diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/StateTag.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/StateTag.java index 388b5e1cc7f90..94cba2f5a109b 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/StateTag.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/StateTag.java @@ -17,6 +17,8 @@ */ package org.apache.beam.sdk.util.state; +import java.io.IOException; +import java.io.Serializable; import org.apache.beam.sdk.annotations.Experimental; import org.apache.beam.sdk.annotations.Experimental.Kind; import org.apache.beam.sdk.coders.Coder; @@ -27,9 +29,6 @@ import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.OutputTimeFn; -import java.io.IOException; -import java.io.Serializable; - /** * An address for persistent state. This includes a unique identifier for the location, the * information necessary to encode the value, and details about the intended access pattern. diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/StateTags.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/StateTags.java index e50ad8d00f9d0..b0797b6fac208 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/StateTags.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/StateTags.java @@ -17,6 +17,10 @@ */ package org.apache.beam.sdk.util.state; +import com.google.common.base.MoreObjects; +import java.io.IOException; +import java.io.Serializable; +import java.util.Objects; import org.apache.beam.sdk.annotations.Experimental; import org.apache.beam.sdk.annotations.Experimental.Kind; import org.apache.beam.sdk.coders.CannotProvideCoderException; @@ -28,12 +32,6 @@ import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.OutputTimeFn; -import com.google.common.base.MoreObjects; - -import java.io.IOException; -import java.io.Serializable; -import java.util.Objects; - /** * Static utility methods for creating {@link StateTag} instances. */ diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/WatermarkHoldState.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/WatermarkHoldState.java index f22e9e06f6ca2..415cc6e143e5a 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/WatermarkHoldState.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/state/WatermarkHoldState.java @@ -21,7 +21,6 @@ import org.apache.beam.sdk.annotations.Experimental.Kind; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.OutputTimeFn; - import org.joda.time.Instant; /** diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/KV.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/KV.java index 1806178736375..dec9a16709e0a 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/KV.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/KV.java @@ -17,17 +17,15 @@ */ package org.apache.beam.sdk.values; -import org.apache.beam.sdk.transforms.Combine; -import org.apache.beam.sdk.transforms.GroupByKey; -import org.apache.beam.sdk.transforms.PTransform; -import org.apache.beam.sdk.transforms.SerializableComparator; - import com.google.common.base.MoreObjects; - import java.io.Serializable; import java.util.Arrays; import java.util.Comparator; import java.util.Objects; +import org.apache.beam.sdk.transforms.Combine; +import org.apache.beam.sdk.transforms.GroupByKey; +import org.apache.beam.sdk.transforms.PTransform; +import org.apache.beam.sdk.transforms.SerializableComparator; /** * An immutable key/value pair. 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 e6a2d1f0b9911..f1dbb374a2c39 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,14 +17,13 @@ */ package org.apache.beam.sdk.values; +import java.util.Collection; +import java.util.Collections; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.io.TextIO.Read; import org.apache.beam.sdk.transforms.Create; import org.apache.beam.sdk.transforms.PTransform; -import java.util.Collection; -import java.util.Collections; - /** * {@link PBegin} is the "input" to a root {@link PTransform}, such as {@link Read Read} or * {@link Create}. 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 01acca7097d21..4c9e220ea0ef3 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 @@ -17,19 +17,17 @@ */ package org.apache.beam.sdk.values; -import org.apache.beam.sdk.Pipeline; -import org.apache.beam.sdk.transforms.AppliedPTransform; -import org.apache.beam.sdk.transforms.Flatten; -import org.apache.beam.sdk.transforms.PTransform; -import org.apache.beam.sdk.transforms.Partition; - import com.google.common.collect.ImmutableList; - import java.util.ArrayList; import java.util.Collection; import java.util.Collections; import java.util.Iterator; import java.util.List; +import org.apache.beam.sdk.Pipeline; +import org.apache.beam.sdk.transforms.AppliedPTransform; +import org.apache.beam.sdk.transforms.Flatten; +import org.apache.beam.sdk.transforms.PTransform; +import org.apache.beam.sdk.transforms.Partition; /** * A {@link PCollectionList PCollectionList<T>} is an immutable list of homogeneously 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 b44499bac1d06..f6776f07979a3 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,6 +17,11 @@ */ package org.apache.beam.sdk.values; +import com.google.common.collect.ImmutableMap; +import java.util.Collection; +import java.util.Collections; +import java.util.LinkedHashMap; +import java.util.Map; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.transforms.AppliedPTransform; import org.apache.beam.sdk.transforms.PTransform; @@ -24,13 +29,6 @@ import org.apache.beam.sdk.util.WindowingStrategy; import org.apache.beam.sdk.values.PCollection.IsBounded; -import com.google.common.collect.ImmutableMap; - -import java.util.Collection; -import java.util.Collections; -import java.util.LinkedHashMap; -import java.util.Map; - /** * A {@link PCollectionTuple} is an immutable tuple of * heterogeneously-typed {@link PCollection PCollections}, "keyed" by diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PCollectionView.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PCollectionView.java index 20f10718b8f3e..0e5f5949f242e 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PCollectionView.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PCollectionView.java @@ -17,6 +17,7 @@ */ package org.apache.beam.sdk.values; +import java.io.Serializable; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.transforms.ParDo; import org.apache.beam.sdk.transforms.View; @@ -24,8 +25,6 @@ import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.WindowingStrategy; -import java.io.Serializable; - /** * A {@link PCollectionView PCollectionView<T>} is an immutable view of a {@link PCollection} * as a value of type {@code T} that can be accessed 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 7c05703442878..83d6a92328185 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,11 +17,10 @@ */ package org.apache.beam.sdk.values; -import org.apache.beam.sdk.Pipeline; -import org.apache.beam.sdk.transforms.PTransform; - import java.util.Collection; import java.util.Collections; +import org.apache.beam.sdk.Pipeline; +import org.apache.beam.sdk.transforms.PTransform; /** * {@link PDone} is the output of a {@link PTransform} that has a trivial result, 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 3faf6b9b80971..98987cda9822b 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,9 +17,8 @@ */ package org.apache.beam.sdk.values; -import org.apache.beam.sdk.Pipeline; - import java.util.Collection; +import org.apache.beam.sdk.Pipeline; /** * The interface for things that might be input to a 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 299d55dc5e568..6be9215d6df17 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,12 +17,11 @@ */ package org.apache.beam.sdk.values; +import java.util.Collection; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.transforms.AppliedPTransform; import org.apache.beam.sdk.transforms.PTransform; -import java.util.Collection; - /** * The interface for things that might be output from a {@link PTransform}. */ 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 8a92fa345f896..685e32fe15b79 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,14 +17,13 @@ */ package org.apache.beam.sdk.values; +import java.util.Collection; +import java.util.Collections; 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.StringUtils; -import java.util.Collection; -import java.util.Collections; - /** * A {@link PValueBase} is an abstract base class that provides * sensible default implementations for methods of {@link PValue}. diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/TimestampedValue.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/TimestampedValue.java index 69bf77de9c15e..f2ad6168bb652 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/TimestampedValue.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/TimestampedValue.java @@ -20,21 +20,19 @@ import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.base.Preconditions.checkNotNull; -import org.apache.beam.sdk.coders.Coder; -import org.apache.beam.sdk.coders.InstantCoder; -import org.apache.beam.sdk.coders.StandardCoder; -import org.apache.beam.sdk.util.PropertyNames; - import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; -import org.joda.time.Instant; - 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 org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.coders.InstantCoder; +import org.apache.beam.sdk.coders.StandardCoder; +import org.apache.beam.sdk.util.PropertyNames; +import org.joda.time.Instant; /** * An immutable pair of a value and a timestamp. diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/TupleTag.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/TupleTag.java index b281a43e4f2b6..a6b63ab1aa3ab 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/TupleTag.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/TupleTag.java @@ -20,18 +20,15 @@ import static org.apache.beam.sdk.util.Structs.addBoolean; import static org.apache.beam.sdk.util.Structs.addString; -import org.apache.beam.sdk.transforms.ParDo; -import org.apache.beam.sdk.util.CloudObject; -import org.apache.beam.sdk.util.PropertyNames; - -import com.google.common.collect.HashMultiset; -import com.google.common.collect.Multiset; - import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; - +import com.google.common.collect.HashMultiset; +import com.google.common.collect.Multiset; import java.io.Serializable; import java.util.Random; +import org.apache.beam.sdk.transforms.ParDo; +import org.apache.beam.sdk.util.CloudObject; +import org.apache.beam.sdk.util.PropertyNames; /** * A {@link TupleTag} is a typed tag to use as the key of a diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/TupleTagList.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/TupleTagList.java index 8a4beb3ab5053..412678b5b0f56 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/TupleTagList.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/TupleTagList.java @@ -17,14 +17,12 @@ */ package org.apache.beam.sdk.values; -import org.apache.beam.sdk.transforms.ParDo; - import com.google.common.collect.ImmutableList; - import java.io.Serializable; import java.util.ArrayList; import java.util.Collections; import java.util.List; +import org.apache.beam.sdk.transforms.ParDo; /** * A {@link TupleTagList} is an immutable list of heterogeneously diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/TypeDescriptor.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/TypeDescriptor.java index eea720aa20153..724b8b6925cae 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/TypeDescriptor.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/TypeDescriptor.java @@ -22,7 +22,6 @@ import com.google.common.reflect.Parameter; import com.google.common.reflect.TypeParameter; import com.google.common.reflect.TypeToken; - import java.io.Serializable; import java.lang.reflect.Field; import java.lang.reflect.Method; @@ -30,7 +29,6 @@ import java.lang.reflect.Type; import java.lang.reflect.TypeVariable; import java.util.List; - import javax.annotation.Nullable; /** diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/TypeDescriptors.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/TypeDescriptors.java index 0e7b9cb4397a2..84b69d7a03505 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/TypeDescriptors.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/TypeDescriptors.java @@ -18,7 +18,6 @@ package org.apache.beam.sdk.values; import com.google.common.reflect.TypeParameter; - import java.math.BigDecimal; import java.math.BigInteger; import java.util.List; diff --git a/sdks/java/core/src/test/java/org/apache/beam/runners/dataflow/util/GcsPathValidatorTest.java b/sdks/java/core/src/test/java/org/apache/beam/runners/dataflow/util/GcsPathValidatorTest.java index 8913916fde42e..398fa6333a43f 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/runners/dataflow/util/GcsPathValidatorTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/runners/dataflow/util/GcsPathValidatorTest.java @@ -30,7 +30,6 @@ import org.apache.beam.sdk.util.GcsUtil; import org.apache.beam.sdk.util.TestCredential; import org.apache.beam.sdk.util.gcsfs.GcsPath; - import org.junit.Before; import org.junit.Rule; import org.junit.Test; 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 930fbe781b05b..335d81f47bbd3 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 @@ -23,6 +23,12 @@ import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableSet; +import java.util.Collection; +import java.util.List; +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.transforms.Aggregator; @@ -33,10 +39,6 @@ import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.ParDo; import org.apache.beam.sdk.transforms.Sum; - -import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableSet; - import org.junit.Before; import org.junit.Test; import org.junit.runner.RunWith; @@ -47,11 +49,6 @@ import org.mockito.invocation.InvocationOnMock; import org.mockito.stubbing.Answer; -import java.util.Collection; -import java.util.List; -import java.util.Map; -import java.util.UUID; - /** * Tests for {@link AggregatorPipelineExtractor}. */ diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/DataflowMatchers.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/DataflowMatchers.java index 2d2a373d8e9a0..c76dcf8cbf2a3 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/DataflowMatchers.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/DataflowMatchers.java @@ -18,12 +18,10 @@ package org.apache.beam.sdk; import com.google.protobuf.ByteString; - +import java.io.Serializable; import org.hamcrest.Description; import org.hamcrest.TypeSafeMatcher; -import java.io.Serializable; - /** * Matchers that are useful when writing Dataflow tests. */ 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 d7b3ac54de9d8..66ae79f69a9b0 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 @@ -24,6 +24,7 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.fail; +import com.google.common.collect.ImmutableList; import org.apache.beam.sdk.Pipeline.PipelineExecutionException; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.options.PipelineOptions.CheckEnabled; @@ -47,9 +48,6 @@ import org.apache.beam.sdk.values.PInput; import org.apache.beam.sdk.values.POutput; import org.apache.beam.sdk.values.TupleTag; - -import com.google.common.collect.ImmutableList; - import org.junit.Assert; import org.junit.Rule; import org.junit.Test; diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/TestUtils.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/TestUtils.java index 35709ed42daca..1224f10e0e8b3 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/TestUtils.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/TestUtils.java @@ -19,19 +19,17 @@ import static org.junit.Assert.assertThat; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; import org.apache.beam.sdk.transforms.Combine.CombineFn; import org.apache.beam.sdk.values.KV; - import org.hamcrest.CoreMatchers; import org.hamcrest.Description; import org.hamcrest.Matcher; import org.hamcrest.TypeSafeMatcher; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collections; -import java.util.List; - /** * Utilities for tests. */ diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/WindowMatchers.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/WindowMatchers.java index 48c25899968d3..3531a8631cbe6 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/WindowMatchers.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/WindowMatchers.java @@ -17,22 +17,19 @@ */ package org.apache.beam.sdk; +import com.google.common.collect.Lists; +import java.util.Collection; +import java.util.Objects; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.IntervalWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.util.WindowedValue; - -import com.google.common.collect.Lists; - import org.hamcrest.Description; import org.hamcrest.Matcher; import org.hamcrest.Matchers; import org.hamcrest.TypeSafeMatcher; import org.joda.time.Instant; -import java.util.Collection; -import java.util.Objects; - /** * Matchers that are useful for working with Windowing, Timestamps, etc. */ diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/WindowMatchersTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/WindowMatchersTest.java index 8b108cda2e1a1..89637e2af2702 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/WindowMatchersTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/WindowMatchersTest.java @@ -19,12 +19,10 @@ import static org.junit.Assert.assertThat; +import com.google.common.collect.ImmutableList; import org.apache.beam.sdk.transforms.windowing.IntervalWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.util.WindowedValue; - -import com.google.common.collect.ImmutableList; - import org.joda.time.Instant; import org.junit.Test; import org.junit.runner.RunWith; 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 3b13e351300a8..a97acfb5bcd49 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 @@ -23,20 +23,22 @@ import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; -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; -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.Create; -import org.apache.beam.sdk.transforms.DoFn; -import org.apache.beam.sdk.transforms.ParDo; -import org.apache.beam.sdk.util.CloudObject; -import org.apache.beam.sdk.util.SerializableUtils; -import org.apache.beam.sdk.values.PCollection; - +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.ObjectInputStream; +import java.io.ObjectOutputStream; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.HashSet; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.SortedMap; +import java.util.SortedSet; +import java.util.TreeMap; +import java.util.TreeSet; import org.apache.avro.AvroTypeException; import org.apache.avro.Schema; import org.apache.avro.SchemaBuilder; @@ -50,6 +52,19 @@ 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; +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.Create; +import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.transforms.ParDo; +import org.apache.beam.sdk.util.CloudObject; +import org.apache.beam.sdk.util.SerializableUtils; +import org.apache.beam.sdk.values.PCollection; import org.hamcrest.Description; import org.hamcrest.Matcher; import org.hamcrest.Matchers; @@ -60,23 +75,6 @@ import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.io.ByteArrayInputStream; -import java.io.ByteArrayOutputStream; -import java.io.ObjectInputStream; -import java.io.ObjectOutputStream; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collection; -import java.util.HashSet; -import java.util.LinkedHashMap; -import java.util.List; -import java.util.Map; -import java.util.Objects; -import java.util.SortedMap; -import java.util.SortedSet; -import java.util.TreeMap; -import java.util.TreeSet; - /** Tests for {@link AvroCoder}. */ @RunWith(JUnit4.class) public class AvroCoderTest { diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/BigDecimalCoderTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/BigDecimalCoderTest.java index a5e3bc26b46c6..f5d56cbc11f0f 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/BigDecimalCoderTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/BigDecimalCoderTest.java @@ -20,17 +20,17 @@ import static org.hamcrest.Matchers.equalTo; import static org.junit.Assert.assertThat; +import com.google.common.collect.ImmutableList; +import java.math.BigDecimal; +import java.util.List; import org.apache.beam.sdk.testing.CoderProperties; import org.apache.beam.sdk.testing.CoderProperties.TestElementByteSizeObserver; import org.apache.beam.sdk.util.CoderUtils; -import com.google.common.collect.ImmutableList; import org.junit.Rule; import org.junit.Test; import org.junit.rules.ExpectedException; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.math.BigDecimal; -import java.util.List; /** * Test case for {@link BigDecimalCoder}. diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/BigEndianIntegerCoderTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/BigEndianIntegerCoderTest.java index f4c22bb5a0f22..091fda25c164c 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/BigEndianIntegerCoderTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/BigEndianIntegerCoderTest.java @@ -17,18 +17,16 @@ */ package org.apache.beam.sdk.coders; +import java.util.Arrays; +import java.util.List; import org.apache.beam.sdk.testing.CoderProperties; import org.apache.beam.sdk.util.CoderUtils; - import org.junit.Rule; import org.junit.Test; import org.junit.rules.ExpectedException; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.util.Arrays; -import java.util.List; - /** * Test case for {@link BigEndianIntegerCoder}. */ diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/BigEndianLongCoderTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/BigEndianLongCoderTest.java index cf148b76f92cb..3a74e301695d3 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/BigEndianLongCoderTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/BigEndianLongCoderTest.java @@ -17,18 +17,16 @@ */ package org.apache.beam.sdk.coders; +import java.util.Arrays; +import java.util.List; import org.apache.beam.sdk.testing.CoderProperties; import org.apache.beam.sdk.util.CoderUtils; - import org.junit.Rule; import org.junit.Test; import org.junit.rules.ExpectedException; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.util.Arrays; -import java.util.List; - /** * Test case for {@link BigEndianLongCoder}. */ diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/BigIntegerCoderTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/BigIntegerCoderTest.java index d04237807b9fa..6ead941a3d8d3 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/BigIntegerCoderTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/BigIntegerCoderTest.java @@ -20,17 +20,17 @@ import static org.hamcrest.Matchers.equalTo; import static org.junit.Assert.assertThat; +import com.google.common.collect.ImmutableList; +import java.math.BigInteger; +import java.util.List; import org.apache.beam.sdk.testing.CoderProperties; import org.apache.beam.sdk.testing.CoderProperties.TestElementByteSizeObserver; import org.apache.beam.sdk.util.CoderUtils; -import com.google.common.collect.ImmutableList; import org.junit.Rule; import org.junit.Test; import org.junit.rules.ExpectedException; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.math.BigInteger; -import java.util.List; /** * Test case for {@link BigIntegerCoder}. diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/ByteArrayCoderTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/ByteArrayCoderTest.java index aca0a8304c702..f80a409fced48 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/ByteArrayCoderTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/ByteArrayCoderTest.java @@ -21,20 +21,18 @@ import static org.hamcrest.Matchers.not; import static org.junit.Assert.assertThat; +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.util.Arrays; +import java.util.List; import org.apache.beam.sdk.testing.CoderProperties; import org.apache.beam.sdk.util.CoderUtils; - import org.junit.Rule; import org.junit.Test; import org.junit.rules.ExpectedException; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.io.ByteArrayOutputStream; -import java.io.IOException; -import java.util.Arrays; -import java.util.List; - /** * Unit tests for {@link ByteArrayCoder}. */ diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/ByteCoderTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/ByteCoderTest.java index 0494fe07d6156..b7673e2274225 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/ByteCoderTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/ByteCoderTest.java @@ -17,18 +17,16 @@ */ package org.apache.beam.sdk.coders; +import java.util.Arrays; +import java.util.List; import org.apache.beam.sdk.testing.CoderProperties; import org.apache.beam.sdk.util.CoderUtils; - import org.junit.Rule; import org.junit.Test; import org.junit.rules.ExpectedException; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.util.Arrays; -import java.util.List; - /** * Test case for {@link ByteCoder}. */ diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/ByteStringCoderTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/ByteStringCoderTest.java index d65772cf20eb1..1d4c0627f1519 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/ByteStringCoderTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/ByteStringCoderTest.java @@ -20,22 +20,19 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; +import com.google.common.collect.ImmutableList; +import com.google.protobuf.ByteString; +import java.util.Arrays; +import java.util.List; import org.apache.beam.sdk.coders.Coder.Context; import org.apache.beam.sdk.testing.CoderProperties; import org.apache.beam.sdk.util.CoderUtils; - -import com.google.common.collect.ImmutableList; -import com.google.protobuf.ByteString; - import org.junit.Rule; import org.junit.Test; import org.junit.rules.ExpectedException; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.util.Arrays; -import java.util.List; - /** * Test case for {@link ByteStringCoder}. */ diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/CoderFactoriesTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/CoderFactoriesTest.java index a871404ed87bc..4ffc9c1a19847 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/CoderFactoriesTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/CoderFactoriesTest.java @@ -19,13 +19,12 @@ import static org.junit.Assert.assertEquals; +import java.util.Arrays; +import java.util.Collections; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.util.Arrays; -import java.util.Collections; - /** * Tests for {@link CoderFactories}. */ diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/CoderProvidersTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/CoderProvidersTest.java index 44d17d4f28c3a..44be56daaff0d 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/CoderProvidersTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/CoderProvidersTest.java @@ -20,16 +20,14 @@ import static org.hamcrest.Matchers.instanceOf; import static org.junit.Assert.assertThat; +import java.util.Map; import org.apache.beam.sdk.values.TypeDescriptor; - import org.junit.Rule; import org.junit.Test; import org.junit.rules.ExpectedException; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.util.Map; - /** * Tests for {@link CoderFactories}. */ 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 da1540585a840..d690a474ead42 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 @@ -21,6 +21,20 @@ import static org.hamcrest.Matchers.containsString; import static org.junit.Assert.assertEquals; +import com.google.cloud.dataflow.sdk.coders.Proto2CoderTestMessages.MessageA; +import com.google.common.collect.ImmutableList; +import com.google.protobuf.Duration; +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.io.Serializable; +import java.lang.reflect.Type; +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +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; @@ -35,11 +49,6 @@ import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.TypeDescriptor; - -import com.google.cloud.dataflow.sdk.coders.Proto2CoderTestMessages.MessageA; -import com.google.common.collect.ImmutableList; -import com.google.protobuf.Duration; - import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -47,18 +56,6 @@ import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.io.IOException; -import java.io.InputStream; -import java.io.OutputStream; -import java.io.Serializable; -import java.lang.reflect.Type; -import java.util.Arrays; -import java.util.Collection; -import java.util.Collections; -import java.util.List; -import java.util.Map; -import java.util.Set; - /** * Tests for CoderRegistry. */ diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/CoderTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/CoderTest.java index b6378fc9bd0d3..ccbffdd15db19 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/CoderTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/CoderTest.java @@ -23,17 +23,15 @@ import static org.junit.Assert.assertNotEquals; import static org.junit.Assert.assertThat; +import java.util.Collections; import org.apache.beam.sdk.coders.Coder.Context; import org.apache.beam.sdk.coders.Coder.NonDeterministicException; - import org.junit.Rule; import org.junit.Test; import org.junit.rules.ExpectedException; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.util.Collections; - /** Tests for constructs defined within {@link Coder}. */ @RunWith(JUnit4.class) public class CoderTest { diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/CollectionCoderTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/CollectionCoderTest.java index a2bc20dff46e6..6a5d94b5ae511 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/CollectionCoderTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/CollectionCoderTest.java @@ -17,22 +17,20 @@ */ package org.apache.beam.sdk.coders; +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.LinkedList; +import java.util.List; +import java.util.TreeSet; import org.apache.beam.sdk.testing.CoderProperties; import org.apache.beam.sdk.util.CoderUtils; - import org.junit.Rule; import org.junit.Test; import org.junit.rules.ExpectedException; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.util.Arrays; -import java.util.Collection; -import java.util.Collections; -import java.util.LinkedList; -import java.util.List; -import java.util.TreeSet; - /** * Test case for {@link CollectionCoder}. */ diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/CustomCoderTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/CustomCoderTest.java index 10259d96da139..31bbdb9e60fb6 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/CustomCoderTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/CustomCoderTest.java @@ -17,11 +17,15 @@ */ package org.apache.beam.sdk.coders; +import java.io.DataInputStream; +import java.io.DataOutputStream; +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; import org.apache.beam.sdk.testing.CoderProperties; import org.apache.beam.sdk.util.CoderUtils; import org.apache.beam.sdk.util.SerializableUtils; import org.apache.beam.sdk.values.KV; - import org.junit.Assert; import org.junit.Rule; import org.junit.Test; @@ -29,12 +33,6 @@ import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.io.DataInputStream; -import java.io.DataOutputStream; -import java.io.IOException; -import java.io.InputStream; -import java.io.OutputStream; - /** Unit tests for {@link CustomCoder}. */ @RunWith(JUnit4.class) public class CustomCoderTest { diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/DefaultCoderTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/DefaultCoderTest.java index 1faf58f946390..a8496c946b639 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/DefaultCoderTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/DefaultCoderTest.java @@ -18,13 +18,13 @@ package org.apache.beam.sdk.coders; import static com.google.common.base.Preconditions.checkArgument; - import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.instanceOf; import static org.junit.Assert.assertThat; +import java.io.Serializable; +import java.util.List; import org.apache.beam.sdk.values.TypeDescriptor; - import org.junit.Before; import org.junit.Rule; import org.junit.Test; @@ -32,9 +32,6 @@ import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.io.Serializable; -import java.util.List; - /** * Tests of Coder defaults. */ diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/DelegateCoderTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/DelegateCoderTest.java index cf770aadbd164..9bb9d51ef47f7 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/DelegateCoderTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/DelegateCoderTest.java @@ -20,15 +20,8 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNotEquals; -import org.apache.beam.sdk.testing.CoderProperties; - import com.google.common.collect.Lists; import com.google.common.collect.Sets; - -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.JUnit4; - import java.io.InputStream; import java.io.OutputStream; import java.io.Serializable; @@ -38,6 +31,10 @@ import java.util.HashSet; import java.util.List; import java.util.Set; +import org.apache.beam.sdk.testing.CoderProperties; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; /** Unit tests for {@link DelegateCoder}. */ @RunWith(JUnit4.class) diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/DoubleCoderTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/DoubleCoderTest.java index 73fb464cb69a3..f43af304e3880 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/DoubleCoderTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/DoubleCoderTest.java @@ -17,18 +17,16 @@ */ package org.apache.beam.sdk.coders; +import java.util.Arrays; +import java.util.List; import org.apache.beam.sdk.testing.CoderProperties; import org.apache.beam.sdk.util.CoderUtils; - import org.junit.Rule; import org.junit.Test; import org.junit.rules.ExpectedException; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.util.Arrays; -import java.util.List; - /** * Test case for {@link DoubleCoder}. */ diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/DurationCoderTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/DurationCoderTest.java index e2daefcbf51de..4cb697eff93dd 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/DurationCoderTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/DurationCoderTest.java @@ -17,11 +17,11 @@ */ package org.apache.beam.sdk.coders; +import com.google.common.collect.Lists; +import java.util.Arrays; +import java.util.List; import org.apache.beam.sdk.testing.CoderProperties; import org.apache.beam.sdk.util.CoderUtils; - -import com.google.common.collect.Lists; - import org.joda.time.Duration; import org.joda.time.ReadableDuration; import org.junit.Rule; @@ -30,9 +30,6 @@ import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.util.Arrays; -import java.util.List; - /** Unit tests for {@link DurationCoder}. */ @RunWith(JUnit4.class) public class DurationCoderTest { diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/InstantCoderTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/InstantCoderTest.java index c1ce7e6472827..16cb703ca3ab8 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/InstantCoderTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/InstantCoderTest.java @@ -17,12 +17,14 @@ */ package org.apache.beam.sdk.coders; -import org.apache.beam.sdk.testing.CoderProperties; -import org.apache.beam.sdk.util.CoderUtils; - import com.google.common.collect.Lists; import com.google.common.primitives.UnsignedBytes; - +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import org.apache.beam.sdk.testing.CoderProperties; +import org.apache.beam.sdk.util.CoderUtils; import org.joda.time.Instant; import org.junit.Assert; import org.junit.Rule; @@ -31,11 +33,6 @@ import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collections; -import java.util.List; - /** Unit tests for {@link InstantCoder}. */ @RunWith(JUnit4.class) public class InstantCoderTest { diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/IterableCoderTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/IterableCoderTest.java index 42397b7bf99fd..15ec44bb2d76b 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/IterableCoderTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/IterableCoderTest.java @@ -20,20 +20,18 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNull; +import java.util.Arrays; +import java.util.Collections; +import java.util.LinkedList; +import java.util.List; import org.apache.beam.sdk.testing.CoderProperties; import org.apache.beam.sdk.util.CoderUtils; - import org.junit.Rule; import org.junit.Test; import org.junit.rules.ExpectedException; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.util.Arrays; -import java.util.Collections; -import java.util.LinkedList; -import java.util.List; - /** Unit tests for {@link IterableCoder}. */ @RunWith(JUnit4.class) public class IterableCoderTest { diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/JAXBCoderTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/JAXBCoderTest.java index 6b59e525d965c..129be61ae7c1e 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/JAXBCoderTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/JAXBCoderTest.java @@ -20,16 +20,7 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.fail; -import org.apache.beam.sdk.testing.CoderProperties; -import org.apache.beam.sdk.util.CoderUtils; -import org.apache.beam.sdk.util.SerializableUtils; - import com.google.common.collect.ImmutableList; - -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.JUnit4; - import java.io.IOException; import java.io.InputStream; import java.io.OutputStream; @@ -39,8 +30,13 @@ import java.util.concurrent.Executors; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicReference; - import javax.xml.bind.annotation.XmlRootElement; +import org.apache.beam.sdk.testing.CoderProperties; +import org.apache.beam.sdk.util.CoderUtils; +import org.apache.beam.sdk.util.SerializableUtils; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; /** Unit tests for {@link JAXBCoder}. */ @RunWith(JUnit4.class) diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/KvCoderTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/KvCoderTest.java index 47fcad79a9147..f0f7d22ea95a8 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/KvCoderTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/KvCoderTest.java @@ -17,23 +17,20 @@ */ package org.apache.beam.sdk.coders; +import com.google.common.collect.ImmutableMap; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.Map; import org.apache.beam.sdk.testing.CoderProperties; import org.apache.beam.sdk.util.CoderUtils; import org.apache.beam.sdk.values.KV; - -import com.google.common.collect.ImmutableMap; - import org.junit.Rule; import org.junit.Test; import org.junit.rules.ExpectedException; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.util.Arrays; -import java.util.Collections; -import java.util.List; -import java.util.Map; - /** * Test case for {@link KvCoder}. */ diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/ListCoderTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/ListCoderTest.java index 05b66813c397c..ba9cc9d170ded 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/ListCoderTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/ListCoderTest.java @@ -20,20 +20,18 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNull; +import java.util.Arrays; +import java.util.Collections; +import java.util.LinkedList; +import java.util.List; import org.apache.beam.sdk.testing.CoderProperties; import org.apache.beam.sdk.util.CoderUtils; - import org.junit.Rule; import org.junit.Test; import org.junit.rules.ExpectedException; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.util.Arrays; -import java.util.Collections; -import java.util.LinkedList; -import java.util.List; - /** Unit tests for {@link ListCoder}. */ @RunWith(JUnit4.class) public class ListCoderTest { diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/MapCoderTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/MapCoderTest.java index b92b656e974e9..dc4a8b527f708 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/MapCoderTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/MapCoderTest.java @@ -20,23 +20,20 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNull; -import org.apache.beam.sdk.testing.CoderProperties; -import org.apache.beam.sdk.util.CoderUtils; - import com.google.common.collect.ImmutableMap; - -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.ExpectedException; -import org.junit.runner.RunWith; -import org.junit.runners.JUnit4; - import java.util.Arrays; import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.TreeMap; +import org.apache.beam.sdk.testing.CoderProperties; +import org.apache.beam.sdk.util.CoderUtils; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.ExpectedException; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; /** Unit tests for {@link MapCoder}. */ @RunWith(JUnit4.class) diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/NullableCoderTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/NullableCoderTest.java index 5bfbe05c1a46f..61e7e41afa765 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/NullableCoderTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/NullableCoderTest.java @@ -24,21 +24,18 @@ import static org.junit.Assert.assertThat; import static org.junit.Assert.assertTrue; -import org.apache.beam.sdk.testing.CoderProperties; - import com.google.common.collect.ImmutableList; - +import java.io.ByteArrayInputStream; +import java.io.InputStream; +import java.util.Arrays; +import java.util.List; +import org.apache.beam.sdk.testing.CoderProperties; import org.junit.Rule; import org.junit.Test; import org.junit.rules.ExpectedException; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.io.ByteArrayInputStream; -import java.io.InputStream; -import java.util.Arrays; -import java.util.List; - /** Unit tests for {@link NullableCoder}. */ @RunWith(JUnit4.class) public class NullableCoderTest { diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/PrintBase64Encodings.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/PrintBase64Encodings.java index 9ab66d13b479b..5cdb71974e0c5 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/PrintBase64Encodings.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/PrintBase64Encodings.java @@ -17,14 +17,12 @@ */ package org.apache.beam.sdk.coders; -import org.apache.beam.sdk.util.CoderUtils; - import com.google.common.base.Joiner; import com.google.common.collect.Lists; - import java.lang.reflect.Field; import java.lang.reflect.Modifier; import java.util.List; +import org.apache.beam.sdk.util.CoderUtils; /** * A command-line utility for printing the base-64 encodings of test values, for generating exact diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/SerializableCoderTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/SerializableCoderTest.java index b5465fae0e999..8d344de2831d0 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/SerializableCoderTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/SerializableCoderTest.java @@ -20,6 +20,12 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNull; +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.Serializable; +import java.util.Arrays; +import java.util.LinkedList; +import java.util.List; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.testing.CoderProperties; import org.apache.beam.sdk.testing.NeedsRunner; @@ -32,7 +38,6 @@ import org.apache.beam.sdk.util.CoderUtils; import org.apache.beam.sdk.util.Serializer; import org.apache.beam.sdk.values.PCollection; - import org.hamcrest.Matchers; import org.junit.Assert; import org.junit.Test; @@ -40,13 +45,6 @@ import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.io.ByteArrayInputStream; -import java.io.ByteArrayOutputStream; -import java.io.Serializable; -import java.util.Arrays; -import java.util.LinkedList; -import java.util.List; - /** * Tests SerializableCoder. */ diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/SetCoderTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/SetCoderTest.java index e353504d30e1c..58b0b8ef580f4 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/SetCoderTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/SetCoderTest.java @@ -17,21 +17,19 @@ */ package org.apache.beam.sdk.coders; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.Set; +import java.util.TreeSet; import org.apache.beam.sdk.testing.CoderProperties; import org.apache.beam.sdk.util.CoderUtils; - import org.junit.Rule; import org.junit.Test; import org.junit.rules.ExpectedException; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.util.Arrays; -import java.util.Collections; -import java.util.List; -import java.util.Set; -import java.util.TreeSet; - /** * Test case for {@link SetCoder}. */ diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/StandardCoderTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/StandardCoderTest.java index 1dc221017e7ed..e3d0537c87658 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/StandardCoderTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/StandardCoderTest.java @@ -17,24 +17,20 @@ */ package org.apache.beam.sdk.coders; -import org.apache.beam.sdk.testing.CoderProperties; - import com.google.common.collect.ImmutableList; - -import org.hamcrest.CoreMatchers; -import org.junit.Assert; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.JUnit4; - import java.io.IOException; import java.io.InputStream; import java.io.OutputStream; import java.util.Arrays; import java.util.Collections; import java.util.List; - import javax.annotation.Nullable; +import org.apache.beam.sdk.testing.CoderProperties; +import org.hamcrest.CoreMatchers; +import org.junit.Assert; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; /** * Test case for {@link StandardCoder}. diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/StringDelegateCoderTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/StringDelegateCoderTest.java index 2a4f2e6a3143e..b8a45fd80d739 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/StringDelegateCoderTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/StringDelegateCoderTest.java @@ -17,17 +17,15 @@ */ package org.apache.beam.sdk.coders; +import java.net.URI; +import java.util.Arrays; +import java.util.List; import org.apache.beam.sdk.coders.Coder.NonDeterministicException; import org.apache.beam.sdk.testing.CoderProperties; - import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.net.URI; -import java.util.Arrays; -import java.util.List; - /** Unit tests for {@link StringDelegateCoder}. */ @RunWith(JUnit4.class) public class StringDelegateCoderTest { diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/StringUtf8CoderTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/StringUtf8CoderTest.java index 5c810f72c4382..cce04ec425678 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/StringUtf8CoderTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/StringUtf8CoderTest.java @@ -17,18 +17,16 @@ */ package org.apache.beam.sdk.coders; +import java.util.Arrays; +import java.util.List; import org.apache.beam.sdk.testing.CoderProperties; import org.apache.beam.sdk.util.CoderUtils; - import org.junit.Rule; import org.junit.Test; import org.junit.rules.ExpectedException; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.util.Arrays; -import java.util.List; - /** * Test case for {@link StringUtf8Coder}. */ diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/TableRowJsonCoderTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/TableRowJsonCoderTest.java index 03f2339741ffc..3da65221a036e 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/TableRowJsonCoderTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/TableRowJsonCoderTest.java @@ -17,17 +17,14 @@ */ package org.apache.beam.sdk.coders; -import org.apache.beam.sdk.testing.CoderProperties; - import com.google.api.services.bigquery.model.TableRow; - +import java.util.Arrays; +import java.util.List; +import org.apache.beam.sdk.testing.CoderProperties; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.util.Arrays; -import java.util.List; - /** * Test case for {@link TableRowJsonCoder}. */ diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/TextualIntegerCoderTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/TextualIntegerCoderTest.java index 252f8a4863ab9..5012d1371d891 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/TextualIntegerCoderTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/TextualIntegerCoderTest.java @@ -17,18 +17,16 @@ */ package org.apache.beam.sdk.coders; +import java.util.Arrays; +import java.util.List; import org.apache.beam.sdk.testing.CoderProperties; import org.apache.beam.sdk.util.CoderUtils; - import org.junit.Rule; import org.junit.Test; import org.junit.rules.ExpectedException; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.util.Arrays; -import java.util.List; - /** * Test case for {@link TextualIntegerCoder}. */ diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/VarIntCoderTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/VarIntCoderTest.java index 3da242d39acec..72200917f78ed 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/VarIntCoderTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/VarIntCoderTest.java @@ -17,18 +17,16 @@ */ package org.apache.beam.sdk.coders; +import java.util.Arrays; +import java.util.List; import org.apache.beam.sdk.testing.CoderProperties; import org.apache.beam.sdk.util.CoderUtils; - import org.junit.Rule; import org.junit.Test; import org.junit.rules.ExpectedException; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.util.Arrays; -import java.util.List; - /** * Test case for {@link VarIntCoder}. */ diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/VarLongCoderTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/VarLongCoderTest.java index 4cd221628072b..615d4fcb05656 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/VarLongCoderTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/VarLongCoderTest.java @@ -17,18 +17,16 @@ */ package org.apache.beam.sdk.coders; +import java.util.Arrays; +import java.util.List; import org.apache.beam.sdk.testing.CoderProperties; import org.apache.beam.sdk.util.CoderUtils; - import org.junit.Rule; import org.junit.Test; import org.junit.rules.ExpectedException; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.util.Arrays; -import java.util.List; - /** * Test case for {@link VarLongCoder}. */ diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/protobuf/ProtoCoderTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/protobuf/ProtoCoderTest.java index b35adf6308e28..8b889dab26290 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/protobuf/ProtoCoderTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/protobuf/ProtoCoderTest.java @@ -20,6 +20,12 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNotEquals; +import com.google.cloud.dataflow.sdk.coders.Proto2CoderTestMessages; +import com.google.cloud.dataflow.sdk.coders.Proto2CoderTestMessages.MessageA; +import com.google.cloud.dataflow.sdk.coders.Proto2CoderTestMessages.MessageB; +import com.google.cloud.dataflow.sdk.coders.Proto2CoderTestMessages.MessageC; +import com.google.cloud.dataflow.sdk.coders.Proto2CoderTestMessages.MessageWithMap; +import com.google.common.collect.ImmutableList; import org.apache.beam.sdk.coders.CannotProvideCoderException; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.Coder.NonDeterministicException; @@ -28,14 +34,6 @@ import org.apache.beam.sdk.testing.CoderProperties; import org.apache.beam.sdk.util.CoderUtils; import org.apache.beam.sdk.values.TypeDescriptor; - -import com.google.cloud.dataflow.sdk.coders.Proto2CoderTestMessages; -import com.google.cloud.dataflow.sdk.coders.Proto2CoderTestMessages.MessageA; -import com.google.cloud.dataflow.sdk.coders.Proto2CoderTestMessages.MessageB; -import com.google.cloud.dataflow.sdk.coders.Proto2CoderTestMessages.MessageC; -import com.google.cloud.dataflow.sdk.coders.Proto2CoderTestMessages.MessageWithMap; -import com.google.common.collect.ImmutableList; - import org.junit.Rule; import org.junit.Test; import org.junit.rules.ExpectedException; diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/protobuf/ProtobufUtilTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/protobuf/ProtobufUtilTest.java index 14fe4d8db7feb..97368248998b9 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/protobuf/ProtobufUtilTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/protobuf/ProtobufUtilTest.java @@ -20,12 +20,9 @@ import static org.apache.beam.sdk.coders.protobuf.ProtobufUtil.checkProto2Syntax; import static org.apache.beam.sdk.coders.protobuf.ProtobufUtil.getRecursiveDescriptorsForClass; import static org.apache.beam.sdk.coders.protobuf.ProtobufUtil.verifyDeterministic; - import static org.hamcrest.Matchers.equalTo; import static org.junit.Assert.assertThat; -import org.apache.beam.sdk.coders.Coder.NonDeterministicException; - import com.google.cloud.dataflow.sdk.coders.Proto2CoderTestMessages; import com.google.cloud.dataflow.sdk.coders.Proto2CoderTestMessages.MessageA; import com.google.cloud.dataflow.sdk.coders.Proto2CoderTestMessages.MessageB; @@ -39,16 +36,15 @@ import com.google.protobuf.Duration; import com.google.protobuf.ExtensionRegistry; import com.google.protobuf.Message; - +import java.util.HashSet; +import java.util.Set; +import org.apache.beam.sdk.coders.Coder.NonDeterministicException; import org.junit.Rule; import org.junit.Test; import org.junit.rules.ExpectedException; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.util.HashSet; -import java.util.Set; - /** * Tests for {@link ProtobufUtil}. */ diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/AvroIOGeneratedClassTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/AvroIOGeneratedClassTest.java index 6e26d33d92e4b..ede135f19ce81 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/AvroIOGeneratedClassTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/AvroIOGeneratedClassTest.java @@ -21,14 +21,11 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertThat; -import org.apache.beam.sdk.coders.AvroCoder; -import org.apache.beam.sdk.coders.Coder; -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.Create; -import org.apache.beam.sdk.values.PCollection; - +import java.io.File; +import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; import org.apache.avro.Schema; import org.apache.avro.file.DataFileReader; import org.apache.avro.file.DataFileWriter; @@ -38,6 +35,13 @@ import org.apache.avro.io.DatumWriter; import org.apache.avro.specific.SpecificDatumReader; import org.apache.avro.specific.SpecificDatumWriter; +import org.apache.beam.sdk.coders.AvroCoder; +import org.apache.beam.sdk.coders.Coder; +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.Create; +import org.apache.beam.sdk.values.PCollection; import org.junit.Before; import org.junit.Rule; import org.junit.Test; @@ -46,12 +50,6 @@ import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.io.File; -import java.io.IOException; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.List; - /** * Tests for AvroIO Read and Write transforms, using classes generated from {@code user.avsc}. */ 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 026724add21e9..a8a7746f214bb 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 @@ -18,7 +18,6 @@ package org.apache.beam.sdk.io; 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; @@ -26,6 +25,19 @@ import static org.junit.Assert.assertThat; import static org.junit.Assert.assertTrue; +import com.google.common.base.MoreObjects; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.Iterators; +import java.io.File; +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.Objects; +import java.util.Set; +import org.apache.avro.Schema; +import org.apache.avro.file.DataFileReader; +import org.apache.avro.generic.GenericRecord; +import org.apache.avro.reflect.Nullable; import org.apache.beam.sdk.coders.AvroCoder; import org.apache.beam.sdk.coders.DefaultCoder; import org.apache.beam.sdk.io.AvroIO.Write.Bound; @@ -40,15 +52,6 @@ import org.apache.beam.sdk.transforms.display.DisplayDataEvaluator; import org.apache.beam.sdk.util.IOChannelUtils; import org.apache.beam.sdk.values.PCollection; - -import com.google.common.base.MoreObjects; -import com.google.common.collect.ImmutableList; -import com.google.common.collect.Iterators; - -import org.apache.avro.Schema; -import org.apache.avro.file.DataFileReader; -import org.apache.avro.generic.GenericRecord; -import org.apache.avro.reflect.Nullable; import org.junit.BeforeClass; import org.junit.Ignore; import org.junit.Rule; @@ -58,13 +61,6 @@ import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.io.File; -import java.io.IOException; -import java.util.ArrayList; -import java.util.List; -import java.util.Objects; -import java.util.Set; - /** * Tests for AvroIO Read and Write transforms. */ diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/AvroSourceTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/AvroSourceTest.java index 09405abeece6e..c1b532f8b4d9e 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/AvroSourceTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/AvroSourceTest.java @@ -18,26 +18,24 @@ package org.apache.beam.sdk.io; import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasDisplayItem; - import static org.hamcrest.Matchers.containsInAnyOrder; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertThat; import static org.junit.Assert.assertTrue; -import org.apache.beam.sdk.coders.AvroCoder; -import org.apache.beam.sdk.coders.DefaultCoder; -import org.apache.beam.sdk.io.AvroSource.AvroReader; -import org.apache.beam.sdk.io.AvroSource.AvroReader.Seeker; -import org.apache.beam.sdk.io.BlockBasedSource.BlockBasedReader; -import org.apache.beam.sdk.io.BoundedSource.BoundedReader; -import org.apache.beam.sdk.options.PipelineOptions; -import org.apache.beam.sdk.options.PipelineOptionsFactory; -import org.apache.beam.sdk.testing.SourceTestUtils; -import org.apache.beam.sdk.transforms.display.DisplayData; - import com.google.common.base.MoreObjects; - +import java.io.ByteArrayInputStream; +import java.io.File; +import java.io.FileOutputStream; +import java.io.IOException; +import java.io.PushbackInputStream; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.NoSuchElementException; +import java.util.Objects; +import java.util.Random; import org.apache.avro.Schema; import org.apache.avro.file.CodecFactory; import org.apache.avro.file.DataFileConstants; @@ -47,6 +45,16 @@ import org.apache.avro.reflect.AvroDefault; import org.apache.avro.reflect.Nullable; import org.apache.avro.reflect.ReflectData; +import org.apache.beam.sdk.coders.AvroCoder; +import org.apache.beam.sdk.coders.DefaultCoder; +import org.apache.beam.sdk.io.AvroSource.AvroReader; +import org.apache.beam.sdk.io.AvroSource.AvroReader.Seeker; +import org.apache.beam.sdk.io.BlockBasedSource.BlockBasedReader; +import org.apache.beam.sdk.io.BoundedSource.BoundedReader; +import org.apache.beam.sdk.options.PipelineOptions; +import org.apache.beam.sdk.options.PipelineOptionsFactory; +import org.apache.beam.sdk.testing.SourceTestUtils; +import org.apache.beam.sdk.transforms.display.DisplayData; import org.hamcrest.Matchers; import org.junit.Rule; import org.junit.Test; @@ -55,18 +63,6 @@ import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.io.ByteArrayInputStream; -import java.io.File; -import java.io.FileOutputStream; -import java.io.IOException; -import java.io.PushbackInputStream; -import java.util.ArrayList; -import java.util.Collections; -import java.util.List; -import java.util.NoSuchElementException; -import java.util.Objects; -import java.util.Random; - /** * Tests for AvroSource. */ 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 fe9415bbb3a10..7bf0f35576bdd 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 @@ -18,12 +18,15 @@ package org.apache.beam.sdk.io; import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.includesDisplayDataFrom; - import static org.hamcrest.Matchers.containsInAnyOrder; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertThat; import static org.junit.Assert.assertTrue; +import java.io.Serializable; +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; @@ -33,18 +36,12 @@ import org.apache.beam.sdk.transforms.display.DisplayData; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; - import org.joda.time.Duration; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.io.Serializable; -import java.util.ArrayList; -import java.util.Collections; -import java.util.List; - /** Unit tests for {@link BoundedReadFromUnboundedSource}. */ @RunWith(JUnit4.class) public class BoundedReadFromUnboundedSourceTest implements Serializable{ 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 4a9f95088804e..d7c451d00cef4 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 @@ -19,7 +19,6 @@ import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasDisplayItem; import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.includesDisplayDataFrom; - import static org.hamcrest.Matchers.containsString; import static org.hamcrest.Matchers.instanceOf; import static org.hamcrest.Matchers.not; @@ -30,6 +29,24 @@ import static org.junit.Assert.assertThat; import static org.junit.Assert.assertTrue; +import com.google.common.collect.Lists; +import com.google.common.collect.Sets; +import com.google.common.io.Files; +import com.google.common.primitives.Bytes; +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.nio.charset.StandardCharsets; +import java.util.ArrayList; +import java.util.List; +import java.util.NoSuchElementException; +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; @@ -46,12 +63,6 @@ import org.apache.beam.sdk.testing.TestPipeline; import org.apache.beam.sdk.transforms.display.DisplayData; import org.apache.beam.sdk.values.PCollection; - -import com.google.common.collect.Lists; -import com.google.common.collect.Sets; -import com.google.common.io.Files; -import com.google.common.primitives.Bytes; - import org.apache.commons.compress.compressors.bzip2.BZip2CompressorOutputStream; import org.apache.commons.compress.compressors.gzip.GzipCompressorOutputStream; import org.hamcrest.Matchers; @@ -64,22 +75,6 @@ import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -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.nio.charset.StandardCharsets; -import java.util.ArrayList; -import java.util.List; -import java.util.NoSuchElementException; -import java.util.Random; -import java.util.zip.GZIPOutputStream; - -import javax.annotation.Nullable; - /** * Tests for CompressedSource. */ 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 4ec2c9ae02da7..0b92b2698ea38 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/CountingInputTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/CountingInputTest.java @@ -18,7 +18,6 @@ package org.apache.beam.sdk.io; import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasDisplayItem; - import static org.hamcrest.Matchers.is; import static org.junit.Assert.assertThat; @@ -38,7 +37,6 @@ import org.apache.beam.sdk.transforms.SerializableFunction; import org.apache.beam.sdk.transforms.display.DisplayData; import org.apache.beam.sdk.values.PCollection; - import org.joda.time.Duration; import org.joda.time.Instant; import org.junit.Test; 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 0bd91c14846f5..7c5fa1377713f 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 @@ -23,6 +23,8 @@ import static org.junit.Assert.assertThat; import static org.junit.Assert.assertTrue; +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; @@ -44,7 +46,6 @@ import org.apache.beam.sdk.util.CoderUtils; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionList; - import org.joda.time.Duration; import org.joda.time.Instant; import org.junit.Test; @@ -52,9 +53,6 @@ import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.io.IOException; -import java.util.List; - /** * Tests of {@link CountingSource}. */ 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 d3454da19f138..0fdb11f1e3f04 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 @@ -24,18 +24,6 @@ import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; -import org.apache.beam.sdk.io.FileBasedSink.FileBasedWriteOperation; -import org.apache.beam.sdk.io.FileBasedSink.FileBasedWriteOperation.TemporaryFileRetention; -import org.apache.beam.sdk.io.FileBasedSink.FileResult; -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.TemporaryFolder; -import org.junit.runner.RunWith; -import org.junit.runners.JUnit4; - import java.io.BufferedReader; import java.io.File; import java.io.FileOutputStream; @@ -47,6 +35,16 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.List; +import org.apache.beam.sdk.io.FileBasedSink.FileBasedWriteOperation; +import org.apache.beam.sdk.io.FileBasedSink.FileBasedWriteOperation.TemporaryFileRetention; +import org.apache.beam.sdk.io.FileBasedSink.FileResult; +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.TemporaryFolder; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; /** * Tests for FileBasedSink. 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 c9f4079e5c1aa..5208910590808 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 @@ -21,7 +21,6 @@ import static org.apache.beam.sdk.testing.SourceTestUtils.assertSplitAtFractionFails; import static org.apache.beam.sdk.testing.SourceTestUtils.assertSplitAtFractionSucceedsAndConsistent; import static org.apache.beam.sdk.testing.SourceTestUtils.readFromSource; - import static org.hamcrest.Matchers.containsInAnyOrder; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertThat; @@ -29,6 +28,20 @@ import static org.junit.Assert.fail; import static org.mockito.Mockito.when; +import com.google.common.collect.ImmutableList; +import java.io.ByteArrayOutputStream; +import java.io.File; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.channels.ReadableByteChannel; +import java.nio.channels.SeekableByteChannel; +import java.nio.charset.StandardCharsets; +import java.nio.file.Files; +import java.util.ArrayList; +import java.util.Arrays; +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; @@ -43,9 +56,6 @@ import org.apache.beam.sdk.util.IOChannelFactory; import org.apache.beam.sdk.util.IOChannelUtils; import org.apache.beam.sdk.values.PCollection; - -import com.google.common.collect.ImmutableList; - import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -54,20 +64,6 @@ import org.junit.runners.JUnit4; import org.mockito.Mockito; -import java.io.ByteArrayOutputStream; -import java.io.File; -import java.io.IOException; -import java.nio.ByteBuffer; -import java.nio.channels.ReadableByteChannel; -import java.nio.channels.SeekableByteChannel; -import java.nio.charset.StandardCharsets; -import java.nio.file.Files; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.List; -import java.util.NoSuchElementException; -import java.util.Random; - /** * Tests code common to all file-based sources. */ diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/OffsetBasedSourceTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/OffsetBasedSourceTest.java index f689f51e97c02..923b4b4ba3c56 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/OffsetBasedSourceTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/OffsetBasedSourceTest.java @@ -19,29 +19,26 @@ import static org.apache.beam.sdk.testing.SourceTestUtils.assertSplitAtFractionExhaustive; import static org.apache.beam.sdk.testing.SourceTestUtils.readFromSource; - import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertNull; import static org.junit.Assert.assertTrue; +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.NoSuchElementException; import org.apache.beam.sdk.coders.BigEndianIntegerCoder; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.io.BoundedSource.BoundedReader; import org.apache.beam.sdk.io.OffsetBasedSource.OffsetBasedReader; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.options.PipelineOptionsFactory; - import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.io.IOException; -import java.util.ArrayList; -import java.util.List; -import java.util.NoSuchElementException; - /** * Tests code common to all offset-based sources. */ 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 1e9ebf2d1529e..4067055b25c75 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 @@ -18,15 +18,14 @@ package org.apache.beam.sdk.io; 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.assertThat; +import java.util.Set; import org.apache.beam.sdk.testing.RunnableOnService; import org.apache.beam.sdk.transforms.display.DisplayData; import org.apache.beam.sdk.transforms.display.DisplayDataEvaluator; - import org.joda.time.Duration; import org.junit.Rule; import org.junit.Test; @@ -35,8 +34,6 @@ import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.util.Set; - /** * Tests for PubsubIO Read and Write transforms. */ 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 db03a5cce7a0b..4edd9c106ea30 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 @@ -18,6 +18,11 @@ package org.apache.beam.sdk.io; +import com.google.common.collect.ImmutableList; +import com.google.common.hash.Hashing; +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; import org.apache.beam.sdk.coders.StringUtf8Coder; import org.apache.beam.sdk.io.PubsubUnboundedSink.RecordIdMethod; import org.apache.beam.sdk.testing.CoderProperties; @@ -31,10 +36,6 @@ import org.apache.beam.sdk.util.PubsubClient.TopicPath; import org.apache.beam.sdk.util.PubsubTestClient; import org.apache.beam.sdk.util.PubsubTestClient.PubsubTestClientFactory; - -import com.google.common.collect.ImmutableList; -import com.google.common.hash.Hashing; - import org.joda.time.Duration; import org.joda.time.Instant; import org.junit.Test; @@ -42,10 +43,6 @@ import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.io.IOException; -import java.util.ArrayList; -import java.util.List; - /** * Test PubsubUnboundedSink. */ 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 a19ccc5197e34..c46eca527dcd8 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 @@ -26,6 +26,14 @@ import static org.junit.Assert.assertThat; import static org.junit.Assert.assertTrue; +import com.google.api.client.util.Clock; +import com.google.common.collect.ImmutableList; +import java.io.IOException; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.concurrent.atomic.AtomicLong; import org.apache.beam.sdk.coders.StringUtf8Coder; import org.apache.beam.sdk.io.PubsubUnboundedSource.PubsubCheckpoint; import org.apache.beam.sdk.io.PubsubUnboundedSource.PubsubReader; @@ -38,23 +46,12 @@ import org.apache.beam.sdk.util.PubsubClient.SubscriptionPath; import org.apache.beam.sdk.util.PubsubTestClient; import org.apache.beam.sdk.util.PubsubTestClient.PubsubTestClientFactory; - -import com.google.api.client.util.Clock; -import com.google.common.collect.ImmutableList; - import org.joda.time.Instant; import org.junit.After; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.io.IOException; -import java.util.ArrayList; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.concurrent.atomic.AtomicLong; - /** * Test PubsubUnboundedSource. */ diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/ReadTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/ReadTest.java index a47ddf2a34cfd..30a8a43e90a56 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/ReadTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/ReadTest.java @@ -19,14 +19,16 @@ import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasDisplayItem; import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.includesDisplayDataFrom; - import static org.hamcrest.MatcherAssert.assertThat; +import java.io.IOException; +import java.io.Serializable; +import java.util.List; +import javax.annotation.Nullable; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.io.UnboundedSource.CheckpointMark; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.display.DisplayData; - import org.joda.time.Duration; import org.junit.Rule; import org.junit.Test; @@ -34,12 +36,6 @@ import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.io.IOException; -import java.io.Serializable; -import java.util.List; - -import javax.annotation.Nullable; - /** * Tests for {@link Read}. */ 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 6fd3093f2155b..358a30f3d2b81 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 @@ -23,7 +23,6 @@ import static org.apache.beam.sdk.TestUtils.NO_LINES_ARRAY; import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasDisplayItem; import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasValue; - import static org.hamcrest.Matchers.containsInAnyOrder; import static org.hamcrest.Matchers.greaterThan; import static org.hamcrest.Matchers.hasItem; @@ -34,6 +33,27 @@ import static org.junit.Assert.assertThat; import static org.junit.Assert.assertTrue; +import com.google.common.collect.ImmutableList; +import java.io.BufferedReader; +import java.io.File; +import java.io.FileOutputStream; +import java.io.FileReader; +import java.io.IOException; +import java.io.OutputStream; +import java.io.PrintStream; +import java.nio.channels.FileChannel; +import java.nio.channels.SeekableByteChannel; +import java.nio.charset.StandardCharsets; +import java.nio.file.Files; +import java.nio.file.StandardOpenOption; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.Set; +import java.util.zip.GZIPOutputStream; +import java.util.zip.ZipEntry; +import java.util.zip.ZipOutputStream; +import javax.annotation.Nullable; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.StringUtf8Coder; @@ -59,9 +79,6 @@ import org.apache.beam.sdk.util.IOChannelUtils; import org.apache.beam.sdk.util.gcsfs.GcsPath; import org.apache.beam.sdk.values.PCollection; - -import com.google.common.collect.ImmutableList; - import org.apache.commons.compress.compressors.bzip2.BZip2CompressorOutputStream; import org.junit.BeforeClass; import org.junit.Ignore; @@ -76,28 +93,6 @@ import org.mockito.invocation.InvocationOnMock; import org.mockito.stubbing.Answer; -import java.io.BufferedReader; -import java.io.File; -import java.io.FileOutputStream; -import java.io.FileReader; -import java.io.IOException; -import java.io.OutputStream; -import java.io.PrintStream; -import java.nio.channels.FileChannel; -import java.nio.channels.SeekableByteChannel; -import java.nio.charset.StandardCharsets; -import java.nio.file.Files; -import java.nio.file.StandardOpenOption; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.List; -import java.util.Set; -import java.util.zip.GZIPOutputStream; -import java.util.zip.ZipEntry; -import java.util.zip.ZipOutputStream; - -import javax.annotation.Nullable; - /** * Tests for TextIO Read and Write transforms. */ 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 b9ba53ba6e130..28651884d2982 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 @@ -28,6 +28,19 @@ import static org.junit.Assert.assertThat; import static org.junit.Assert.assertTrue; +import com.google.common.base.MoreObjects; +import com.google.common.base.Optional; +import java.io.Serializable; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashSet; +import java.util.List; +import java.util.Objects; +import java.util.Set; +import java.util.UUID; +import java.util.concurrent.ThreadLocalRandom; +import java.util.concurrent.atomic.AtomicInteger; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.SerializableCoder; @@ -52,10 +65,6 @@ import org.apache.beam.sdk.transforms.windowing.Window; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; - -import com.google.common.base.MoreObjects; -import com.google.common.base.Optional; - import org.hamcrest.Matchers; import org.joda.time.Duration; import org.junit.Rule; @@ -65,18 +74,6 @@ import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.io.Serializable; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collections; -import java.util.HashSet; -import java.util.List; -import java.util.Objects; -import java.util.Set; -import java.util.UUID; -import java.util.concurrent.ThreadLocalRandom; -import java.util.concurrent.atomic.AtomicInteger; - /** * Tests for the Write PTransform. */ 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 ea0db73865d91..2788ea664621e 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 @@ -18,26 +18,11 @@ package org.apache.beam.sdk.io; import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasDisplayItem; - import static org.hamcrest.MatcherAssert.assertThat; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNotNull; -import org.apache.beam.sdk.io.XmlSink.XmlWriteOperation; -import org.apache.beam.sdk.io.XmlSink.XmlWriter; -import org.apache.beam.sdk.options.PipelineOptions; -import org.apache.beam.sdk.options.PipelineOptionsFactory; -import org.apache.beam.sdk.transforms.display.DisplayData; - import com.google.common.collect.Lists; - -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 java.io.BufferedReader; import java.io.File; import java.io.FileOutputStream; @@ -46,10 +31,20 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.List; - import javax.xml.bind.annotation.XmlElement; import javax.xml.bind.annotation.XmlRootElement; import javax.xml.bind.annotation.XmlType; +import org.apache.beam.sdk.io.XmlSink.XmlWriteOperation; +import org.apache.beam.sdk.io.XmlSink.XmlWriter; +import org.apache.beam.sdk.options.PipelineOptions; +import org.apache.beam.sdk.options.PipelineOptionsFactory; +import org.apache.beam.sdk.transforms.display.DisplayData; +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; /** * Tests for XmlSink. 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 37e3881888136..1f154d577ccef 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 @@ -21,7 +21,6 @@ import static org.apache.beam.sdk.testing.SourceTestUtils.assertSplitAtFractionFails; import static org.apache.beam.sdk.testing.SourceTestUtils.assertSplitAtFractionSucceedsAndConsistent; import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasDisplayItem; - import static org.hamcrest.Matchers.both; import static org.hamcrest.Matchers.containsInAnyOrder; import static org.hamcrest.Matchers.containsString; @@ -29,6 +28,18 @@ import static org.junit.Assert.assertThat; import static org.junit.Assert.assertTrue; +import com.google.common.collect.ImmutableList; +import java.io.BufferedWriter; +import java.io.File; +import java.io.FileWriter; +import java.io.IOException; +import java.nio.charset.StandardCharsets; +import java.nio.file.Files; +import java.util.ArrayList; +import java.util.List; +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; @@ -38,9 +49,6 @@ import org.apache.beam.sdk.testing.TestPipeline; import org.apache.beam.sdk.transforms.display.DisplayData; import org.apache.beam.sdk.values.PCollection; - -import com.google.common.collect.ImmutableList; - import org.hamcrest.Matchers; import org.junit.Ignore; import org.junit.Rule; @@ -51,19 +59,6 @@ import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.io.BufferedWriter; -import java.io.File; -import java.io.FileWriter; -import java.io.IOException; -import java.nio.charset.StandardCharsets; -import java.nio.file.Files; -import java.util.ArrayList; -import java.util.List; -import java.util.Random; - -import javax.xml.bind.annotation.XmlAttribute; -import javax.xml.bind.annotation.XmlRootElement; - /** * Tests XmlSource. */ diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/range/ByteKeyRangeEstimateFractionTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/range/ByteKeyRangeEstimateFractionTest.java index 50ec2721d3de3..92daf89cad2fc 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/range/ByteKeyRangeEstimateFractionTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/range/ByteKeyRangeEstimateFractionTest.java @@ -21,7 +21,6 @@ import static org.junit.Assert.assertThat; import com.google.common.collect.ImmutableList; - import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.Parameterized; diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/range/ByteKeyRangeInterpolateKeyTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/range/ByteKeyRangeInterpolateKeyTest.java index e8010673e13c7..a6445eb537f32 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/range/ByteKeyRangeInterpolateKeyTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/range/ByteKeyRangeInterpolateKeyTest.java @@ -23,7 +23,6 @@ import static org.junit.Assert.assertThat; import com.google.common.collect.ImmutableList; - import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.Parameterized; diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/range/ByteKeyRangeTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/range/ByteKeyRangeTest.java index f068ec9be73af..40f6d8fe9a791 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/range/ByteKeyRangeTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/range/ByteKeyRangeTest.java @@ -27,14 +27,12 @@ import com.google.common.collect.ImmutableList; import com.google.protobuf.ByteString; - +import java.util.Arrays; +import java.util.List; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.util.Arrays; -import java.util.List; - /** * Tests for {@link ByteKeyRange}. */ diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/range/ByteKeyTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/range/ByteKeyTest.java index b8c85ee1292e7..1117ac7ea10fe 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/range/ByteKeyTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/range/ByteKeyTest.java @@ -24,12 +24,11 @@ import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; +import java.util.Arrays; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.util.Arrays; - /** * Tests of {@link ByteKey}. */ 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 ec2902e2f538e..34077a2bdfcdf 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 @@ -18,20 +18,21 @@ package org.apache.beam.sdk.options; import static com.google.common.base.Strings.isNullOrEmpty; - import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNull; import static org.junit.Assert.assertTrue; import static org.mockito.Mockito.spy; import static org.mockito.Mockito.when; +import com.google.common.collect.ImmutableMap; +import com.google.common.io.Files; +import java.io.File; +import java.io.IOException; +import java.nio.charset.StandardCharsets; +import java.util.Map; import org.apache.beam.sdk.options.GcpOptions.DefaultProjectFactory; import org.apache.beam.sdk.testing.RestoreSystemProperties; import org.apache.beam.sdk.util.NoopPathValidator; - -import com.google.common.collect.ImmutableMap; -import com.google.common.io.Files; - import org.junit.Rule; import org.junit.Test; import org.junit.rules.TemporaryFolder; @@ -39,11 +40,6 @@ import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.io.File; -import java.io.IOException; -import java.nio.charset.StandardCharsets; -import java.util.Map; - /** Tests for {@link GcpOptions}. */ @RunWith(JUnit4.class) public class GcpOptionsTest { diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/options/GoogleApiDebugOptionsTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/options/GoogleApiDebugOptionsTest.java index 8e1439bd7de1c..dae7208487e1d 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/options/GoogleApiDebugOptionsTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/options/GoogleApiDebugOptionsTest.java @@ -21,14 +21,12 @@ import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertNull; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.api.services.bigquery.Bigquery.Datasets.Delete; +import com.google.api.services.storage.Storage; import org.apache.beam.sdk.options.GoogleApiDebugOptions.GoogleApiTracer; import org.apache.beam.sdk.util.TestCredential; import org.apache.beam.sdk.util.Transport; - -import com.google.api.services.bigquery.Bigquery.Datasets.Delete; -import com.google.api.services.storage.Storage; - -import com.fasterxml.jackson.databind.ObjectMapper; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/options/PipelineOptionsFactoryTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/options/PipelineOptionsFactoryTest.java index 0c1b5963c849a..70c89837ca5b2 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/options/PipelineOptionsFactoryTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/options/PipelineOptionsFactoryTest.java @@ -27,6 +27,18 @@ import static org.junit.Assert.assertThat; import static org.junit.Assert.assertTrue; +import com.fasterxml.jackson.annotation.JsonIgnore; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.auto.service.AutoService; +import com.google.common.collect.ArrayListMultimap; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.ListMultimap; +import java.io.ByteArrayOutputStream; +import java.io.PrintStream; +import java.util.List; +import java.util.Map; +import java.util.Set; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.PipelineResult; import org.apache.beam.sdk.runners.PipelineRunner; @@ -34,15 +46,6 @@ import org.apache.beam.sdk.testing.CrashingRunner; import org.apache.beam.sdk.testing.ExpectedLogs; import org.apache.beam.sdk.testing.RestoreSystemProperties; - -import com.google.auto.service.AutoService; -import com.google.common.collect.ArrayListMultimap; -import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableMap; -import com.google.common.collect.ListMultimap; - -import com.fasterxml.jackson.annotation.JsonIgnore; -import com.fasterxml.jackson.annotation.JsonProperty; import org.hamcrest.Matchers; import org.junit.Rule; import org.junit.Test; @@ -51,12 +54,6 @@ import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.io.ByteArrayOutputStream; -import java.io.PrintStream; -import java.util.List; -import java.util.Map; -import java.util.Set; - /** Tests for {@link PipelineOptionsFactory}. */ @RunWith(JUnit4.class) public class PipelineOptionsFactoryTest { diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/options/PipelineOptionsReflectorTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/options/PipelineOptionsReflectorTest.java index 8f801c79688a5..7632b50ab279a 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/options/PipelineOptionsReflectorTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/options/PipelineOptionsReflectorTest.java @@ -26,9 +26,9 @@ import static org.hamcrest.Matchers.isOneOf; import static org.hamcrest.Matchers.not; -import com.google.common.collect.ImmutableSet; - import com.fasterxml.jackson.annotation.JsonIgnore; +import com.google.common.collect.ImmutableSet; +import java.util.Set; import org.hamcrest.FeatureMatcher; import org.hamcrest.Matcher; import org.hamcrest.Matchers; @@ -36,8 +36,6 @@ import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.util.Set; - /** * Unit tests for {@link PipelineOptionsReflector}. */ 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 b2efa61f69434..012a5b04c55d8 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 @@ -23,21 +23,19 @@ import static org.junit.Assert.assertNull; import static org.junit.Assert.assertTrue; -import com.google.common.collect.Lists; -import com.google.common.collect.Sets; - import com.fasterxml.jackson.annotation.JsonIgnore; import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.collect.Lists; +import com.google.common.collect.Sets; +import java.io.IOException; +import java.util.List; +import java.util.Set; import org.junit.Rule; import org.junit.Test; import org.junit.rules.ExpectedException; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.io.IOException; -import java.util.List; -import java.util.Set; - /** Unit tests for {@link PipelineOptions}. */ @RunWith(JUnit4.class) public class PipelineOptionsTest { diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/options/PipelineOptionsValidatorTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/options/PipelineOptionsValidatorTest.java index 2b684a8a047c0..80d2a60b678d2 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/options/PipelineOptionsValidatorTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/options/PipelineOptionsValidatorTest.java @@ -18,7 +18,6 @@ package org.apache.beam.sdk.options; import org.apache.beam.sdk.testing.CrashingRunner; - import org.junit.Rule; import org.junit.Test; import org.junit.rules.ExpectedException; 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 148b590807665..1ba6b43821b6c 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 @@ -32,19 +32,23 @@ import static org.junit.Assert.assertThat; import static org.junit.Assert.assertTrue; -import org.apache.beam.sdk.transforms.display.DisplayData; - +import com.fasterxml.jackson.annotation.JsonIgnore; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.databind.JsonMappingException; +import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; import com.google.common.collect.Maps; import com.google.common.testing.EqualsTester; - -import com.fasterxml.jackson.annotation.JsonIgnore; -import com.fasterxml.jackson.annotation.JsonProperty; -import com.fasterxml.jackson.databind.JsonMappingException; -import com.fasterxml.jackson.databind.ObjectMapper; - +import java.io.IOException; +import java.io.Serializable; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.Set; +import org.apache.beam.sdk.transforms.display.DisplayData; import org.hamcrest.Matchers; import org.joda.time.Instant; import org.junit.Rule; @@ -55,14 +59,6 @@ import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.io.IOException; -import java.io.Serializable; -import java.util.HashSet; -import java.util.List; -import java.util.Map; -import java.util.Objects; -import java.util.Set; - /** Tests for {@link ProxyInvocationHandler}. */ @RunWith(JUnit4.class) public class ProxyInvocationHandlerTest { diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/PipelineRunnerTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/PipelineRunnerTest.java index f9ce018fc4799..e98049738615c 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/PipelineRunnerTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/PipelineRunnerTest.java @@ -27,7 +27,6 @@ import org.apache.beam.sdk.testing.CrashingRunner; import org.apache.beam.sdk.util.GcsUtil; import org.apache.beam.sdk.util.TestCredential; - import org.junit.Assert; import org.junit.Before; import org.junit.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 9009a777a6456..def3a027e9d75 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 @@ -24,6 +24,9 @@ import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; +import java.io.File; +import java.util.Arrays; +import java.util.EnumSet; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.VoidCoder; @@ -41,7 +44,6 @@ import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionList; import org.apache.beam.sdk.values.PDone; - import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -49,10 +51,6 @@ import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.io.File; -import java.util.Arrays; -import java.util.EnumSet; - /** * Tests for {@link TransformTreeNode} and {@link TransformHierarchy}. */ diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/dataflow/TestCountingSource.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/dataflow/TestCountingSource.java index 10631c2aa6bca..b53d1fccefc27 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/dataflow/TestCountingSource.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/dataflow/TestCountingSource.java @@ -19,6 +19,11 @@ import static org.apache.beam.sdk.util.CoderUtils.encodeToByteArray; +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.ThreadLocalRandom; +import javax.annotation.Nullable; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.DelegateCoder; import org.apache.beam.sdk.coders.KvCoder; @@ -26,18 +31,10 @@ import org.apache.beam.sdk.io.UnboundedSource; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.values.KV; - import org.joda.time.Instant; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.io.IOException; -import java.util.ArrayList; -import java.util.List; -import java.util.concurrent.ThreadLocalRandom; - -import javax.annotation.Nullable; - /** * An unbounded source for testing the unbounded sources framework code. * diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/dataflow/TestCountingSourceTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/dataflow/TestCountingSourceTest.java index 6ba060e8cfefc..72a4585104cff 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/dataflow/TestCountingSourceTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/dataflow/TestCountingSourceTest.java @@ -22,15 +22,13 @@ import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; +import java.io.IOException; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.options.PipelineOptionsFactory; - import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.io.IOException; - /** * Test the TestCountingSource. */ diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/CoderPropertiesTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/CoderPropertiesTest.java index 5bf55b08d6191..f337f36215a92 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/CoderPropertiesTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/CoderPropertiesTest.java @@ -20,13 +20,14 @@ import static org.junit.Assert.assertThat; import static org.junit.Assert.fail; +import com.google.common.base.Strings; +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; import org.apache.beam.sdk.coders.Coder.Context; import org.apache.beam.sdk.coders.CoderException; import org.apache.beam.sdk.coders.CustomCoder; import org.apache.beam.sdk.coders.StringUtf8Coder; - -import com.google.common.base.Strings; - import org.hamcrest.CoreMatchers; import org.junit.Assert; import org.junit.Rule; @@ -35,10 +36,6 @@ import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.io.IOException; -import java.io.InputStream; -import java.io.OutputStream; - /** Unit tests for {@link CoderProperties}. */ @RunWith(JUnit4.class) public class CoderPropertiesTest { diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/CrashingRunnerTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/CrashingRunnerTest.java index 041a73ae2d261..c66aa50cb57f7 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/CrashingRunnerTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/CrashingRunnerTest.java @@ -26,7 +26,6 @@ import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.runners.PipelineRunner; import org.apache.beam.sdk.transforms.Create; - import org.junit.Rule; import org.junit.Test; import org.junit.rules.ExpectedException; diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/ExpectedLogs.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/ExpectedLogs.java index caf8fd7919d24..a8e3f94250312 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/ExpectedLogs.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/ExpectedLogs.java @@ -19,12 +19,6 @@ import static org.junit.Assert.fail; -import org.hamcrest.Description; -import org.hamcrest.Matcher; -import org.hamcrest.TypeSafeMatcher; -import org.junit.rules.ExternalResource; -import org.junit.rules.TestRule; - import java.util.Collection; import java.util.concurrent.ConcurrentLinkedDeque; import java.util.logging.Formatter; @@ -33,8 +27,12 @@ import java.util.logging.LogRecord; import java.util.logging.Logger; import java.util.logging.SimpleFormatter; - import javax.annotation.concurrent.ThreadSafe; +import org.hamcrest.Description; +import org.hamcrest.Matcher; +import org.hamcrest.TypeSafeMatcher; +import org.junit.rules.ExternalResource; +import org.junit.rules.TestRule; /** * This {@link TestRule} enables the ability to capture JUL logging events during test execution and diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/ExpectedLogsTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/ExpectedLogsTest.java index 1d7e18a0db155..84d55844f8252 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/ExpectedLogsTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/ExpectedLogsTest.java @@ -19,13 +19,6 @@ import static org.apache.beam.sdk.testing.SystemNanoTimeSleeper.sleepMillis; -import org.junit.Rule; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.JUnit4; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - import java.io.IOException; import java.util.ArrayList; import java.util.List; @@ -35,6 +28,12 @@ import java.util.concurrent.ExecutorCompletionService; import java.util.concurrent.Executors; import java.util.concurrent.TimeUnit; +import org.junit.Rule; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** Tests for {@link FastNanoClockAndSleeper}. */ @RunWith(JUnit4.class) diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/FastNanoClockAndSleeper.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/FastNanoClockAndSleeper.java index a97726be545f8..6bfafa5a8637e 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/FastNanoClockAndSleeper.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/FastNanoClockAndSleeper.java @@ -19,7 +19,6 @@ import com.google.api.client.util.NanoClock; import com.google.api.client.util.Sleeper; - import org.junit.rules.ExternalResource; import org.junit.rules.TestRule; diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/FastNanoClockAndSleeperTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/FastNanoClockAndSleeperTest.java index 9d15c0fcabb6e..7d209515421b9 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/FastNanoClockAndSleeperTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/FastNanoClockAndSleeperTest.java @@ -20,13 +20,12 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; +import java.util.concurrent.TimeUnit; import org.junit.Rule; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.util.concurrent.TimeUnit; - /** Tests for {@link FastNanoClockAndSleeper}. */ @RunWith(JUnit4.class) public class FastNanoClockAndSleeperTest { 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 d94ffe2c47615..b2f2ec853573a 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 @@ -20,9 +20,12 @@ import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.containsString; +import com.google.common.io.Files; +import java.io.File; +import java.io.IOException; +import java.nio.charset.StandardCharsets; import org.apache.beam.sdk.PipelineResult; import org.apache.beam.sdk.util.IOChannelUtils; -import com.google.common.io.Files; import org.junit.Rule; import org.junit.Test; import org.junit.rules.ExpectedException; @@ -32,10 +35,6 @@ import org.mockito.Mock; import org.mockito.Mockito; -import java.io.File; -import java.io.IOException; -import java.nio.charset.StandardCharsets; - /** Tests for {@link FileChecksumMatcher}. */ @RunWith(JUnit4.class) public class FileChecksumMatcherTest { 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 acc2b48f96442..54ddd3f99f1a9 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 @@ -22,6 +22,13 @@ import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.google.common.collect.Iterables; +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.io.Serializable; +import java.util.regex.Pattern; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.coders.AtomicCoder; import org.apache.beam.sdk.coders.CoderException; @@ -35,10 +42,6 @@ import org.apache.beam.sdk.util.common.ElementByteSizeObserver; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.TimestampedValue; - -import com.google.common.collect.Iterables; - -import com.fasterxml.jackson.annotation.JsonCreator; import org.joda.time.Duration; import org.joda.time.Instant; import org.junit.Rule; @@ -48,12 +51,6 @@ import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.io.IOException; -import java.io.InputStream; -import java.io.OutputStream; -import java.io.Serializable; -import java.util.regex.Pattern; - /** * Test case for {@link PAssert}. */ diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/PCollectionViewTesting.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/PCollectionViewTesting.java index 517ed689b87a3..ea8e0af5030bc 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/PCollectionViewTesting.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/PCollectionViewTesting.java @@ -17,6 +17,12 @@ */ package org.apache.beam.sdk.testing; +import com.google.common.base.Function; +import com.google.common.base.MoreObjects; +import com.google.common.collect.Iterables; +import com.google.common.collect.Lists; +import java.util.List; +import java.util.Objects; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.IterableCoder; import org.apache.beam.sdk.transforms.ViewFn; @@ -29,18 +35,9 @@ import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.PValueBase; import org.apache.beam.sdk.values.TupleTag; - -import com.google.common.base.Function; -import com.google.common.base.MoreObjects; -import com.google.common.collect.Iterables; -import com.google.common.collect.Lists; - import org.joda.time.Duration; import org.joda.time.Instant; -import java.util.List; -import java.util.Objects; - /** * Methods for creating and using {@link PCollectionView} instances. */ 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 1487eeae0dda4..ef501d495c50c 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 @@ -21,14 +21,12 @@ import static org.hamcrest.Matchers.emptyIterable; import static org.junit.Assert.assertThat; +import com.google.common.collect.ImmutableList; import org.apache.beam.sdk.transforms.SerializableFunction; 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 com.google.common.collect.ImmutableList; - import org.joda.time.Instant; import org.junit.Rule; import org.junit.Test; diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/RestoreSystemProperties.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/RestoreSystemProperties.java index 423026d28d234..d1d0507a7606e 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/RestoreSystemProperties.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/RestoreSystemProperties.java @@ -17,12 +17,11 @@ */ package org.apache.beam.sdk.testing; -import org.junit.rules.ExternalResource; -import org.junit.rules.TestRule; - import java.io.ByteArrayInputStream; import java.io.ByteArrayOutputStream; import java.io.IOException; +import org.junit.rules.ExternalResource; +import org.junit.rules.TestRule; /** * Saves and restores the current system properties for tests. diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/SerializableMatchersTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/SerializableMatchersTest.java index 7cbd4a8c8b926..db5ff2e3049bb 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/SerializableMatchersTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/SerializableMatchersTest.java @@ -22,17 +22,17 @@ import static org.apache.beam.sdk.testing.SerializableMatchers.containsInAnyOrder; import static org.apache.beam.sdk.testing.SerializableMatchers.kvWithKey; import static org.apache.beam.sdk.testing.SerializableMatchers.not; - import static org.junit.Assert.assertThat; import static org.junit.Assert.fail; +import com.google.common.collect.ImmutableList; +import java.io.InputStream; +import java.io.OutputStream; +import java.io.Serializable; import org.apache.beam.sdk.coders.AtomicCoder; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.util.SerializableUtils; import org.apache.beam.sdk.values.KV; - -import com.google.common.collect.ImmutableList; - import org.hamcrest.Matchers; import org.junit.Rule; import org.junit.Test; @@ -40,10 +40,6 @@ import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.io.InputStream; -import java.io.OutputStream; -import java.io.Serializable; - /** * Test case for {@link SerializableMatchers}. * diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/SourceTestUtilsTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/SourceTestUtilsTest.java index f2b332bb7dd3a..efb385de630ad 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/SourceTestUtilsTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/SourceTestUtilsTest.java @@ -20,21 +20,18 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNull; +import com.google.common.collect.Sets; +import java.util.List; +import java.util.Set; import org.apache.beam.sdk.io.BoundedSource; import org.apache.beam.sdk.io.BoundedSource.BoundedReader; import org.apache.beam.sdk.io.CountingSource; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.options.PipelineOptionsFactory; - -import com.google.common.collect.Sets; - import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.util.List; -import java.util.Set; - /** * Tests for {@link SourceTestUtils}. */ diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/StaticWindowsTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/StaticWindowsTest.java index fd715dc5a5674..e6626190006ef 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/StaticWindowsTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/StaticWindowsTest.java @@ -19,13 +19,11 @@ import static org.junit.Assert.assertThat; +import com.google.common.collect.ImmutableList; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.GlobalWindow; import org.apache.beam.sdk.transforms.windowing.IntervalWindow; import org.apache.beam.sdk.transforms.windowing.WindowFn; - -import com.google.common.collect.ImmutableList; - import org.hamcrest.Matchers; import org.joda.time.Instant; import org.junit.Rule; diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/SystemNanoTimeSleeper.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/SystemNanoTimeSleeper.java index 254a1d57e2b98..810b6f182aaa1 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/SystemNanoTimeSleeper.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/SystemNanoTimeSleeper.java @@ -18,7 +18,6 @@ package org.apache.beam.sdk.testing; import com.google.api.client.util.Sleeper; - import java.util.concurrent.TimeUnit; import java.util.concurrent.locks.LockSupport; diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/SystemNanoTimeSleeperTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/SystemNanoTimeSleeperTest.java index 623224d838967..fe97675e7e7a7 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/SystemNanoTimeSleeperTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/SystemNanoTimeSleeperTest.java @@ -18,7 +18,6 @@ package org.apache.beam.sdk.testing; import static org.apache.beam.sdk.testing.SystemNanoTimeSleeper.sleepMillis; - import static org.junit.Assert.assertTrue; import org.junit.Test; 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 0bd789326d3e4..ed65f158753de 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/TestPipelineTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/TestPipelineTest.java @@ -23,14 +23,17 @@ import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertThat; +import com.fasterxml.jackson.databind.ObjectMapper; +import java.util.Arrays; +import java.util.Date; +import java.util.List; +import java.util.UUID; import org.apache.beam.sdk.PipelineResult; import org.apache.beam.sdk.options.ApplicationNameOptions; import org.apache.beam.sdk.options.GcpOptions; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.transforms.Create; - -import com.fasterxml.jackson.databind.ObjectMapper; import org.hamcrest.BaseMatcher; import org.hamcrest.Description; import org.junit.Rule; @@ -40,11 +43,6 @@ import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.util.Arrays; -import java.util.Date; -import java.util.List; -import java.util.UUID; - /** Tests for {@link TestPipeline}. */ @RunWith(JUnit4.class) public class TestPipelineTest { 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 df37d7f967a54..6457f910308f4 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 @@ -23,6 +23,7 @@ import static org.hamcrest.Matchers.lessThanOrEqualTo; import static org.junit.Assert.assertThat; +import java.io.Serializable; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.coders.StringUtf8Coder; import org.apache.beam.sdk.coders.VarIntCoder; @@ -50,7 +51,6 @@ import org.apache.beam.sdk.transforms.windowing.Window.ClosingBehavior; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.TimestampedValue; - import org.joda.time.Duration; import org.joda.time.Instant; import org.junit.Rule; @@ -60,8 +60,6 @@ import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.io.Serializable; - /** * Tests for {@link TestStream}. */ diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/WindowSupplierTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/WindowSupplierTest.java index 1a2fd1d560416..1ab4c275eac8f 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/WindowSupplierTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/WindowSupplierTest.java @@ -19,14 +19,16 @@ import static org.junit.Assert.assertThat; +import com.google.common.collect.ImmutableList; +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.util.Collections; import org.apache.beam.sdk.coders.AtomicCoder; import org.apache.beam.sdk.coders.CoderException; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.IntervalWindow; import org.apache.beam.sdk.util.SerializableUtils; - -import com.google.common.collect.ImmutableList; - import org.hamcrest.Matchers; import org.joda.time.Instant; import org.junit.Rule; @@ -35,11 +37,6 @@ import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.io.IOException; -import java.io.InputStream; -import java.io.OutputStream; -import java.util.Collections; - /** * Tests for {@link WindowSupplier}. */ 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 fc10d4b29b379..ab1394697c4fa 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 @@ -19,10 +19,14 @@ import static org.apache.beam.sdk.TestUtils.checkCombineFn; import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasDisplayItem; - import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.collection.IsIterableContainingInOrder.contains; +import java.io.Serializable; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Comparator; +import java.util.List; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.coders.BigEndianIntegerCoder; import org.apache.beam.sdk.coders.KvCoder; @@ -34,7 +38,6 @@ import org.apache.beam.sdk.transforms.display.DisplayData; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; - import org.hamcrest.CoreMatchers; import org.hamcrest.Description; import org.hamcrest.Matcher; @@ -44,12 +47,6 @@ import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.io.Serializable; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Comparator; -import java.util.List; - /** * Tests for {@link ApproximateQuantiles}. */ 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 7b6d671aa9351..ba1ddfeaf2cde 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 @@ -18,12 +18,18 @@ package org.apache.beam.sdk.transforms; import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasDisplayItem; - import static org.hamcrest.MatcherAssert.assertThat; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.Lists; +import java.io.Serializable; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.TestUtils; import org.apache.beam.sdk.testing.NeedsRunner; @@ -34,21 +40,11 @@ import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionView; - -import com.google.common.collect.ImmutableList; -import com.google.common.collect.Lists; - import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.io.Serializable; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collections; -import java.util.List; - /** * Tests for the ApproximateUnique aggregator transform. */ 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 95ba1aa0d8d14..35f985855dae9 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 @@ -19,9 +19,15 @@ import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasDisplayItem; import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.includesDisplayDataFrom; - import static org.junit.Assert.assertThat; +import com.google.common.collect.ImmutableList; +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +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; @@ -43,9 +49,6 @@ import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TupleTag; - -import com.google.common.collect.ImmutableList; - import org.hamcrest.Matchers; import org.junit.Rule; import org.junit.Test; @@ -54,13 +57,6 @@ import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.io.IOException; -import java.io.InputStream; -import java.io.OutputStream; -import java.io.Serializable; -import java.util.Arrays; -import java.util.List; - /** * Unit tests for {@link CombineFns}. */ 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 897d17a036380..77a1d6b03e69c 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 @@ -17,18 +17,31 @@ */ package org.apache.beam.sdk.transforms; +import static com.google.common.base.Preconditions.checkArgument; +import static com.google.common.base.Preconditions.checkNotNull; +import static com.google.common.base.Preconditions.checkState; import static org.apache.beam.sdk.TestUtils.checkCombineFn; import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasDisplayItem; import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasNamespace; import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.includesDisplayDataFrom; -import static com.google.common.base.Preconditions.checkArgument; -import static com.google.common.base.Preconditions.checkNotNull; -import static com.google.common.base.Preconditions.checkState; - import static org.hamcrest.Matchers.hasItem; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertThat; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.base.MoreObjects; +import com.google.common.collect.ImmutableSet; +import com.google.common.collect.Sets; +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.io.Serializable; +import java.util.Arrays; +import java.util.HashSet; +import java.util.List; +import java.util.Objects; +import java.util.Set; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.coders.BigEndianIntegerCoder; import org.apache.beam.sdk.coders.BigEndianLongCoder; @@ -66,13 +79,6 @@ import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.POutput; - -import com.google.common.base.MoreObjects; -import com.google.common.collect.ImmutableSet; -import com.google.common.collect.Sets; - -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonProperty; import org.hamcrest.Matchers; import org.joda.time.Duration; import org.junit.Test; @@ -81,16 +87,6 @@ import org.junit.runners.JUnit4; import org.mockito.Mock; -import java.io.IOException; -import java.io.InputStream; -import java.io.OutputStream; -import java.io.Serializable; -import java.util.Arrays; -import java.util.HashSet; -import java.util.List; -import java.util.Objects; -import java.util.Set; - /** * Tests for Combine transforms. */ 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 ca898b3bc2b9c..7f77ae7f64c38 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 @@ -18,9 +18,10 @@ package org.apache.beam.sdk.transforms; import static org.apache.beam.sdk.TestUtils.NO_LINES; - import static org.junit.Assert.assertEquals; +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; @@ -28,15 +29,11 @@ import org.apache.beam.sdk.testing.TestPipeline; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; - import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.util.Arrays; -import java.util.List; - /** * Tests for Count. */ 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 9db01368cc66b..2a89a1837df73 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 @@ -21,13 +21,23 @@ import static org.apache.beam.sdk.TestUtils.LINES_ARRAY; import static org.apache.beam.sdk.TestUtils.NO_LINES; import static org.apache.beam.sdk.TestUtils.NO_LINES_ARRAY; - import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.hasSize; import static org.hamcrest.Matchers.is; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertThat; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.Lists; +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.io.Serializable; +import java.util.ArrayList; +import java.util.Arrays; +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; @@ -50,10 +60,6 @@ import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.TimestampedValue; - -import com.google.common.collect.ImmutableList; -import com.google.common.collect.Lists; - import org.hamcrest.Matchers; import org.joda.time.Instant; import org.junit.Rule; @@ -63,16 +69,6 @@ import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.io.IOException; -import java.io.InputStream; -import java.io.OutputStream; -import java.io.Serializable; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collections; -import java.util.List; -import java.util.Random; - /** * Tests for Create. */ diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/DoFnDelegatingAggregatorTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/DoFnDelegatingAggregatorTest.java index 4e8d06c3e5970..25b909aabbaf3 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/DoFnDelegatingAggregatorTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/DoFnDelegatingAggregatorTest.java @@ -25,7 +25,6 @@ import org.apache.beam.sdk.transforms.Combine.CombineFn; import org.apache.beam.sdk.transforms.OldDoFn.DelegatingAggregator; - import org.junit.Before; import org.junit.Rule; import org.junit.Test; 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 3fb3193f4d5ad..e5f5cb6547f5b 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 @@ -23,13 +23,13 @@ import static org.junit.Assert.assertNotEquals; import static org.junit.Assert.assertThat; +import java.io.Serializable; import org.apache.beam.sdk.Pipeline.PipelineExecutionException; import org.apache.beam.sdk.testing.NeedsRunner; import org.apache.beam.sdk.testing.TestPipeline; import org.apache.beam.sdk.transforms.Combine.CombineFn; import org.apache.beam.sdk.transforms.Max.MaxIntegerFn; import org.apache.beam.sdk.transforms.display.DisplayData; - import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -37,8 +37,6 @@ import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.io.Serializable; - /** Tests for {@link DoFn}. */ @RunWith(JUnit4.class) public class DoFnTest implements Serializable { 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 e379f11279238..2f1519cc42764 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 @@ -24,6 +24,7 @@ import static org.junit.Assert.assertThat; import static org.junit.Assert.assertTrue; +import java.util.List; import org.apache.beam.sdk.coders.VarIntCoder; import org.apache.beam.sdk.testing.TestPipeline; import org.apache.beam.sdk.transforms.windowing.GlobalWindow; @@ -32,15 +33,12 @@ import org.apache.beam.sdk.util.WindowingStrategy; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TimestampedValue; - import org.hamcrest.Matchers; import org.joda.time.Instant; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.util.List; - /** * Tests for {@link DoFnTester}. */ 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 2edab05418e86..5221f75b956b0 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 @@ -18,22 +18,19 @@ package org.apache.beam.sdk.transforms; import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasDisplayItem; - import static org.hamcrest.MatcherAssert.assertThat; +import java.io.Serializable; 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.display.DisplayData; import org.apache.beam.sdk.values.PCollection; - import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.io.Serializable; - /** * Tests for {@link Filter}. */ 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 cb7892cad5729..311c8de369d45 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 @@ -18,10 +18,15 @@ package org.apache.beam.sdk.transforms; import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasDisplayItem; - import static org.hamcrest.Matchers.equalTo; import static org.junit.Assert.assertThat; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableSet; +import java.io.Serializable; +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; @@ -30,10 +35,6 @@ import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.TypeDescriptor; - -import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableSet; - import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -41,11 +42,6 @@ import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.io.Serializable; -import java.util.Collections; -import java.util.List; -import java.util.Set; - /** * Tests for {@link FlatMapElements}. */ 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 3469223c38fe2..d755e28b8548e 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 @@ -23,6 +23,13 @@ import static org.apache.beam.sdk.TestUtils.NO_LINES; import static org.apache.beam.sdk.TestUtils.NO_LINES_ARRAY; +import com.google.common.collect.ImmutableSet; +import java.io.Serializable; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.List; +import java.util.Set; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.CollectionCoder; @@ -41,9 +48,6 @@ import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionList; import org.apache.beam.sdk.values.PCollectionView; - -import com.google.common.collect.ImmutableSet; - import org.joda.time.Duration; import org.junit.Assert; import org.junit.Rule; @@ -53,13 +57,6 @@ import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.io.Serializable; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collection; -import java.util.List; -import java.util.Set; - /** * Tests for Flatten. */ 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 afe460fcdc7ce..bea0e2d901851 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 @@ -19,7 +19,6 @@ import static org.apache.beam.sdk.TestUtils.KvMatcher.isKv; import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasDisplayItem; - import static org.hamcrest.CoreMatchers.equalTo; import static org.hamcrest.CoreMatchers.hasItem; import static org.hamcrest.Matchers.empty; @@ -27,6 +26,19 @@ import static org.hamcrest.core.Is.is; import static org.junit.Assert.assertThat; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.google.common.base.Function; +import com.google.common.collect.Iterables; +import java.io.DataInputStream; +import java.io.DataOutputStream; +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.util.ArrayList; +import java.util.Arrays; +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; @@ -51,11 +63,6 @@ import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.TimestampedValue; import org.apache.beam.sdk.values.TypeDescriptor; - -import com.google.common.base.Function; -import com.google.common.collect.Iterables; - -import com.fasterxml.jackson.annotation.JsonCreator; import org.joda.time.Duration; import org.joda.time.Instant; import org.junit.Assert; @@ -66,17 +73,6 @@ import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.io.DataInputStream; -import java.io.DataOutputStream; -import java.io.IOException; -import java.io.InputStream; -import java.io.OutputStream; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.List; -import java.util.Map; -import java.util.concurrent.ThreadLocalRandom; - /** * Tests for GroupByKey. */ diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/IntraBundleParallelizationTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/IntraBundleParallelizationTest.java index fa2fae96d419c..b9afd35513df8 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/IntraBundleParallelizationTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/IntraBundleParallelizationTest.java @@ -20,7 +20,6 @@ import static org.apache.beam.sdk.testing.SystemNanoTimeSleeper.sleepMillis; import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasDisplayItem; import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.includesDisplayDataFrom; - import static org.hamcrest.Matchers.both; import static org.hamcrest.Matchers.containsString; import static org.hamcrest.Matchers.greaterThanOrEqualTo; @@ -31,20 +30,18 @@ import static org.junit.Assert.assertThat; import static org.junit.Assert.fail; +import java.util.ArrayList; +import java.util.concurrent.atomic.AtomicInteger; 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.display.DisplayData; - import org.junit.Before; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.util.ArrayList; -import java.util.concurrent.atomic.AtomicInteger; - /** * Tests for RateLimiter. */ 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 cf3094068bf23..fce5b2f205c34 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 @@ -17,6 +17,7 @@ */ 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; @@ -26,14 +27,11 @@ import org.apache.beam.sdk.testing.TestPipeline; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; - import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.util.Arrays; - /** * Tests for Keys transform. */ 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 1a2d7f6ca0606..3598198754156 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 @@ -17,6 +17,7 @@ */ 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; @@ -26,14 +27,11 @@ import org.apache.beam.sdk.testing.TestPipeline; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; - import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.util.Arrays; - /** * Tests for KvSwap transform. */ 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 7217bca663fae..4a34c57b58b6c 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 @@ -18,11 +18,12 @@ package org.apache.beam.sdk.transforms; import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasDisplayItem; - import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.hasItem; import static org.junit.Assert.assertThat; +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; @@ -33,7 +34,6 @@ 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; @@ -41,9 +41,6 @@ import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.io.Serializable; -import java.util.Set; - /** * Tests for {@link MapElements}. */ diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/MaxTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/MaxTest.java index 87fa5541fa6bb..5c78b3f15bb3a 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/MaxTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/MaxTest.java @@ -19,14 +19,11 @@ import static org.apache.beam.sdk.TestUtils.checkCombineFn; import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasDisplayItem; - import static org.hamcrest.MatcherAssert.assertThat; import static org.junit.Assert.assertEquals; -import org.apache.beam.sdk.transforms.display.DisplayData; - import com.google.common.collect.Lists; - +import org.apache.beam.sdk.transforms.display.DisplayData; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/MeanTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/MeanTest.java index 65c876ef07547..1c94e35a957a6 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/MeanTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/MeanTest.java @@ -18,23 +18,19 @@ package org.apache.beam.sdk.transforms; import static org.apache.beam.sdk.TestUtils.checkCombineFn; - import static org.junit.Assert.assertEquals; +import com.google.common.collect.Lists; +import java.util.Arrays; +import java.util.List; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.testing.CoderProperties; import org.apache.beam.sdk.transforms.Mean.CountSum; import org.apache.beam.sdk.transforms.Mean.CountSumCoder; - -import com.google.common.collect.Lists; - import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.util.Arrays; -import java.util.List; - /** * Tests for Mean. */ diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/MinTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/MinTest.java index cd03a74c18945..a0eca072e2bf4 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/MinTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/MinTest.java @@ -20,14 +20,11 @@ import static org.apache.beam.sdk.TestUtils.checkCombineFn; import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasDisplayItem; - import static org.hamcrest.MatcherAssert.assertThat; import static org.junit.Assert.assertEquals; -import org.apache.beam.sdk.transforms.display.DisplayData; - import com.google.common.collect.Lists; - +import org.apache.beam.sdk.transforms.display.DisplayData; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; 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 8f2bd5e4a43a9..c73251000650a 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 @@ -24,7 +24,6 @@ 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; /** 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 5946d9a0dff94..e7ae135c5683d 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 @@ -24,6 +24,9 @@ import static org.junit.Assert.assertNotEquals; import static org.junit.Assert.assertThat; +import com.google.common.collect.ImmutableMap; +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; @@ -34,9 +37,6 @@ 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 com.google.common.collect.ImmutableMap; - import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -44,9 +44,6 @@ import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.io.Serializable; -import java.util.Map; - /** * Tests for OldDoFn. */ 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 f7074c35714ff..bfe8225c9dc93 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 @@ -22,7 +22,6 @@ import org.apache.beam.sdk.transforms.display.DisplayData; import org.apache.beam.sdk.values.PCollection; - import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; 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 c4ba8b7049b5b..f69c867bbe22d 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 @@ -25,21 +25,19 @@ import static org.junit.Assert.assertThat; import static org.junit.Assert.fail; +import java.io.Serializable; +import java.util.concurrent.atomic.AtomicBoolean; import org.apache.beam.sdk.testing.NeedsRunner; import org.apache.beam.sdk.testing.RunnableOnService; import org.apache.beam.sdk.testing.TestPipeline; import org.apache.beam.sdk.values.PCollectionList; import org.apache.beam.sdk.values.TupleTag; import org.apache.beam.sdk.values.TupleTagList; - import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.io.Serializable; -import java.util.concurrent.atomic.AtomicBoolean; - /** * Tests that {@link ParDo} exercises {@link DoFn} methods in the appropriate sequence. */ 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 13dec9adfee87..0a4b3cd271f88 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 @@ -17,6 +17,7 @@ */ package org.apache.beam.sdk.transforms; +import static com.google.common.base.Preconditions.checkNotNull; import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasDisplayItem; import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasKey; import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasType; @@ -24,7 +25,6 @@ import static org.apache.beam.sdk.util.SerializableUtils.serializeToByteArray; import static org.apache.beam.sdk.util.StringUtils.byteArrayToJsonString; import static org.apache.beam.sdk.util.StringUtils.jsonStringToByteArray; -import static com.google.common.base.Preconditions.checkNotNull; import static org.hamcrest.Matchers.allOf; import static org.hamcrest.Matchers.anyOf; import static org.hamcrest.Matchers.equalTo; @@ -34,6 +34,15 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertThat; +import com.fasterxml.jackson.annotation.JsonCreator; +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.io.Serializable; +import java.util.ArrayList; +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; @@ -56,9 +65,6 @@ import org.apache.beam.sdk.values.TimestampedValue; import org.apache.beam.sdk.values.TupleTag; import org.apache.beam.sdk.values.TupleTagList; - -import com.fasterxml.jackson.annotation.JsonCreator; - import org.joda.time.Duration; import org.joda.time.Instant; import org.junit.Rule; @@ -68,15 +74,6 @@ import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.io.IOException; -import java.io.InputStream; -import java.io.OutputStream; -import java.io.Serializable; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collections; -import java.util.List; - /** * Tests for ParDo. */ 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 0cc804ef62131..1cbe344adc54b 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 @@ -18,11 +18,13 @@ package org.apache.beam.sdk.transforms; import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasDisplayItem; - import static org.hamcrest.MatcherAssert.assertThat; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; +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; @@ -32,7 +34,6 @@ import org.apache.beam.sdk.transforms.display.DisplayData; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionList; - import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -40,10 +41,6 @@ import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.io.Serializable; -import java.util.ArrayList; -import java.util.List; - /** * Tests for {@link Partition}. */ diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/RemoveDuplicatesTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/RemoveDuplicatesTest.java index 0e919d2c83b7b..312cba6b45fbe 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/RemoveDuplicatesTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/RemoveDuplicatesTest.java @@ -20,6 +20,10 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; +import java.util.Arrays; +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; @@ -27,17 +31,11 @@ import org.apache.beam.sdk.testing.TestPipeline; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; - import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.util.Arrays; -import java.util.HashMap; -import java.util.List; -import java.util.Map; - /** * Tests for RemovedDuplicates. */ 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 e7f8cd00930e0..a0555fa60800b 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 @@ -17,16 +17,21 @@ */ package org.apache.beam.sdk.transforms; +import static com.google.common.base.Preconditions.checkArgument; import static org.apache.beam.sdk.TestUtils.LINES; import static org.apache.beam.sdk.TestUtils.NO_LINES; import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasDisplayItem; - -import static com.google.common.base.Preconditions.checkArgument; - import static org.hamcrest.MatcherAssert.assertThat; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; +import com.google.common.base.Joiner; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashSet; +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; @@ -36,21 +41,11 @@ import org.apache.beam.sdk.testing.TestPipeline; import org.apache.beam.sdk.transforms.display.DisplayData; import org.apache.beam.sdk.values.PCollection; - -import com.google.common.base.Joiner; - import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.util.ArrayList; -import java.util.Collections; -import java.util.HashSet; -import java.util.List; -import java.util.Set; -import java.util.TreeSet; - /** * Tests for Sample transform. */ diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/SimpleStatsFnsTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/SimpleStatsFnsTest.java index dc8355b2620a8..a782ecc201529 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/SimpleStatsFnsTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/SimpleStatsFnsTest.java @@ -19,15 +19,14 @@ import static org.junit.Assert.assertEquals; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; import org.joda.time.Instant; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.util.Arrays; -import java.util.Collections; -import java.util.List; - /** * Tests of Min, Max, Mean, and Sum. */ diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/SumTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/SumTest.java index 4af6db19a58d6..b4f723d0d6f80 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/SumTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/SumTest.java @@ -18,19 +18,16 @@ package org.apache.beam.sdk.transforms; import static org.apache.beam.sdk.TestUtils.checkCombineFn; - import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNotEquals; +import com.google.common.collect.Lists; import org.apache.beam.sdk.coders.BigEndianIntegerCoder; import org.apache.beam.sdk.coders.BigEndianLongCoder; import org.apache.beam.sdk.coders.CoderRegistry; import org.apache.beam.sdk.coders.DoubleCoder; import org.apache.beam.sdk.coders.VarIntCoder; import org.apache.beam.sdk.coders.VarLongCoder; - -import com.google.common.collect.Lists; - import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; 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 fc0e659f5b9a6..b6242524b6a31 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 @@ -18,10 +18,14 @@ package org.apache.beam.sdk.transforms; import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasDisplayItem; - import static org.hamcrest.MatcherAssert.assertThat; import static org.junit.Assert.assertEquals; +import java.io.Serializable; +import java.util.Arrays; +import java.util.Collections; +import java.util.Comparator; +import java.util.List; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.coders.BigEndianIntegerCoder; import org.apache.beam.sdk.coders.KvCoder; @@ -35,7 +39,6 @@ import org.apache.beam.sdk.transforms.windowing.Window.Bound; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; - import org.hamcrest.Matchers; import org.joda.time.Duration; import org.junit.Rule; @@ -45,12 +48,6 @@ import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.io.Serializable; -import java.util.Arrays; -import java.util.Collections; -import java.util.Comparator; -import java.util.List; - /** Tests for Top. */ @RunWith(JUnit4.class) public class TopTest { 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 238ba7b648424..0bf2e2e9232e8 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 @@ -19,6 +19,7 @@ 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; @@ -28,14 +29,11 @@ import org.apache.beam.sdk.testing.TestPipeline; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; - import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.util.Arrays; - /** * Tests for Values transform. */ 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 170e6ce093de8..69b618607057c 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 @@ -17,16 +17,26 @@ */ package org.apache.beam.sdk.transforms; -import static org.apache.beam.sdk.values.KV.of; - import static com.google.common.base.Preconditions.checkArgument; - +import static org.apache.beam.sdk.values.KV.of; import static org.hamcrest.Matchers.isA; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; +import com.google.common.collect.ImmutableList; +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.io.Serializable; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; +import java.util.NoSuchElementException; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.Pipeline.PipelineExecutionException; import org.apache.beam.sdk.coders.Coder; @@ -52,9 +62,6 @@ import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TimestampedValue; import org.apache.beam.sdk.values.TypeDescriptor; - -import com.google.common.collect.ImmutableList; - import org.hamcrest.Matchers; import org.joda.time.Duration; import org.joda.time.Instant; @@ -66,18 +73,6 @@ import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.io.IOException; -import java.io.InputStream; -import java.io.OutputStream; -import java.io.Serializable; -import java.util.ArrayList; -import java.util.HashMap; -import java.util.Iterator; -import java.util.List; -import java.util.Map; -import java.util.Map.Entry; -import java.util.NoSuchElementException; - /** * Tests for {@link View}. See also {@link ParDoTest}, which * provides additional coverage since views can only be 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 c23dd3694ad76..f958807fcf771 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 @@ -19,6 +19,8 @@ import static org.junit.Assert.assertEquals; +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; @@ -27,15 +29,11 @@ import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.TypeDescriptor; - import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.util.Arrays; -import java.util.List; - /** * Tests for ExtractKeys transform. */ 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 e3814708788ee..923b97c52ee38 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 @@ -19,6 +19,7 @@ import static org.hamcrest.Matchers.isA; +import java.io.Serializable; import org.apache.beam.sdk.Pipeline.PipelineExecutionException; import org.apache.beam.sdk.testing.NeedsRunner; import org.apache.beam.sdk.testing.PAssert; @@ -26,7 +27,6 @@ import org.apache.beam.sdk.testing.TestPipeline; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; - import org.joda.time.Duration; import org.joda.time.Instant; import org.junit.Rule; @@ -36,8 +36,6 @@ import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.io.Serializable; - /** * Tests for {@link WithTimestamps}. */ 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 dc8c1e9fe90c6..1783a73101d3c 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 @@ -17,6 +17,9 @@ */ package org.apache.beam.sdk.transforms.display; +import com.google.common.collect.Sets; +import java.util.Objects; +import java.util.Set; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.options.PipelineOptions; @@ -29,11 +32,6 @@ import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.POutput; -import com.google.common.collect.Sets; - -import java.util.Objects; -import java.util.Set; - /** * Test utilities to evaluate the {@link DisplayData} in the context of a {@link PipelineRunner}. */ 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 e2331146ea8df..7630779bb3e18 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 @@ -18,11 +18,12 @@ package org.apache.beam.sdk.transforms.display; import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasDisplayItem; - import static org.hamcrest.Matchers.hasItem; import static org.hamcrest.Matchers.not; import static org.junit.Assert.assertThat; +import java.io.Serializable; +import java.util.Set; import org.apache.beam.sdk.io.TextIO; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.PTransform; @@ -30,14 +31,10 @@ import org.apache.beam.sdk.values.PBegin; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.POutput; - import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.io.Serializable; -import java.util.Set; - /** * Unit tests for {@link DisplayDataEvaluator}. */ diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/display/DisplayDataMatchers.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/display/DisplayDataMatchers.java index 025a1f765e5d3..e9db5222232bb 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/display/DisplayDataMatchers.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/display/DisplayDataMatchers.java @@ -19,10 +19,9 @@ import static org.hamcrest.Matchers.allOf; -import org.apache.beam.sdk.transforms.display.DisplayData.Item; - import com.google.common.collect.Sets; - +import java.util.Collection; +import org.apache.beam.sdk.transforms.display.DisplayData.Item; import org.hamcrest.CustomTypeSafeMatcher; import org.hamcrest.Description; import org.hamcrest.FeatureMatcher; @@ -32,8 +31,6 @@ import org.joda.time.Duration; import org.joda.time.Instant; -import java.util.Collection; - /** * Hamcrest matcher for making assertions on {@link DisplayData} instances. */ 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 fa44390ab627f..3ea6830cbc3fd 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 @@ -22,7 +22,6 @@ import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasType; import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasValue; import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.includesDisplayDataFrom; - import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertThat; @@ -30,7 +29,6 @@ import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.display.DisplayData.Builder; import org.apache.beam.sdk.values.PCollection; - import org.hamcrest.Description; import org.hamcrest.Matcher; import org.hamcrest.StringDescription; 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 e2f38b44190a7..a709bd80af477 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 @@ -24,7 +24,6 @@ import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasType; import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasValue; import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.includesDisplayDataFrom; - import static org.hamcrest.Matchers.allOf; import static org.hamcrest.Matchers.empty; import static org.hamcrest.Matchers.everyItem; @@ -41,20 +40,23 @@ import static org.junit.Assert.assertThat; import static org.junit.Assert.fail; +import com.fasterxml.jackson.databind.JsonNode; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.ImmutableMultimap; +import com.google.common.collect.Multimap; +import com.google.common.testing.EqualsTester; +import java.io.IOException; +import java.io.Serializable; +import java.util.Collection; +import java.util.Map; +import java.util.regex.Pattern; 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.display.DisplayData.Builder; import org.apache.beam.sdk.transforms.display.DisplayData.Item; import org.apache.beam.sdk.values.PCollection; - -import com.google.common.collect.ImmutableMap; -import com.google.common.collect.ImmutableMultimap; -import com.google.common.collect.Multimap; -import com.google.common.testing.EqualsTester; - -import com.fasterxml.jackson.databind.JsonNode; -import com.fasterxml.jackson.databind.ObjectMapper; import org.hamcrest.CustomTypeSafeMatcher; import org.hamcrest.FeatureMatcher; import org.hamcrest.Matcher; @@ -69,12 +71,6 @@ import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.io.IOException; -import java.io.Serializable; -import java.util.Collection; -import java.util.Map; -import java.util.regex.Pattern; - /** * Tests for {@link DisplayData} class. */ diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/join/CoGbkResultCoderTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/join/CoGbkResultCoderTest.java index 0a0a3f1320200..18ecd9bab2e60 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/join/CoGbkResultCoderTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/join/CoGbkResultCoderTest.java @@ -19,6 +19,7 @@ import static org.junit.Assert.assertFalse; +import com.google.common.collect.ImmutableList; import org.apache.beam.sdk.coders.BigEndianIntegerCoder; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.DoubleCoder; @@ -28,9 +29,6 @@ import org.apache.beam.sdk.transforms.join.CoGbkResult.CoGbkResultCoder; import org.apache.beam.sdk.values.TupleTag; import org.apache.beam.sdk.values.TupleTagList; - -import com.google.common.collect.ImmutableList; - import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/join/CoGbkResultTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/join/CoGbkResultTest.java index aa7cb5c3e5a80..50a623772d135 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/join/CoGbkResultTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/join/CoGbkResultTest.java @@ -22,18 +22,16 @@ import static org.hamcrest.Matchers.equalTo; import static org.junit.Assert.assertThat; +import java.util.ArrayList; +import java.util.List; import org.apache.beam.sdk.util.common.Reiterable; import org.apache.beam.sdk.util.common.Reiterator; import org.apache.beam.sdk.values.TupleTag; import org.apache.beam.sdk.values.TupleTagList; - import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.util.ArrayList; -import java.util.List; - /** * Tests the CoGbkResult. */ 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 c6f82ecb0ca41..e8c8b15ddbfdf 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 @@ -21,6 +21,12 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertThat; +import com.google.common.collect.Iterables; +import java.io.Serializable; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.Map; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.coders.BigEndianIntegerCoder; import org.apache.beam.sdk.coders.KvCoder; @@ -40,21 +46,12 @@ import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.TupleTag; - -import com.google.common.collect.Iterables; - import org.joda.time.Duration; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.io.Serializable; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.List; -import java.util.Map; - /** * Tests for CoGroupByKeyTest. Implements Serializable for anonymous DoFns. */ diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/join/UnionCoderTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/join/UnionCoderTest.java index fb80fb65490df..41ba95289dc1d 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/join/UnionCoderTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/join/UnionCoderTest.java @@ -19,18 +19,16 @@ import static org.junit.Assert.assertEquals; +import java.util.Arrays; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.DoubleCoder; import org.apache.beam.sdk.coders.StringUtf8Coder; import org.apache.beam.sdk.util.CloudObject; import org.apache.beam.sdk.util.Serializer; - import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.util.Arrays; - /** * Tests the UnionCoder. */ 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 28f0143d69c14..9317ea21d5588 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 @@ -24,7 +24,6 @@ import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.util.UserCodeException; - import org.junit.Before; import org.junit.Rule; import org.junit.Test; 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 1a26df22cc34e..447b993224406 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 @@ -17,19 +17,16 @@ */ package org.apache.beam.sdk.transforms.reflect; -import org.apache.beam.sdk.transforms.DoFn; - import com.google.common.reflect.TypeToken; - +import java.lang.reflect.Method; +import java.util.List; +import org.apache.beam.sdk.transforms.DoFn; import org.junit.Rule; import org.junit.Test; import org.junit.rules.ExpectedException; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.lang.reflect.Method; -import java.util.List; - /** Tests for {@link DoFnSignatures}. */ @RunWith(JUnit4.class) public class DoFnSignaturesTest { diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/AfterAllTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/AfterAllTest.java index b7980132cb5fd..b5912296bd84f 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/AfterAllTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/AfterAllTest.java @@ -24,7 +24,6 @@ import org.apache.beam.sdk.transforms.windowing.Trigger.OnceTrigger; import org.apache.beam.sdk.util.TriggerTester; import org.apache.beam.sdk.util.TriggerTester.SimpleTriggerTester; - import org.joda.time.Duration; import org.joda.time.Instant; import org.junit.Test; diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/AfterEachTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/AfterEachTest.java index df557eb301325..c413c6ed25c4a 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/AfterEachTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/AfterEachTest.java @@ -24,7 +24,6 @@ import org.apache.beam.sdk.transforms.windowing.Trigger.OnceTrigger; import org.apache.beam.sdk.util.TriggerTester; import org.apache.beam.sdk.util.TriggerTester.SimpleTriggerTester; - import org.joda.time.Duration; import org.joda.time.Instant; import org.junit.Before; diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/AfterFirstTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/AfterFirstTest.java index a16669425dd11..415060b6c22b7 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/AfterFirstTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/AfterFirstTest.java @@ -25,7 +25,6 @@ import org.apache.beam.sdk.transforms.windowing.Trigger.OnceTrigger; import org.apache.beam.sdk.util.TriggerTester; import org.apache.beam.sdk.util.TriggerTester.SimpleTriggerTester; - import org.joda.time.Duration; import org.joda.time.Instant; import org.junit.Before; diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/AfterPaneTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/AfterPaneTest.java index 76ee49c0b4021..38d030ec6be1f 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/AfterPaneTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/AfterPaneTest.java @@ -23,7 +23,6 @@ import org.apache.beam.sdk.util.TriggerTester; import org.apache.beam.sdk.util.TriggerTester.SimpleTriggerTester; - import org.joda.time.Duration; import org.joda.time.Instant; import org.junit.Test; diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/AfterProcessingTimeTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/AfterProcessingTimeTest.java index ea9c2b088e228..13a7acf8ca1e6 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/AfterProcessingTimeTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/AfterProcessingTimeTest.java @@ -24,7 +24,6 @@ import org.apache.beam.sdk.transforms.windowing.Trigger.OnceTrigger; import org.apache.beam.sdk.util.TriggerTester; import org.apache.beam.sdk.util.TriggerTester.SimpleTriggerTester; - import org.joda.time.Duration; import org.joda.time.Instant; import org.junit.Test; diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/AfterSynchronizedProcessingTimeTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/AfterSynchronizedProcessingTimeTest.java index 4c089db19c64e..7e6e938f3c8b8 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/AfterSynchronizedProcessingTimeTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/AfterSynchronizedProcessingTimeTest.java @@ -23,7 +23,6 @@ import org.apache.beam.sdk.util.TriggerTester; import org.apache.beam.sdk.util.TriggerTester.SimpleTriggerTester; - import org.joda.time.Duration; import org.joda.time.Instant; import org.junit.Test; diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/AfterWatermarkTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/AfterWatermarkTest.java index 418f746ea452a..084027b3e5b10 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/AfterWatermarkTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/AfterWatermarkTest.java @@ -26,7 +26,6 @@ import org.apache.beam.sdk.transforms.windowing.Trigger.OnceTrigger; import org.apache.beam.sdk.util.TriggerTester; import org.apache.beam.sdk.util.TriggerTester.SimpleTriggerTester; - import org.joda.time.Duration; import org.joda.time.Instant; import org.junit.Before; diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/CalendarWindowsTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/CalendarWindowsTest.java index 4598a2777c113..54cdd06195ec7 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/CalendarWindowsTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/CalendarWindowsTest.java @@ -20,12 +20,15 @@ import static org.apache.beam.sdk.testing.WindowFnTestUtils.runWindowFn; import static org.apache.beam.sdk.testing.WindowFnTestUtils.set; import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasDisplayItem; - import static org.hamcrest.MatcherAssert.assertThat; import static org.junit.Assert.assertEquals; +import java.util.Arrays; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Set; import org.apache.beam.sdk.transforms.display.DisplayData; - import org.joda.time.DateTime; import org.joda.time.DateTimeConstants; import org.joda.time.DateTimeZone; @@ -34,12 +37,6 @@ import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.util.Arrays; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.Set; - /** * Tests for CalendarWindows WindowFn. */ diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/DefaultTriggerTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/DefaultTriggerTest.java index 6ed1c8190df0a..673e5554b0087 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/DefaultTriggerTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/DefaultTriggerTest.java @@ -23,7 +23,6 @@ import org.apache.beam.sdk.util.TriggerTester; import org.apache.beam.sdk.util.TriggerTester.SimpleTriggerTester; - import org.joda.time.Duration; import org.joda.time.Instant; import org.junit.Test; diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/FixedWindowsTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/FixedWindowsTest.java index fc1caac40c09a..afa5d60b8a91e 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/FixedWindowsTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/FixedWindowsTest.java @@ -20,7 +20,6 @@ import static org.apache.beam.sdk.testing.WindowFnTestUtils.runWindowFn; import static org.apache.beam.sdk.testing.WindowFnTestUtils.set; import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasDisplayItem; - import static org.hamcrest.CoreMatchers.containsString; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; @@ -28,20 +27,18 @@ import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; +import java.util.Arrays; +import java.util.HashMap; +import java.util.Map; +import java.util.Set; import org.apache.beam.sdk.testing.WindowFnTestUtils; import org.apache.beam.sdk.transforms.display.DisplayData; - import org.joda.time.Duration; import org.joda.time.Instant; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.util.Arrays; -import java.util.HashMap; -import java.util.Map; -import java.util.Set; - /** * Tests for FixedWindows WindowFn. */ diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/IntervalWindowTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/IntervalWindowTest.java index aaface2d658b5..fdfbdcb1bbcaa 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/IntervalWindowTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/IntervalWindowTest.java @@ -20,20 +20,17 @@ import static org.hamcrest.Matchers.equalTo; import static org.junit.Assert.assertThat; +import com.google.common.collect.Lists; +import java.util.List; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.InstantCoder; import org.apache.beam.sdk.testing.CoderProperties; import org.apache.beam.sdk.util.CoderUtils; - -import com.google.common.collect.Lists; - import org.joda.time.Instant; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.util.List; - /** * Tests for {@link Window}. */ diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/NeverTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/NeverTest.java index ed64f84ee46dc..fb2b4d5c730de 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/NeverTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/NeverTest.java @@ -23,7 +23,6 @@ import org.apache.beam.sdk.util.TriggerTester; import org.apache.beam.sdk.util.TriggerTester.SimpleTriggerTester; import org.apache.beam.sdk.values.TimestampedValue; - import org.joda.time.Duration; import org.joda.time.Instant; import org.junit.Before; diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/OrFinallyTriggerTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/OrFinallyTriggerTest.java index 93971793cfe21..7289d97d7b633 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/OrFinallyTriggerTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/OrFinallyTriggerTest.java @@ -24,7 +24,6 @@ import org.apache.beam.sdk.transforms.windowing.Trigger.OnceTrigger; import org.apache.beam.sdk.util.TriggerTester; import org.apache.beam.sdk.util.TriggerTester.SimpleTriggerTester; - import org.joda.time.Duration; import org.joda.time.Instant; import org.junit.Test; diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/PaneInfoTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/PaneInfoTest.java index ea4928ea12df3..1ce2d8c8d6a23 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/PaneInfoTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/PaneInfoTest.java @@ -23,7 +23,6 @@ import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.testing.CoderProperties; import org.apache.beam.sdk.transforms.windowing.PaneInfo.Timing; - import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/RepeatedlyTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/RepeatedlyTest.java index 3a33182f0f99b..6e8930dbd8830 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/RepeatedlyTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/RepeatedlyTest.java @@ -27,7 +27,6 @@ import org.apache.beam.sdk.util.TriggerTester; import org.apache.beam.sdk.util.TriggerTester.SimpleTriggerTester; - import org.joda.time.Duration; import org.joda.time.Instant; import org.junit.Test; diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/SessionsTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/SessionsTest.java index a543359e9c76f..b13168818cf89 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/SessionsTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/SessionsTest.java @@ -20,30 +20,26 @@ import static org.apache.beam.sdk.testing.WindowFnTestUtils.runWindowFn; import static org.apache.beam.sdk.testing.WindowFnTestUtils.set; import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasDisplayItem; - import static org.hamcrest.Matchers.allOf; import static org.hamcrest.Matchers.containsString; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertThat; import static org.junit.Assert.assertTrue; +import com.google.common.collect.ImmutableList; +import java.util.Arrays; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Set; import org.apache.beam.sdk.testing.WindowFnTestUtils; import org.apache.beam.sdk.transforms.display.DisplayData; - -import com.google.common.collect.ImmutableList; - import org.joda.time.Duration; import org.joda.time.Instant; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.util.Arrays; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.Set; - /** * Tests for Sessions WindowFn. */ diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/SlidingWindowsTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/SlidingWindowsTest.java index 047a413242b5d..54c01a828bc14 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/SlidingWindowsTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/SlidingWindowsTest.java @@ -20,26 +20,23 @@ import static org.apache.beam.sdk.testing.WindowFnTestUtils.runWindowFn; import static org.apache.beam.sdk.testing.WindowFnTestUtils.set; import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasDisplayItem; - import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertThat; import static org.junit.Assert.assertTrue; +import java.util.Arrays; +import java.util.HashMap; +import java.util.Map; +import java.util.Set; import org.apache.beam.sdk.testing.WindowFnTestUtils; import org.apache.beam.sdk.transforms.display.DisplayData; - import org.joda.time.Duration; import org.joda.time.Instant; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.util.Arrays; -import java.util.HashMap; -import java.util.Map; -import java.util.Set; - /** * Tests for the SlidingWindows WindowFn. */ diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/StubTrigger.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/StubTrigger.java index 06218cf2c25b0..b258a791fd400 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/StubTrigger.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/StubTrigger.java @@ -18,10 +18,8 @@ package org.apache.beam.sdk.transforms.windowing; import com.google.common.collect.Lists; - -import org.joda.time.Instant; - import java.util.List; +import org.joda.time.Instant; /** * No-op {@link OnceTrigger} implementation for testing. diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/TriggerTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/TriggerTest.java index 43c8bd8c5d761..cfc03b29becaf 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/TriggerTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/windowing/TriggerTest.java @@ -21,14 +21,13 @@ import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; +import java.util.Arrays; +import java.util.List; import org.joda.time.Instant; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.util.Arrays; -import java.util.List; - /** * Tests for {@link Trigger}. */ 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 c583860d2834f..9744fc6c78bac 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 @@ -20,7 +20,6 @@ import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasDisplayItem; import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasKey; import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.includesDisplayDataFrom; - import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.isOneOf; import static org.hamcrest.Matchers.not; @@ -29,6 +28,7 @@ import static org.junit.Assert.assertTrue; 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; @@ -44,7 +44,6 @@ import org.apache.beam.sdk.util.WindowingStrategy.AccumulationMode; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.TimestampedValue; - import org.hamcrest.Matchers; import org.joda.time.Duration; import org.joda.time.Instant; @@ -56,8 +55,6 @@ import org.junit.runners.JUnit4; import org.mockito.Mockito; -import java.io.Serializable; - /** * Tests for {@link Window}. */ 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 159e7004f80fa..ab208dd57cc37 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 @@ -17,6 +17,10 @@ */ package org.apache.beam.sdk.transforms.windowing; +import java.io.File; +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; @@ -34,7 +38,6 @@ import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionList; import org.apache.beam.sdk.values.TimestampedValue; - import org.joda.time.Duration; import org.joda.time.Instant; import org.junit.Rule; @@ -44,11 +47,6 @@ import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.io.File; -import java.io.FileOutputStream; -import java.io.PrintStream; -import java.io.Serializable; - /** Unit tests for bucketing. */ @RunWith(JUnit4.class) public class WindowingTest implements Serializable { diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/ApiSurfaceTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/ApiSurfaceTest.java index b3f874360777f..4b76277a027a5 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/ApiSurfaceTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/ApiSurfaceTest.java @@ -28,7 +28,10 @@ import com.google.common.collect.Lists; import com.google.common.collect.Maps; import com.google.common.collect.Sets; - +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.Set; import org.hamcrest.Description; import org.hamcrest.Matcher; import org.hamcrest.TypeSafeDiagnosingMatcher; @@ -36,11 +39,6 @@ import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.util.Collections; -import java.util.List; -import java.util.Map; -import java.util.Set; - /** * Tests for ApiSurface. These both test the functionality and also that our * public API is conformant to a hard-coded policy. diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/AttemptAndTimeBoundedExponentialBackOffTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/AttemptAndTimeBoundedExponentialBackOffTest.java index 08f5f56310569..59e0fb7dab284 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/AttemptAndTimeBoundedExponentialBackOffTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/AttemptAndTimeBoundedExponentialBackOffTest.java @@ -25,10 +25,8 @@ import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; -import org.apache.beam.sdk.testing.FastNanoClockAndSleeper; - import com.google.api.client.util.BackOff; - +import org.apache.beam.sdk.testing.FastNanoClockAndSleeper; import org.junit.Rule; import org.junit.Test; import org.junit.rules.ExpectedException; diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/AttemptBoundedExponentialBackOffTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/AttemptBoundedExponentialBackOffTest.java index b4a075ccb3714..3cfa961865fb3 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/AttemptBoundedExponentialBackOffTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/AttemptBoundedExponentialBackOffTest.java @@ -26,7 +26,6 @@ import static org.junit.Assert.assertTrue; import com.google.api.client.util.BackOff; - import org.junit.Rule; import org.junit.Test; import org.junit.rules.ExpectedException; diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/AvroUtilsTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/AvroUtilsTest.java index b72ab9ac13348..d8c345c822c09 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/AvroUtilsTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/AvroUtilsTest.java @@ -19,29 +19,27 @@ import static org.junit.Assert.assertEquals; -import org.apache.beam.sdk.coders.AvroCoder; -import org.apache.beam.sdk.coders.DefaultCoder; -import org.apache.beam.sdk.util.AvroUtils.AvroMetadata; - +import java.io.File; +import java.io.FileOutputStream; +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.Random; import org.apache.avro.Schema; import org.apache.avro.file.CodecFactory; import org.apache.avro.file.DataFileConstants; import org.apache.avro.file.DataFileWriter; import org.apache.avro.io.DatumWriter; import org.apache.avro.reflect.Nullable; +import org.apache.beam.sdk.coders.AvroCoder; +import org.apache.beam.sdk.coders.DefaultCoder; +import org.apache.beam.sdk.util.AvroUtils.AvroMetadata; import org.junit.Rule; import org.junit.Test; import org.junit.rules.TemporaryFolder; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.io.File; -import java.io.FileOutputStream; -import java.io.IOException; -import java.util.ArrayList; -import java.util.List; -import java.util.Random; - /** * Tests for AvroUtils. */ diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/BucketingFunctionTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/BucketingFunctionTest.java index ee5a2b3ff4b6b..196b904403881 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/BucketingFunctionTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/BucketingFunctionTest.java @@ -23,7 +23,6 @@ import static org.junit.Assert.assertTrue; import org.apache.beam.sdk.transforms.Combine; - import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/BufferedElementCountingOutputStreamTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/BufferedElementCountingOutputStreamTest.java index 09d8992e0db71..36f7028627eec 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/BufferedElementCountingOutputStreamTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/BufferedElementCountingOutputStreamTest.java @@ -21,18 +21,7 @@ import static org.junit.Assert.assertThat; import static org.junit.Assert.assertTrue; -import org.apache.beam.sdk.coders.ByteArrayCoder; -import org.apache.beam.sdk.coders.Coder.Context; - import com.google.common.collect.ImmutableList; - -import org.hamcrest.collection.IsIterableContainingInOrder; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.ExpectedException; -import org.junit.runner.RunWith; -import org.junit.runners.JUnit4; - import java.io.ByteArrayInputStream; import java.io.ByteArrayOutputStream; import java.io.IOException; @@ -42,6 +31,14 @@ import java.util.Collections; import java.util.List; import java.util.Random; +import org.apache.beam.sdk.coders.ByteArrayCoder; +import org.apache.beam.sdk.coders.Coder.Context; +import org.hamcrest.collection.IsIterableContainingInOrder; +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 BufferedElementCountingOutputStream}. diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/CoderUtilsTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/CoderUtilsTest.java index 0733c4c707ab0..4bd2f1916692f 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/CoderUtilsTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/CoderUtilsTest.java @@ -18,12 +18,13 @@ package org.apache.beam.sdk.util; import static org.apache.beam.sdk.util.CoderUtils.makeCloudEncoding; - import static org.mockito.Matchers.any; import static org.mockito.Matchers.anyString; import static org.mockito.Mockito.doThrow; import static org.mockito.Mockito.mock; +import java.io.InputStream; +import java.io.OutputStream; import org.apache.beam.sdk.coders.AtomicCoder; import org.apache.beam.sdk.coders.BigEndianIntegerCoder; import org.apache.beam.sdk.coders.Coder; @@ -34,7 +35,6 @@ import org.apache.beam.sdk.coders.StringUtf8Coder; import org.apache.beam.sdk.coders.VoidCoder; import org.apache.beam.sdk.testing.CoderPropertiesTest.ClosingCoder; - import org.hamcrest.CoreMatchers; import org.junit.Assert; import org.junit.Rule; @@ -43,9 +43,6 @@ import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.io.InputStream; -import java.io.OutputStream; - /** * Tests for CoderUtils. */ diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/CombineFnUtilTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/CombineFnUtilTest.java index fdb69e4deff71..fe81275ef6636 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/CombineFnUtilTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/CombineFnUtilTest.java @@ -22,14 +22,16 @@ import static org.mockito.Mockito.mock; import static org.mockito.Mockito.withSettings; +import com.google.common.collect.ImmutableList; +import java.io.ByteArrayOutputStream; +import java.io.NotSerializableException; +import java.io.ObjectOutputStream; +import java.util.List; import org.apache.beam.sdk.transforms.CombineWithContext.CombineFnWithContext; import org.apache.beam.sdk.transforms.CombineWithContext.Context; import org.apache.beam.sdk.transforms.CombineWithContext.KeyedCombineFnWithContext; import org.apache.beam.sdk.transforms.Sum; import org.apache.beam.sdk.util.state.StateContexts; - -import com.google.common.collect.ImmutableList; - import org.junit.Before; import org.junit.Rule; import org.junit.Test; @@ -37,11 +39,6 @@ import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.io.ByteArrayOutputStream; -import java.io.NotSerializableException; -import java.io.ObjectOutputStream; -import java.util.List; - /** * Unit tests for {@link CombineFnUtil}. */ diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/ExecutableTriggerTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/ExecutableTriggerTest.java index fdcd1bd8cea8e..1e3a1ff8b9ee0 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/ExecutableTriggerTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/ExecutableTriggerTest.java @@ -20,17 +20,15 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertSame; +import java.util.Arrays; +import java.util.List; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.Trigger; - import org.joda.time.Instant; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.util.Arrays; -import java.util.List; - /** * Tests for {@link ExecutableTrigger}. */ diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/ExposedByteArrayInputStreamTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/ExposedByteArrayInputStreamTest.java index d717cafd001f6..31cf1a81b4abf 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/ExposedByteArrayInputStreamTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/ExposedByteArrayInputStreamTest.java @@ -22,13 +22,12 @@ import static org.junit.Assert.assertNotEquals; import static org.junit.Assert.assertSame; +import java.io.ByteArrayInputStream; +import java.io.IOException; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.io.ByteArrayInputStream; -import java.io.IOException; - /** Unit tests for {@link ExposedByteArrayInputStream}. */ @RunWith(JUnit4.class) public class ExposedByteArrayInputStreamTest { diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/ExposedByteArrayOutputStreamTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/ExposedByteArrayOutputStreamTest.java index 9819a9b1a769c..a3a7a1d2d6cfe 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/ExposedByteArrayOutputStreamTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/ExposedByteArrayOutputStreamTest.java @@ -22,13 +22,12 @@ import static org.junit.Assert.assertNotSame; import static org.junit.Assert.assertSame; +import java.io.ByteArrayOutputStream; +import java.io.IOException; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.io.ByteArrayOutputStream; -import java.io.IOException; - /** Unit tests for {@link ExposedByteArrayOutputStream}. */ @RunWith(JUnit4.class) public class ExposedByteArrayOutputStreamTest { diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/FileIOChannelFactoryTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/FileIOChannelFactoryTest.java index 79e6e5cd6b0c2..011b4f5a977cd 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/FileIOChannelFactoryTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/FileIOChannelFactoryTest.java @@ -25,15 +25,6 @@ import com.google.common.collect.ImmutableList; import com.google.common.io.Files; import com.google.common.io.LineReader; - -import org.hamcrest.Matchers; -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 java.io.File; import java.io.FileNotFoundException; import java.io.Reader; @@ -42,6 +33,13 @@ import java.nio.charset.StandardCharsets; import java.nio.file.Path; import java.util.List; +import org.hamcrest.Matchers; +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; /** Tests for {@link FileIOChannelFactory}. */ @RunWith(JUnit4.class) diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/FinishedTriggersSetTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/FinishedTriggersSetTest.java index b3b18561d3e41..072d264f231c4 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/FinishedTriggersSetTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/FinishedTriggersSetTest.java @@ -21,12 +21,11 @@ import static org.hamcrest.Matchers.theInstance; import static org.junit.Assert.assertThat; +import java.util.HashSet; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.util.HashSet; - /** * Tests for {@link FinishedTriggersSet}. */ 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/util/GatherAllPanesTest.java index a6522efd36c9a..d195623d46343 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/util/GatherAllPanesTest.java @@ -19,6 +19,8 @@ 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; @@ -36,9 +38,6 @@ import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionList; import org.apache.beam.sdk.values.TypeDescriptor; - -import com.google.common.collect.Iterables; - import org.joda.time.Duration; import org.joda.time.Instant; import org.junit.Test; @@ -46,8 +45,6 @@ import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.io.Serializable; - /** * Tests for {@link GatherAllPanes}. */ diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/GcsIOChannelFactoryTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/GcsIOChannelFactoryTest.java index aae313715c756..6bdb782530d67 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/GcsIOChannelFactoryTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/GcsIOChannelFactoryTest.java @@ -21,7 +21,6 @@ import org.apache.beam.sdk.options.GcsOptions; import org.apache.beam.sdk.options.PipelineOptionsFactory; - import org.junit.Before; import org.junit.Test; import org.junit.runner.RunWith; 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 997340a22fa2f..681b0aaaa0597 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 @@ -29,11 +29,6 @@ import static org.junit.Assert.assertTrue; import static org.mockito.Mockito.when; -import org.apache.beam.sdk.options.GcsOptions; -import org.apache.beam.sdk.options.PipelineOptionsFactory; -import org.apache.beam.sdk.testing.FastNanoClockAndSleeper; -import org.apache.beam.sdk.util.gcsfs.GcsPath; - import com.google.api.client.googleapis.batch.BatchRequest; import com.google.api.client.googleapis.json.GoogleJsonError.ErrorInfo; import com.google.api.client.googleapis.json.GoogleJsonResponseException; @@ -58,14 +53,6 @@ import com.google.cloud.hadoop.gcsio.GoogleCloudStorageReadChannel; import com.google.cloud.hadoop.util.ClientRequestHelper; import com.google.common.collect.ImmutableList; - -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.Mockito; - import java.io.FileNotFoundException; import java.io.IOException; import java.math.BigInteger; @@ -79,6 +66,16 @@ import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.TimeUnit; +import org.apache.beam.sdk.options.GcsOptions; +import org.apache.beam.sdk.options.PipelineOptionsFactory; +import org.apache.beam.sdk.testing.FastNanoClockAndSleeper; +import org.apache.beam.sdk.util.gcsfs.GcsPath; +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.Mockito; /** Test case for {@link GcsUtil}. */ @RunWith(JUnit4.class) diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/IOChannelUtilsTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/IOChannelUtilsTest.java index 8a7eb0216954a..d92d3cd2e10af 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/IOChannelUtilsTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/IOChannelUtilsTest.java @@ -23,7 +23,8 @@ import static org.junit.Assert.fail; import com.google.common.io.Files; - +import java.io.File; +import java.nio.charset.StandardCharsets; import org.junit.Assert; import org.junit.Rule; import org.junit.Test; @@ -31,9 +32,6 @@ import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.io.File; -import java.nio.charset.StandardCharsets; - /** * Tests for IOChannelUtils. */ diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/IdentitySideInputWindowFn.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/IdentitySideInputWindowFn.java index 705003eb2e6b7..2b48b9fc4dd75 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/IdentitySideInputWindowFn.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/IdentitySideInputWindowFn.java @@ -17,15 +17,14 @@ */ package org.apache.beam.sdk.util; +import java.util.Collection; +import java.util.Collections; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.GlobalWindow; import org.apache.beam.sdk.transforms.windowing.NonMergingWindowFn; import org.apache.beam.sdk.transforms.windowing.WindowFn; -import java.util.Collection; -import java.util.Collections; - /** * A {@link WindowFn} for use during tests that returns the input window for calls to * {@link #getSideInputWindow(BoundedWindow)}. diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/InstanceBuilderTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/InstanceBuilderTest.java index 8d197baa1b7b9..505b2d325a4da 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/InstanceBuilderTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/InstanceBuilderTest.java @@ -18,7 +18,6 @@ package org.apache.beam.sdk.util; import org.apache.beam.sdk.values.TupleTag; - import org.hamcrest.Matchers; import org.junit.Assert; import org.junit.Rule; diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/KeyedWorkItemCoderTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/KeyedWorkItemCoderTest.java index 2151f88050578..1974d9e705bc4 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/KeyedWorkItemCoderTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/KeyedWorkItemCoderTest.java @@ -17,15 +17,13 @@ */ package org.apache.beam.sdk.util; +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.TimerInternals.TimerData; import org.apache.beam.sdk.util.state.StateNamespaces; - -import com.google.common.collect.ImmutableList; - import org.joda.time.Instant; import org.junit.Test; import org.junit.runner.RunWith; diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/MergingActiveWindowSetTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/MergingActiveWindowSetTest.java index 4750af1c8cad3..676a25ab42a0f 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/MergingActiveWindowSetTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/MergingActiveWindowSetTest.java @@ -22,16 +22,17 @@ import static org.mockito.Mockito.mock; import static org.mockito.Mockito.verify; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableSet; +import java.util.HashMap; +import java.util.List; +import java.util.Map; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.GlobalWindow; import org.apache.beam.sdk.transforms.windowing.IntervalWindow; import org.apache.beam.sdk.transforms.windowing.Sessions; import org.apache.beam.sdk.util.state.InMemoryStateInternals; import org.apache.beam.sdk.util.state.StateInternals; - -import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableSet; - import org.joda.time.Duration; import org.joda.time.Instant; import org.junit.After; @@ -40,10 +41,6 @@ import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.util.HashMap; -import java.util.List; -import java.util.Map; - /** * Test NonMergingActiveWindowSet. */ diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/MovingFunctionTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/MovingFunctionTest.java index b95f235c430fe..8d57bf4d24b6a 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/MovingFunctionTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/MovingFunctionTest.java @@ -23,7 +23,6 @@ import static org.junit.Assert.assertTrue; import org.apache.beam.sdk.transforms.Combine; - import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/MutationDetectorsTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/MutationDetectorsTest.java index 0763912b56678..ebd8297bbf803 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/MutationDetectorsTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/MutationDetectorsTest.java @@ -17,26 +17,23 @@ */ package org.apache.beam.sdk.util; +import com.google.common.collect.FluentIterable; +import com.google.common.collect.Lists; +import com.google.common.collect.Sets; +import java.util.Arrays; +import java.util.LinkedList; +import java.util.List; +import java.util.Set; import org.apache.beam.sdk.coders.ByteArrayCoder; import org.apache.beam.sdk.coders.IterableCoder; import org.apache.beam.sdk.coders.ListCoder; import org.apache.beam.sdk.coders.VarIntCoder; - -import com.google.common.collect.FluentIterable; -import com.google.common.collect.Lists; -import com.google.common.collect.Sets; - import org.junit.Rule; import org.junit.Test; import org.junit.rules.ExpectedException; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.util.Arrays; -import java.util.LinkedList; -import java.util.List; -import java.util.Set; - /** * Tests for {@link MutationDetectors}. */ diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/PTupleTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/PTupleTest.java index fe2014a7d3c44..abead0409c2ad 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/PTupleTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/PTupleTest.java @@ -23,7 +23,6 @@ import static org.junit.Assert.assertTrue; import org.apache.beam.sdk.values.TupleTag; - import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/PubsubClientTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/PubsubClientTest.java index 07031240d16ce..1a99d38c714ff 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/PubsubClientTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/PubsubClientTest.java @@ -20,12 +20,11 @@ import static org.junit.Assert.assertEquals; +import com.google.common.collect.ImmutableMap; +import java.util.Map; import org.apache.beam.sdk.util.PubsubClient.ProjectPath; import org.apache.beam.sdk.util.PubsubClient.SubscriptionPath; import org.apache.beam.sdk.util.PubsubClient.TopicPath; - -import com.google.common.collect.ImmutableMap; - import org.joda.time.Instant; import org.junit.Rule; import org.junit.Test; @@ -33,8 +32,6 @@ import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.util.Map; - /** * Tests for helper classes and methods in PubsubClient. */ diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/PubsubGrpcClientTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/PubsubGrpcClientTest.java index 4d0ec29ca2822..b36b93474f3be 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/PubsubGrpcClientTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/PubsubGrpcClientTest.java @@ -20,11 +20,6 @@ import static org.junit.Assert.assertEquals; -import org.apache.beam.sdk.util.PubsubClient.IncomingMessage; -import org.apache.beam.sdk.util.PubsubClient.OutgoingMessage; -import org.apache.beam.sdk.util.PubsubClient.SubscriptionPath; -import org.apache.beam.sdk.util.PubsubClient.TopicPath; - import com.google.auth.oauth2.GoogleCredentials; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; @@ -38,9 +33,13 @@ import com.google.pubsub.v1.PullResponse; import com.google.pubsub.v1.ReceivedMessage; import com.google.pubsub.v1.SubscriberGrpc; - import io.grpc.ManagedChannel; - +import java.io.IOException; +import java.util.List; +import org.apache.beam.sdk.util.PubsubClient.IncomingMessage; +import org.apache.beam.sdk.util.PubsubClient.OutgoingMessage; +import org.apache.beam.sdk.util.PubsubClient.SubscriptionPath; +import org.apache.beam.sdk.util.PubsubClient.TopicPath; import org.junit.After; import org.junit.Before; import org.junit.Test; @@ -48,9 +47,6 @@ import org.junit.runners.JUnit4; import org.mockito.Mockito; -import java.io.IOException; -import java.util.List; - /** * Tests for PubsubGrpcClient. */ diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/PubsubJsonClientTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/PubsubJsonClientTest.java index 21689358197d9..b6d7ccb2c0e38 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/PubsubJsonClientTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/PubsubJsonClientTest.java @@ -20,11 +20,6 @@ import static org.junit.Assert.assertEquals; -import org.apache.beam.sdk.util.PubsubClient.IncomingMessage; -import org.apache.beam.sdk.util.PubsubClient.OutgoingMessage; -import org.apache.beam.sdk.util.PubsubClient.SubscriptionPath; -import org.apache.beam.sdk.util.PubsubClient.TopicPath; - import com.google.api.services.pubsub.Pubsub; import com.google.api.services.pubsub.model.PublishRequest; import com.google.api.services.pubsub.model.PublishResponse; @@ -34,7 +29,12 @@ import com.google.api.services.pubsub.model.ReceivedMessage; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; - +import java.io.IOException; +import java.util.List; +import org.apache.beam.sdk.util.PubsubClient.IncomingMessage; +import org.apache.beam.sdk.util.PubsubClient.OutgoingMessage; +import org.apache.beam.sdk.util.PubsubClient.SubscriptionPath; +import org.apache.beam.sdk.util.PubsubClient.TopicPath; import org.junit.After; import org.junit.Before; import org.junit.Test; @@ -42,9 +42,6 @@ import org.junit.runners.JUnit4; import org.mockito.Mockito; -import java.io.IOException; -import java.util.List; - /** * Tests for PubsubJsonClient. */ diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/PubsubTestClientTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/PubsubTestClientTest.java index d86f5e7ca89d8..b9b1d3f8bc8ef 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/PubsubTestClientTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/PubsubTestClientTest.java @@ -20,25 +20,22 @@ import static org.junit.Assert.assertEquals; +import com.google.api.client.util.Clock; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.Lists; +import com.google.common.collect.Sets; +import java.io.IOException; +import java.util.List; +import java.util.concurrent.atomic.AtomicLong; import org.apache.beam.sdk.util.PubsubClient.IncomingMessage; import org.apache.beam.sdk.util.PubsubClient.OutgoingMessage; import org.apache.beam.sdk.util.PubsubClient.SubscriptionPath; import org.apache.beam.sdk.util.PubsubClient.TopicPath; import org.apache.beam.sdk.util.PubsubTestClient.PubsubTestClientFactory; - -import com.google.api.client.util.Clock; -import com.google.common.collect.ImmutableList; -import com.google.common.collect.Lists; -import com.google.common.collect.Sets; - import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.io.IOException; -import java.util.List; -import java.util.concurrent.atomic.AtomicLong; - /** * Tests for PubsubTestClient. */ 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 f4ae996143c5e..d990ee066e277 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 @@ -19,6 +19,8 @@ import static org.junit.Assert.assertEquals; +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; @@ -33,17 +35,12 @@ import org.apache.beam.sdk.transforms.windowing.Window; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; - -import com.google.common.collect.ImmutableList; - import org.joda.time.Duration; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.util.List; - /** * Tests for {@link Reshuffle}. */ diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/ReshuffleTriggerTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/ReshuffleTriggerTest.java index b17ce81cf4823..83077f4cc37fa 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/ReshuffleTriggerTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/ReshuffleTriggerTest.java @@ -25,7 +25,6 @@ import org.apache.beam.sdk.transforms.windowing.FixedWindows; import org.apache.beam.sdk.transforms.windowing.IntervalWindow; import org.apache.beam.sdk.transforms.windowing.Trigger; - import org.joda.time.Duration; import org.joda.time.Instant; import org.junit.Test; diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/RetryHttpRequestInitializerTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/RetryHttpRequestInitializerTest.java index 91d74db619c40..71554b573ac37 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/RetryHttpRequestInitializerTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/RetryHttpRequestInitializerTest.java @@ -46,7 +46,11 @@ import com.google.api.client.util.Sleeper; import com.google.api.services.storage.Storage; import com.google.api.services.storage.Storage.Objects.Get; - +import java.io.IOException; +import java.net.SocketTimeoutException; +import java.security.PrivateKey; +import java.util.Arrays; +import java.util.concurrent.atomic.AtomicLong; import org.hamcrest.Matchers; import org.junit.After; import org.junit.Assert; @@ -59,12 +63,6 @@ import org.mockito.invocation.InvocationOnMock; import org.mockito.stubbing.Answer; -import java.io.IOException; -import java.net.SocketTimeoutException; -import java.security.PrivateKey; -import java.util.Arrays; -import java.util.concurrent.atomic.AtomicLong; - /** * Tests for RetryHttpRequestInitializer. */ diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/SerializableUtilsTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/SerializableUtilsTest.java index 30406fcb10f71..5435a4588e841 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/SerializableUtilsTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/SerializableUtilsTest.java @@ -19,26 +19,23 @@ import static org.junit.Assert.assertEquals; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.collect.ImmutableList; +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.io.Serializable; +import java.util.List; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.CoderException; import org.apache.beam.sdk.coders.DeterministicStandardCoder; - -import com.google.common.collect.ImmutableList; - -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonProperty; import org.junit.Rule; import org.junit.Test; import org.junit.rules.ExpectedException; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.io.IOException; -import java.io.InputStream; -import java.io.OutputStream; -import java.io.Serializable; -import java.util.List; - /** Tests for {@link SerializableUtils}. */ @RunWith(JUnit4.class) public class SerializableUtilsTest { diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/StreamUtilsTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/StreamUtilsTest.java index 60eeb0daa88dc..7a31184dc5853 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/StreamUtilsTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/StreamUtilsTest.java @@ -21,16 +21,15 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertSame; -import org.junit.Before; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.JUnit4; - import java.io.BufferedInputStream; import java.io.ByteArrayInputStream; import java.io.IOException; import java.io.InputStream; import java.util.Arrays; +import org.junit.Before; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; /** Unit tests for {@link ExposedByteArrayInputStream}. */ @RunWith(JUnit4.class) 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 e87bbee58f68c..042e9e3d76ecd 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 @@ -24,7 +24,6 @@ import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.values.PBegin; import org.apache.beam.sdk.values.PDone; - import org.junit.Rule; import org.junit.Test; import org.junit.rules.ExpectedException; diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/StructsTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/StructsTest.java index 59dbb56ea0e9a..91090d1a2d6df 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/StructsTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/StructsTest.java @@ -34,18 +34,17 @@ import static org.apache.beam.sdk.util.Structs.getString; import static org.apache.beam.sdk.util.Structs.getStrings; -import org.hamcrest.Matchers; -import org.junit.Assert; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.JUnit4; - import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; +import org.hamcrest.Matchers; +import org.junit.Assert; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; /** * Tests for Structs. 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 bc2930c65c376..e8ffdb34411e3 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 @@ -30,7 +30,6 @@ import org.apache.beam.sdk.util.TimerInternals.TimerDataCoder; import org.apache.beam.sdk.util.state.StateNamespace; import org.apache.beam.sdk.util.state.StateNamespaces; - import org.joda.time.Instant; import org.junit.Test; import org.junit.runner.RunWith; diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/TriggerTester.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/TriggerTester.java index 4892bbd23c727..a1f1d21fc3dcb 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/TriggerTester.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/TriggerTester.java @@ -20,9 +20,21 @@ import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.base.Preconditions.checkNotNull; import static com.google.common.base.Preconditions.checkState; - import static org.junit.Assert.assertTrue; +import com.google.common.base.MoreObjects; +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +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.transforms.windowing.GlobalWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; @@ -41,26 +53,9 @@ import org.apache.beam.sdk.util.state.StateTag; import org.apache.beam.sdk.util.state.WatermarkHoldState; import org.apache.beam.sdk.values.TimestampedValue; - -import com.google.common.base.MoreObjects; -import com.google.common.collect.Lists; -import com.google.common.collect.Maps; - import org.joda.time.Duration; import org.joda.time.Instant; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collection; -import java.util.HashMap; -import java.util.HashSet; -import java.util.List; -import java.util.Map; -import java.util.PriorityQueue; -import java.util.Set; - -import javax.annotation.Nullable; - /** * Test utility that runs a {@link Trigger}, using in-memory stub implementation to provide * the {@link StateInternals}. diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/UnownedInputStreamTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/UnownedInputStreamTest.java index dac147c9036cc..e3a8d714c1768 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/UnownedInputStreamTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/UnownedInputStreamTest.java @@ -20,6 +20,7 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; +import java.io.ByteArrayInputStream; import org.junit.Before; import org.junit.Rule; import org.junit.Test; @@ -27,8 +28,6 @@ import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.io.ByteArrayInputStream; - /** Unit tests for {@link UnownedInputStream}. */ @RunWith(JUnit4.class) public class UnownedInputStreamTest { diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/UnownedOutputStreamTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/UnownedOutputStreamTest.java index 30761e7ef7037..e36b9fbbd9be7 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/UnownedOutputStreamTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/UnownedOutputStreamTest.java @@ -19,6 +19,7 @@ import static org.junit.Assert.assertEquals; +import java.io.ByteArrayOutputStream; import org.junit.Before; import org.junit.Rule; import org.junit.Test; @@ -26,8 +27,6 @@ import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.io.ByteArrayOutputStream; - /** Unit tests for {@link UnownedOutputStream}. */ @RunWith(JUnit4.class) public class UnownedOutputStreamTest { diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/UploadIdResponseInterceptorTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/UploadIdResponseInterceptorTest.java index 572897732524c..8b9f77e66fbf3 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/UploadIdResponseInterceptorTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/UploadIdResponseInterceptorTest.java @@ -17,22 +17,19 @@ */ package org.apache.beam.sdk.util; -import org.apache.beam.sdk.testing.ExpectedLogs; - import com.google.api.client.http.GenericUrl; import com.google.api.client.http.HttpResponse; import com.google.api.client.testing.http.HttpTesting; import com.google.api.client.testing.http.MockHttpTransport; import com.google.api.client.testing.http.MockLowLevelHttpResponse; - +import java.io.IOException; +import org.apache.beam.sdk.testing.ExpectedLogs; import org.junit.Rule; import org.junit.Test; import org.junit.rules.ExpectedException; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.io.IOException; - /** * A test for {@link UploadIdResponseInterceptor}. */ diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/UserCodeExceptionTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/UserCodeExceptionTest.java index 3be114605928a..a6d1cb57ef636 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/UserCodeExceptionTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/UserCodeExceptionTest.java @@ -24,6 +24,7 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertThat; +import java.io.IOException; import org.hamcrest.Description; import org.hamcrest.FeatureMatcher; import org.hamcrest.Matcher; @@ -34,8 +35,6 @@ import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.io.IOException; - /** * Tests for {@link UserCodeException} functionality. */ diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/VarIntTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/VarIntTest.java index 82a3689d78a76..02136affbe51f 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/VarIntTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/VarIntTest.java @@ -21,17 +21,16 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertThat; +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.EOFException; +import java.io.IOException; import org.junit.Rule; import org.junit.Test; import org.junit.rules.ExpectedException; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.io.ByteArrayInputStream; -import java.io.ByteArrayOutputStream; -import java.io.EOFException; -import java.io.IOException; - /** Unit tests for {@link VarInt}. */ @RunWith(JUnit4.class) public class VarIntTest { 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 90969b7841062..0c69a594df5ab 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 @@ -22,6 +22,9 @@ import static org.hamcrest.Matchers.equalTo; import static org.junit.Assert.assertThat; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.Iterables; +import java.util.Arrays; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.CoderException; import org.apache.beam.sdk.coders.StringUtf8Coder; @@ -29,18 +32,12 @@ import org.apache.beam.sdk.transforms.windowing.IntervalWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.transforms.windowing.PaneInfo.Timing; - -import com.google.common.collect.ImmutableList; -import com.google.common.collect.Iterables; - import org.joda.time.Instant; import org.junit.Assert; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.util.Arrays; - /** Test case for {@link WindowedValue}. */ @RunWith(JUnit4.class) public class WindowedValueTest { diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/ZipFilesTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/ZipFilesTest.java index 31e6b143d3ea7..1c038487bd998 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/ZipFilesTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/ZipFilesTest.java @@ -29,14 +29,6 @@ import com.google.common.io.ByteSource; import com.google.common.io.CharSource; import com.google.common.io.Files; - -import org.junit.Before; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; -import org.junit.runner.RunWith; -import org.junit.runners.JUnit4; - import java.io.File; import java.io.FileOutputStream; import java.io.IOException; @@ -46,6 +38,12 @@ import java.util.Enumeration; import java.util.zip.ZipEntry; import java.util.zip.ZipFile; +import org.junit.Before; +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 the {@link ZipFiles} class. These tests make sure that the handling diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/common/ReflectHelpersTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/common/ReflectHelpersTest.java index 9a5dc92023d07..e1073da484575 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/common/ReflectHelpersTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/common/ReflectHelpersTest.java @@ -19,15 +19,13 @@ import static org.junit.Assert.assertEquals; +import java.util.List; +import java.util.Map; import org.apache.beam.sdk.values.TypeDescriptor; - import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.util.List; -import java.util.Map; - /** * Tests for {@link ReflectHelpers}. */ diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/gcsfs/GcsPathTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/gcsfs/GcsPathTest.java index fdd1dfd6e7aa5..5c861845bd3ff 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/gcsfs/GcsPathTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/gcsfs/GcsPathTest.java @@ -23,18 +23,17 @@ import static org.junit.Assert.assertThat; import static org.junit.Assert.assertTrue; -import org.hamcrest.Matchers; -import org.junit.Assert; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.JUnit4; - import java.net.URI; import java.nio.file.Path; import java.nio.file.Paths; import java.util.Arrays; import java.util.Iterator; import java.util.List; +import org.hamcrest.Matchers; +import org.junit.Assert; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; /** * Tests of GcsPath. diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/state/CopyOnAccessInMemoryStateInternalsTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/state/CopyOnAccessInMemoryStateInternalsTest.java index b7388ee741d79..ad70bcafe7535 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/state/CopyOnAccessInMemoryStateInternalsTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/state/CopyOnAccessInMemoryStateInternalsTest.java @@ -39,7 +39,6 @@ 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.joda.time.Instant; import org.junit.Rule; import org.junit.Test; diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/state/InMemoryStateInternalsTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/state/InMemoryStateInternalsTest.java index 48d1a30c4cb87..08a6bc1d29d71 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/state/InMemoryStateInternalsTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/state/InMemoryStateInternalsTest.java @@ -21,21 +21,19 @@ import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertThat; +import java.util.Arrays; import org.apache.beam.sdk.coders.StringUtf8Coder; import org.apache.beam.sdk.coders.VarIntCoder; import org.apache.beam.sdk.transforms.Sum; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.IntervalWindow; import org.apache.beam.sdk.transforms.windowing.OutputTimeFns; - import org.hamcrest.Matchers; import org.joda.time.Instant; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.util.Arrays; - /** * Tests for {@link InMemoryStateInternals}. */ diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/state/StateNamespacesTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/state/StateNamespacesTest.java index 385ab6c66e8f5..f546e561b53a5 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/state/StateNamespacesTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/state/StateNamespacesTest.java @@ -24,7 +24,6 @@ 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.hamcrest.Matchers; import org.joda.time.Instant; import org.junit.Test; diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/state/StateTagTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/state/StateTagTest.java index ec7698d0b3811..2c8c9ccd00d86 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/state/StateTagTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/state/StateTagTest.java @@ -30,7 +30,6 @@ import org.apache.beam.sdk.transforms.Min.MinIntegerFn; import org.apache.beam.sdk.transforms.windowing.OutputTimeFns; import org.apache.beam.sdk.util.CombineFnUtil; - import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/values/KVTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/values/KVTest.java index f87e2ae94e358..202d6f34344d7 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/values/KVTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/values/KVTest.java @@ -23,13 +23,11 @@ import static org.junit.Assert.assertThat; import com.google.common.collect.ImmutableList; - +import java.util.Comparator; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.util.Comparator; - /** * Tests for KV. */ 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 af2c14c9bd677..f76bf7e23bf52 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 @@ -21,12 +21,11 @@ import static org.junit.Assert.assertThat; import static org.junit.Assert.fail; +import java.util.Collections; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.util.Collections; - /** * Tests for PCollectionLists. */ 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 13218b2979c71..1467ae8a52fbf 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 @@ -21,6 +21,9 @@ import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertTrue; +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; @@ -30,16 +33,11 @@ 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.Test; import org.junit.experimental.categories.Category; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.io.Serializable; -import java.util.Arrays; -import java.util.List; - /** Unit tests for {@link PCollectionTuple}. */ @RunWith(JUnit4.class) public final class PCollectionTupleTest implements Serializable { 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 5554b313b505f..4000e5db0276d 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 @@ -19,6 +19,7 @@ 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; @@ -26,7 +27,6 @@ import org.apache.beam.sdk.testing.TestPipeline; import org.apache.beam.sdk.transforms.Create; import org.apache.beam.sdk.transforms.PTransform; - import org.junit.Ignore; import org.junit.Rule; import org.junit.Test; @@ -35,8 +35,6 @@ import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.io.File; - /** * Tests for PDone. */ diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/values/TypeDescriptorTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/values/TypeDescriptorTest.java index 162c991f8f4eb..39472f94ec858 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/values/TypeDescriptorTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/values/TypeDescriptorTest.java @@ -20,18 +20,16 @@ import static org.junit.Assert.assertEquals; import com.google.common.reflect.TypeToken; - +import java.lang.reflect.Method; +import java.lang.reflect.TypeVariable; +import java.util.List; +import java.util.Set; import org.junit.Rule; import org.junit.Test; import org.junit.rules.ExpectedException; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.lang.reflect.Method; -import java.lang.reflect.TypeVariable; -import java.util.List; -import java.util.Set; - /** * Tests for TypeDescriptor. */ diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/values/TypeDescriptorsTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/values/TypeDescriptorsTest.java index 59acdf007ddc3..1bf0fc9cf20da 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/values/TypeDescriptorsTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/values/TypeDescriptorsTest.java @@ -23,17 +23,15 @@ import static org.apache.beam.sdk.values.TypeDescriptors.lists; import static org.apache.beam.sdk.values.TypeDescriptors.sets; import static org.apache.beam.sdk.values.TypeDescriptors.strings; - import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNotEquals; +import java.util.List; +import java.util.Set; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.util.List; -import java.util.Set; - /** * Tests for {@link TypeDescriptors}. */ 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 287223f44b15c..f33b3a2b692d3 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 @@ -28,7 +28,6 @@ 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; 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 6622fdc67e0ef..423ab9c5cdcbb 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 @@ -17,19 +17,17 @@ */ package org.apache.beam.sdk.extensions.joinlibrary; +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.Test; -import java.util.ArrayList; -import java.util.List; - /** * This test Inner Join functionality. */ 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 91b0740f30b3c..c32163fd55807 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 @@ -17,19 +17,17 @@ */ package org.apache.beam.sdk.extensions.joinlibrary; +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.Test; -import java.util.ArrayList; -import java.util.List; - /** * This test Outer Left Join functionality. 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 7977df77d8613..5a45f73e59e7a 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 @@ -17,19 +17,17 @@ */ package org.apache.beam.sdk.extensions.joinlibrary; +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.Test; -import java.util.ArrayList; -import java.util.List; - /** * This test Outer Right Join functionality. diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryAvroUtils.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryAvroUtils.java index 48e2258f83fd5..7826559e23f8c 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryAvroUtils.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryAvroUtils.java @@ -26,7 +26,8 @@ import com.google.api.services.bigquery.model.TableSchema; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; - +import java.util.List; +import javax.annotation.Nullable; import org.apache.avro.Schema; import org.apache.avro.Schema.Field; import org.apache.avro.Schema.Type; @@ -34,10 +35,6 @@ import org.joda.time.format.DateTimeFormat; import org.joda.time.format.DateTimeFormatter; -import java.util.List; - -import javax.annotation.Nullable; - /** * A set of utilities for working with Avro files. * 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 e61dcca993a00..04fb041badd13 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 @@ -21,6 +21,60 @@ import static com.google.common.base.Preconditions.checkNotNull; import static com.google.common.base.Preconditions.checkState; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.api.client.json.JsonFactory; +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.model.Job; +import com.google.api.services.bigquery.model.JobConfigurationExtract; +import com.google.api.services.bigquery.model.JobConfigurationLoad; +import com.google.api.services.bigquery.model.JobConfigurationQuery; +import com.google.api.services.bigquery.model.JobConfigurationTableCopy; +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.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.common.annotations.VisibleForTesting; +import com.google.common.base.Function; +import com.google.common.base.MoreObjects; +import com.google.common.base.Strings; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.Lists; +import com.google.common.io.CountingOutputStream; +import java.io.IOException; +import java.io.InputStream; +import java.io.ObjectInputStream; +import java.io.OutputStream; +import java.io.Serializable; +import java.nio.channels.Channels; +import java.nio.channels.WritableByteChannel; +import java.nio.charset.StandardCharsets; +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; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.NoSuchElementException; +import java.util.Set; +import java.util.UUID; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ThreadLocalRandom; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicReference; +import java.util.regex.Matcher; +import java.util.regex.Pattern; +import javax.annotation.Nullable; +import org.apache.avro.generic.GenericRecord; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.coders.AtomicCoder; import org.apache.beam.sdk.coders.Coder; @@ -75,67 +129,10 @@ import org.apache.beam.sdk.values.PInput; import org.apache.beam.sdk.values.TupleTag; import org.apache.beam.sdk.values.TupleTagList; - -import com.google.api.client.json.JsonFactory; -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.model.Job; -import com.google.api.services.bigquery.model.JobConfigurationExtract; -import com.google.api.services.bigquery.model.JobConfigurationLoad; -import com.google.api.services.bigquery.model.JobConfigurationQuery; -import com.google.api.services.bigquery.model.JobConfigurationTableCopy; -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.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.common.annotations.VisibleForTesting; -import com.google.common.base.Function; -import com.google.common.base.MoreObjects; -import com.google.common.base.Strings; -import com.google.common.collect.ImmutableList; -import com.google.common.collect.Lists; -import com.google.common.io.CountingOutputStream; - -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonProperty; -import org.apache.avro.generic.GenericRecord; import org.joda.time.Instant; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.io.IOException; -import java.io.InputStream; -import java.io.ObjectInputStream; -import java.io.OutputStream; -import java.io.Serializable; -import java.nio.channels.Channels; -import java.nio.channels.WritableByteChannel; -import java.nio.charset.StandardCharsets; -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; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.NoSuchElementException; -import java.util.Set; -import java.util.UUID; -import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.ThreadLocalRandom; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicReference; -import java.util.regex.Matcher; -import java.util.regex.Pattern; -import javax.annotation.Nullable; - /** * {@link PTransform}s for reading and writing * BigQuery tables. 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 0af6df8a33176..c0951fcdeae5e 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 @@ -17,8 +17,6 @@ */ package org.apache.beam.sdk.io.gcp.bigquery; -import org.apache.beam.sdk.options.BigQueryOptions; - import com.google.api.services.bigquery.model.Dataset; import com.google.api.services.bigquery.model.Job; import com.google.api.services.bigquery.model.JobConfigurationExtract; @@ -30,13 +28,12 @@ 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 java.io.IOException; import java.io.Serializable; import java.util.List; import java.util.NoSuchElementException; - import javax.annotation.Nullable; +import org.apache.beam.sdk.options.BigQueryOptions; /** An interface for real, mock, or fake implementations of Cloud BigQuery services. */ interface BigQueryServices extends Serializable { 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 bd1097f54c0c0..6aff3b097710a 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 @@ -19,13 +19,6 @@ import static com.google.common.base.Preconditions.checkNotNull; -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.util.AttemptBoundedExponentialBackOff; -import org.apache.beam.sdk.util.IntervalBoundedExponentialBackOff; -import org.apache.beam.sdk.util.Transport; - import com.google.api.client.googleapis.json.GoogleJsonResponseException; import com.google.api.client.googleapis.services.AbstractGoogleClientRequest; import com.google.api.client.util.BackOff; @@ -51,10 +44,6 @@ import com.google.api.services.bigquery.model.TableRow; import com.google.cloud.hadoop.util.ApiErrorExtractor; import com.google.common.annotations.VisibleForTesting; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - import java.io.IOException; import java.util.ArrayList; import java.util.LinkedList; @@ -65,8 +54,15 @@ 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.util.AttemptBoundedExponentialBackOff; +import org.apache.beam.sdk.util.IntervalBoundedExponentialBackOff; +import org.apache.beam.sdk.util.Transport; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * An implementation of {@link BigQueryServices} that actually communicates with the cloud BigQuery 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 index bf038f5e8c509..a64dc9fe9b9af 100644 --- 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 @@ -17,10 +17,6 @@ */ package org.apache.beam.sdk.io.gcp.bigquery; -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 com.google.api.client.util.BackOff; import com.google.api.client.util.BackOffUtils; import com.google.api.client.util.ExponentialBackOff; @@ -32,14 +28,14 @@ import com.google.api.services.bigquery.model.TableSchema; import com.google.cloud.hadoop.util.ApiErrorExtractor; import com.google.common.annotations.VisibleForTesting; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - 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. diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryTableRowIterator.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryTableRowIterator.java index 3afdffaca5952..729da97f7f7ec 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryTableRowIterator.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryTableRowIterator.java @@ -21,8 +21,6 @@ import static com.google.common.base.Preconditions.checkNotNull; import static com.google.common.base.Preconditions.checkState; -import org.apache.beam.sdk.util.AttemptBoundedExponentialBackOff; - import com.google.api.client.googleapis.services.AbstractGoogleClientRequest; import com.google.api.client.util.BackOff; import com.google.api.client.util.BackOffUtils; @@ -49,11 +47,6 @@ import com.google.common.base.MoreObjects; import com.google.common.collect.ImmutableList; import com.google.common.util.concurrent.Uninterruptibles; - -import org.joda.time.Duration; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - import java.io.IOException; import java.util.Collection; import java.util.Collections; @@ -64,8 +57,11 @@ import java.util.Objects; import java.util.Random; import java.util.concurrent.TimeUnit; - import javax.annotation.Nullable; +import org.apache.beam.sdk.util.AttemptBoundedExponentialBackOff; +import org.joda.time.Duration; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * Iterates over all rows in a table. 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 bfdf4aaf0a637..3a9ffce487f6f 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 @@ -21,6 +21,26 @@ import static com.google.common.base.Preconditions.checkNotNull; import static com.google.common.base.Preconditions.checkState; +import com.google.bigtable.v2.MutateRowResponse; +import com.google.bigtable.v2.Mutation; +import com.google.bigtable.v2.Row; +import com.google.bigtable.v2.RowFilter; +import com.google.bigtable.v2.SampleRowKeysResponse; +import com.google.cloud.bigtable.config.BigtableOptions; +import com.google.cloud.bigtable.config.RetryOptions; +import com.google.common.base.MoreObjects; +import com.google.common.collect.ImmutableList; +import com.google.common.util.concurrent.FutureCallback; +import com.google.common.util.concurrent.Futures; +import com.google.protobuf.ByteString; +import io.grpc.Status; +import java.io.IOException; +import java.util.Collections; +import java.util.Iterator; +import java.util.List; +import java.util.NoSuchElementException; +import java.util.concurrent.ConcurrentLinkedQueue; +import javax.annotation.Nullable; import org.apache.beam.sdk.annotations.Experimental; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.protobuf.ProtoCoder; @@ -40,32 +60,9 @@ import org.apache.beam.sdk.values.PBegin; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PDone; - -import com.google.bigtable.v2.MutateRowResponse; -import com.google.bigtable.v2.Mutation; -import com.google.bigtable.v2.Row; -import com.google.bigtable.v2.RowFilter; -import com.google.bigtable.v2.SampleRowKeysResponse; -import com.google.cloud.bigtable.config.BigtableOptions; -import com.google.cloud.bigtable.config.RetryOptions; -import com.google.common.base.MoreObjects; -import com.google.common.collect.ImmutableList; -import com.google.common.util.concurrent.FutureCallback; -import com.google.common.util.concurrent.Futures; -import com.google.protobuf.ByteString; - -import io.grpc.Status; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.io.IOException; -import java.util.Collections; -import java.util.Iterator; -import java.util.List; -import java.util.NoSuchElementException; -import java.util.concurrent.ConcurrentLinkedQueue; -import javax.annotation.Nullable; - /** * A bounded source and sink for Google Cloud Bigtable. * diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableService.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableService.java index 2a7e3a016a96b..ecd38a7468e4b 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableService.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableService.java @@ -17,20 +17,18 @@ */ package org.apache.beam.sdk.io.gcp.bigtable; -import org.apache.beam.sdk.io.gcp.bigtable.BigtableIO.BigtableSource; -import org.apache.beam.sdk.values.KV; - import com.google.bigtable.v2.MutateRowResponse; import com.google.bigtable.v2.Mutation; import com.google.bigtable.v2.Row; import com.google.bigtable.v2.SampleRowKeysResponse; import com.google.common.util.concurrent.ListenableFuture; import com.google.protobuf.ByteString; - import java.io.IOException; import java.io.Serializable; import java.util.List; import java.util.NoSuchElementException; +import org.apache.beam.sdk.io.gcp.bigtable.BigtableIO.BigtableSource; +import org.apache.beam.sdk.values.KV; /** * An interface for real or fake implementations of Cloud Bigtable. 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 b9288df11eeb1..07a183efc9d96 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 @@ -17,9 +17,6 @@ */ package org.apache.beam.sdk.io.gcp.bigtable; -import org.apache.beam.sdk.io.gcp.bigtable.BigtableIO.BigtableSource; -import org.apache.beam.sdk.values.KV; - import com.google.bigtable.admin.v2.GetTableRequest; import com.google.bigtable.v2.MutateRowRequest; import com.google.bigtable.v2.MutateRowResponse; @@ -40,16 +37,15 @@ import com.google.common.io.Closer; import com.google.common.util.concurrent.ListenableFuture; import com.google.protobuf.ByteString; - import io.grpc.Status.Code; import io.grpc.StatusRuntimeException; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - import java.io.IOException; import java.util.List; import java.util.NoSuchElementException; +import org.apache.beam.sdk.io.gcp.bigtable.BigtableIO.BigtableSource; +import org.apache.beam.sdk.values.KV; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * An implementation of {@link BigtableService} that actually communicates with the Cloud Bigtable 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 852595aa79ab1..c7433d37d1607 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 @@ -30,27 +30,6 @@ import static com.google.datastore.v1.client.DatastoreHelper.makeUpsert; import static com.google.datastore.v1.client.DatastoreHelper.makeValue; -import org.apache.beam.sdk.annotations.Experimental; -import org.apache.beam.sdk.options.GcpOptions; -import org.apache.beam.sdk.options.PipelineOptions; -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.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.Values; -import org.apache.beam.sdk.transforms.display.DisplayData; -import org.apache.beam.sdk.transforms.display.DisplayData.Builder; -import org.apache.beam.sdk.util.AttemptBoundedExponentialBackOff; -import org.apache.beam.sdk.util.RetryHttpRequestInitializer; -import org.apache.beam.sdk.values.KV; -import org.apache.beam.sdk.values.PBegin; -import org.apache.beam.sdk.values.PCollection; -import org.apache.beam.sdk.values.PDone; - import com.google.api.client.auth.oauth2.Credential; import com.google.api.client.util.BackOff; import com.google.api.client.util.BackOffUtils; @@ -76,16 +55,34 @@ import com.google.datastore.v1.client.DatastoreOptions; import com.google.datastore.v1.client.QuerySplitter; import com.google.protobuf.Int32Value; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - import java.io.IOException; import java.io.Serializable; import java.util.ArrayList; import java.util.List; import java.util.NoSuchElementException; import javax.annotation.Nullable; +import org.apache.beam.sdk.annotations.Experimental; +import org.apache.beam.sdk.options.GcpOptions; +import org.apache.beam.sdk.options.PipelineOptions; +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.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.Values; +import org.apache.beam.sdk.transforms.display.DisplayData; +import org.apache.beam.sdk.transforms.display.DisplayData.Builder; +import org.apache.beam.sdk.util.AttemptBoundedExponentialBackOff; +import org.apache.beam.sdk.util.RetryHttpRequestInitializer; +import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.PBegin; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.PDone; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** *

        {@link DatastoreV1} provides an API to Read, Write and Delete {@link PCollection PCollections} diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryAvroUtilsTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryAvroUtilsTest.java index 316392fe8f77d..59cf1f7c0d2b5 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryAvroUtilsTest.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryAvroUtilsTest.java @@ -19,25 +19,22 @@ import static org.junit.Assert.assertEquals; -import org.apache.beam.sdk.coders.AvroCoder; -import org.apache.beam.sdk.coders.DefaultCoder; - import com.google.api.services.bigquery.model.TableFieldSchema; import com.google.api.services.bigquery.model.TableRow; import com.google.api.services.bigquery.model.TableSchema; import com.google.common.collect.Lists; - +import java.util.ArrayList; +import java.util.List; import org.apache.avro.Schema; import org.apache.avro.generic.GenericData; import org.apache.avro.generic.GenericRecord; import org.apache.avro.reflect.Nullable; +import org.apache.beam.sdk.coders.AvroCoder; +import org.apache.beam.sdk.coders.DefaultCoder; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.util.ArrayList; -import java.util.List; - /** * Tests for {@link BigQueryAvroUtils}. */ 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 ca60696960e8d..a6d7e2f5e957c 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 @@ -17,12 +17,10 @@ */ package org.apache.beam.sdk.io.gcp.bigquery; +import static com.google.common.base.Preconditions.checkArgument; 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 com.google.common.base.Preconditions.checkArgument; - import static org.hamcrest.Matchers.hasItem; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNull; @@ -34,6 +32,39 @@ import static org.mockito.Mockito.doThrow; import static org.mockito.Mockito.when; +import com.google.api.client.util.Data; +import com.google.api.client.util.Strings; +import com.google.api.services.bigquery.model.ErrorProto; +import com.google.api.services.bigquery.model.Job; +import com.google.api.services.bigquery.model.JobConfigurationExtract; +import com.google.api.services.bigquery.model.JobConfigurationLoad; +import com.google.api.services.bigquery.model.JobConfigurationQuery; +import com.google.api.services.bigquery.model.JobConfigurationTableCopy; +import com.google.api.services.bigquery.model.JobReference; +import com.google.api.services.bigquery.model.JobStatistics; +import com.google.api.services.bigquery.model.JobStatistics2; +import com.google.api.services.bigquery.model.JobStatistics4; +import com.google.api.services.bigquery.model.JobStatus; +import com.google.api.services.bigquery.model.Table; +import com.google.api.services.bigquery.model.TableFieldSchema; +import com.google.api.services.bigquery.model.TableReference; +import com.google.api.services.bigquery.model.TableRow; +import com.google.api.services.bigquery.model.TableSchema; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Lists; +import java.io.File; +import java.io.FileFilter; +import java.io.IOException; +import java.io.Serializable; +import java.nio.file.Files; +import java.nio.file.Paths; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.NoSuchElementException; +import java.util.Set; +import javax.annotation.Nullable; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.coders.CoderException; import org.apache.beam.sdk.coders.KvCoder; @@ -85,29 +116,6 @@ import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TupleTag; - -import com.google.api.client.util.Data; -import com.google.api.client.util.Strings; -import com.google.api.services.bigquery.model.ErrorProto; -import com.google.api.services.bigquery.model.Job; -import com.google.api.services.bigquery.model.JobConfigurationExtract; -import com.google.api.services.bigquery.model.JobConfigurationLoad; -import com.google.api.services.bigquery.model.JobConfigurationQuery; -import com.google.api.services.bigquery.model.JobConfigurationTableCopy; -import com.google.api.services.bigquery.model.JobReference; -import com.google.api.services.bigquery.model.JobStatistics; -import com.google.api.services.bigquery.model.JobStatistics2; -import com.google.api.services.bigquery.model.JobStatistics4; -import com.google.api.services.bigquery.model.JobStatus; -import com.google.api.services.bigquery.model.Table; -import com.google.api.services.bigquery.model.TableFieldSchema; -import com.google.api.services.bigquery.model.TableReference; -import com.google.api.services.bigquery.model.TableRow; -import com.google.api.services.bigquery.model.TableSchema; -import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableMap; -import com.google.common.collect.Lists; - import org.hamcrest.CoreMatchers; import org.hamcrest.Matchers; import org.junit.Assert; @@ -124,19 +132,6 @@ import org.mockito.Mockito; import org.mockito.MockitoAnnotations; -import java.io.File; -import java.io.FileFilter; -import java.io.IOException; -import java.io.Serializable; -import java.nio.file.Files; -import java.nio.file.Paths; -import java.util.Collections; -import java.util.List; -import java.util.Map; -import java.util.NoSuchElementException; -import java.util.Set; -import javax.annotation.Nullable; - /** * Tests for BigQueryIO. */ 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 686685bd81451..eb5fbe6e05f5d 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 @@ -23,15 +23,6 @@ import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; -import org.apache.beam.sdk.io.gcp.bigquery.BigQueryServicesImpl.DatasetServiceImpl; -import org.apache.beam.sdk.io.gcp.bigquery.BigQueryServicesImpl.JobServiceImpl; -import org.apache.beam.sdk.options.PipelineOptionsFactory; -import org.apache.beam.sdk.testing.ExpectedLogs; -import org.apache.beam.sdk.testing.FastNanoClockAndSleeper; -import org.apache.beam.sdk.util.AttemptBoundedExponentialBackOff; -import org.apache.beam.sdk.util.RetryHttpRequestInitializer; -import org.apache.beam.sdk.util.Transport; - 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; @@ -55,7 +46,19 @@ import com.google.api.services.bigquery.model.TableRow; import com.google.cloud.hadoop.util.ApiErrorExtractor; import com.google.common.collect.ImmutableList; - +import java.io.ByteArrayInputStream; +import java.io.IOException; +import java.io.InputStream; +import java.util.ArrayList; +import java.util.List; +import org.apache.beam.sdk.io.gcp.bigquery.BigQueryServicesImpl.DatasetServiceImpl; +import org.apache.beam.sdk.io.gcp.bigquery.BigQueryServicesImpl.JobServiceImpl; +import org.apache.beam.sdk.options.PipelineOptionsFactory; +import org.apache.beam.sdk.testing.ExpectedLogs; +import org.apache.beam.sdk.testing.FastNanoClockAndSleeper; +import org.apache.beam.sdk.util.AttemptBoundedExponentialBackOff; +import org.apache.beam.sdk.util.RetryHttpRequestInitializer; +import org.apache.beam.sdk.util.Transport; import org.junit.Before; import org.junit.Rule; import org.junit.Test; @@ -65,12 +68,6 @@ import org.mockito.Mock; import org.mockito.MockitoAnnotations; -import java.io.ByteArrayInputStream; -import java.io.IOException; -import java.io.InputStream; -import java.util.ArrayList; -import java.util.List; - /** * Tests for {@link BigQueryServicesImpl}. */ 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 index dac3911d82a17..fb79c74215d20 100644 --- 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 @@ -18,7 +18,6 @@ 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; @@ -28,12 +27,6 @@ import static org.mockito.Mockito.verifyNoMoreInteractions; import static org.mockito.Mockito.when; -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 com.google.api.client.googleapis.json.GoogleJsonError; import com.google.api.client.googleapis.json.GoogleJsonError.ErrorInfo; import com.google.api.client.googleapis.json.GoogleJsonErrorContainer; @@ -51,7 +44,14 @@ 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; @@ -62,10 +62,6 @@ import org.mockito.Mock; import org.mockito.MockitoAnnotations; -import java.io.ByteArrayInputStream; -import java.io.IOException; -import java.io.InputStream; - /** * Tests of {@link BigQueryTableInserter}. */ diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryTableRowIteratorTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryTableRowIteratorTest.java index 457b071cac1ee..ab848f5b37796 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryTableRowIteratorTest.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryTableRowIteratorTest.java @@ -45,7 +45,10 @@ 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.Arrays; +import java.util.LinkedList; +import java.util.List; import org.junit.After; import org.junit.Before; import org.junit.Rule; @@ -56,11 +59,6 @@ import org.mockito.Mock; import org.mockito.MockitoAnnotations; -import java.io.IOException; -import java.util.Arrays; -import java.util.LinkedList; -import java.util.List; - /** * Tests for {@link BigQueryTableRowIterator}. */ 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 89284df48ee33..e539b33b0ee04 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 @@ -30,10 +30,6 @@ import static org.mockito.Mockito.verifyNoMoreInteractions; import static org.mockito.Mockito.when; -import org.apache.beam.sdk.io.gcp.bigquery.BigQueryServicesImpl.DatasetServiceImpl; -import org.apache.beam.sdk.options.PipelineOptions; -import org.apache.beam.sdk.options.PipelineOptionsFactory; - import com.google.api.services.bigquery.Bigquery; import com.google.api.services.bigquery.model.Table; import com.google.api.services.bigquery.model.TableCell; @@ -45,7 +41,14 @@ import com.google.api.services.bigquery.model.TableRow; import com.google.api.services.bigquery.model.TableSchema; import com.google.common.collect.ImmutableList; - +import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.LinkedList; +import java.util.List; +import org.apache.beam.sdk.io.gcp.bigquery.BigQueryServicesImpl.DatasetServiceImpl; +import org.apache.beam.sdk.options.PipelineOptions; +import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.hamcrest.Matchers; import org.junit.After; import org.junit.Assert; @@ -60,12 +63,6 @@ import org.mockito.invocation.InvocationOnMock; import org.mockito.stubbing.Answer; -import java.io.IOException; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.LinkedList; -import java.util.List; - /** * Tests for util classes related to BigQuery. */ 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 adf45a44f38c6..61b404ae441db 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 @@ -17,6 +17,8 @@ */ package org.apache.beam.sdk.io.gcp.bigtable; +import static com.google.common.base.Preconditions.checkArgument; +import static com.google.common.base.Verify.verifyNotNull; import static org.apache.beam.sdk.testing.SourceTestUtils.assertSourcesEqualReferenceSource; import static org.apache.beam.sdk.testing.SourceTestUtils.assertSplitAtFractionExhaustive; import static org.apache.beam.sdk.testing.SourceTestUtils.assertSplitAtFractionFails; @@ -25,31 +27,12 @@ import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasKey; import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasLabel; import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasValue; - -import static com.google.common.base.Preconditions.checkArgument; -import static com.google.common.base.Verify.verifyNotNull; import static org.hamcrest.Matchers.allOf; import static org.hamcrest.Matchers.hasSize; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertThat; -import org.apache.beam.sdk.Pipeline.PipelineExecutionException; -import org.apache.beam.sdk.coders.Coder; -import org.apache.beam.sdk.io.BoundedSource.BoundedReader; -import org.apache.beam.sdk.io.gcp.bigtable.BigtableIO.BigtableSource; -import org.apache.beam.sdk.io.range.ByteKey; -import org.apache.beam.sdk.io.range.ByteKeyRange; -import org.apache.beam.sdk.testing.ExpectedLogs; -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.transforms.display.DisplayData; -import org.apache.beam.sdk.transforms.display.DisplayDataEvaluator; -import org.apache.beam.sdk.values.KV; -import org.apache.beam.sdk.values.PCollection; -import org.apache.beam.sdk.values.TypeDescriptor; - import com.google.bigtable.v2.Cell; import com.google.bigtable.v2.Column; import com.google.bigtable.v2.Family; @@ -69,15 +52,6 @@ import com.google.common.util.concurrent.Futures; import com.google.common.util.concurrent.ListenableFuture; import com.google.protobuf.ByteString; - -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; - import java.io.IOException; import java.io.Serializable; import java.util.ArrayList; @@ -90,8 +64,29 @@ import java.util.Set; import java.util.SortedMap; import java.util.TreeMap; - import javax.annotation.Nullable; +import org.apache.beam.sdk.Pipeline.PipelineExecutionException; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.io.BoundedSource.BoundedReader; +import org.apache.beam.sdk.io.gcp.bigtable.BigtableIO.BigtableSource; +import org.apache.beam.sdk.io.range.ByteKey; +import org.apache.beam.sdk.io.range.ByteKeyRange; +import org.apache.beam.sdk.testing.ExpectedLogs; +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.transforms.display.DisplayData; +import org.apache.beam.sdk.transforms.display.DisplayDataEvaluator; +import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.TypeDescriptor; +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; /** * Unit tests for {@link BigtableIO}. diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableReadIT.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableReadIT.java index 02d403f9b9ac8..a064bd64235ba 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableReadIT.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableReadIT.java @@ -17,16 +17,14 @@ */ package org.apache.beam.sdk.io.gcp.bigtable; +import com.google.bigtable.v2.Row; +import com.google.cloud.bigtable.config.BigtableOptions; 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.TestPipeline; import org.apache.beam.sdk.transforms.Count; import org.apache.beam.sdk.values.PCollection; - -import com.google.bigtable.v2.Row; -import com.google.cloud.bigtable.config.BigtableOptions; - import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableWriteIT.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableWriteIT.java index ee3a6f97642bb..9afcd60f264a4 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableWriteIT.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableWriteIT.java @@ -19,14 +19,6 @@ import static org.junit.Assert.assertThat; -import org.apache.beam.sdk.Pipeline; -import org.apache.beam.sdk.io.CountingInput; -import org.apache.beam.sdk.options.PipelineOptionsFactory; -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.KV; - import com.google.bigtable.admin.v2.ColumnFamily; import com.google.bigtable.admin.v2.CreateTableRequest; import com.google.bigtable.admin.v2.DeleteTableRequest; @@ -44,20 +36,25 @@ import com.google.cloud.bigtable.grpc.scanner.ResultScanner; import com.google.common.collect.ImmutableList; import com.google.protobuf.ByteString; - -import org.hamcrest.Matchers; -import org.junit.After; -import org.junit.Before; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.JUnit4; - import java.io.IOException; import java.io.Serializable; import java.util.ArrayList; import java.util.Date; 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.options.PipelineOptionsFactory; +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.KV; +import org.hamcrest.Matchers; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; /** * End-to-end tests of BigtableWrite. diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/DatastoreV1Test.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/DatastoreV1Test.java index 31b5da43aa1ea..ab1df2f3e9f5d 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/DatastoreV1Test.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/DatastoreV1Test.java @@ -17,13 +17,6 @@ */ package org.apache.beam.sdk.io.gcp.datastore; -import static org.apache.beam.sdk.io.gcp.datastore.DatastoreV1.DATASTORE_BATCH_UPDATE_LIMIT; -import static org.apache.beam.sdk.io.gcp.datastore.DatastoreV1.Read.DEFAULT_BUNDLE_SIZE_BYTES; -import static org.apache.beam.sdk.io.gcp.datastore.DatastoreV1.Read.QUERY_BATCH_LIMIT; -import static org.apache.beam.sdk.io.gcp.datastore.DatastoreV1.Read.getEstimatedSizeBytes; -import static org.apache.beam.sdk.io.gcp.datastore.DatastoreV1.Read.makeRequest; -import static org.apache.beam.sdk.io.gcp.datastore.DatastoreV1.isValidKey; -import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasDisplayItem; import static com.google.datastore.v1.PropertyFilter.Operator.EQUAL; import static com.google.datastore.v1.PropertyOrder.Direction.DESCENDING; import static com.google.datastore.v1.client.DatastoreHelper.makeDelete; @@ -32,6 +25,13 @@ import static com.google.datastore.v1.client.DatastoreHelper.makeOrder; import static com.google.datastore.v1.client.DatastoreHelper.makeUpsert; import static com.google.datastore.v1.client.DatastoreHelper.makeValue; +import static org.apache.beam.sdk.io.gcp.datastore.DatastoreV1.DATASTORE_BATCH_UPDATE_LIMIT; +import static org.apache.beam.sdk.io.gcp.datastore.DatastoreV1.Read.DEFAULT_BUNDLE_SIZE_BYTES; +import static org.apache.beam.sdk.io.gcp.datastore.DatastoreV1.Read.QUERY_BATCH_LIMIT; +import static org.apache.beam.sdk.io.gcp.datastore.DatastoreV1.Read.getEstimatedSizeBytes; +import static org.apache.beam.sdk.io.gcp.datastore.DatastoreV1.Read.makeRequest; +import static org.apache.beam.sdk.io.gcp.datastore.DatastoreV1.isValidKey; +import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasDisplayItem; import static org.hamcrest.Matchers.greaterThanOrEqualTo; import static org.hamcrest.Matchers.hasItem; import static org.hamcrest.Matchers.lessThanOrEqualTo; @@ -47,6 +47,24 @@ import static org.mockito.Mockito.verifyZeroInteractions; import static org.mockito.Mockito.when; +import com.google.datastore.v1.CommitRequest; +import com.google.datastore.v1.Entity; +import com.google.datastore.v1.EntityResult; +import com.google.datastore.v1.Key; +import com.google.datastore.v1.Mutation; +import com.google.datastore.v1.PartitionId; +import com.google.datastore.v1.Query; +import com.google.datastore.v1.QueryResultBatch; +import com.google.datastore.v1.RunQueryRequest; +import com.google.datastore.v1.RunQueryResponse; +import com.google.datastore.v1.client.Datastore; +import com.google.datastore.v1.client.QuerySplitter; +import com.google.protobuf.Int32Value; +import java.util.ArrayList; +import java.util.HashSet; +import java.util.LinkedList; +import java.util.List; +import java.util.Set; import org.apache.beam.sdk.io.gcp.datastore.DatastoreV1.DatastoreWriterFn; import org.apache.beam.sdk.io.gcp.datastore.DatastoreV1.DeleteEntity; import org.apache.beam.sdk.io.gcp.datastore.DatastoreV1.DeleteEntityFn; @@ -69,21 +87,6 @@ import org.apache.beam.sdk.values.PBegin; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.POutput; - -import com.google.datastore.v1.CommitRequest; -import com.google.datastore.v1.Entity; -import com.google.datastore.v1.EntityResult; -import com.google.datastore.v1.Key; -import com.google.datastore.v1.Mutation; -import com.google.datastore.v1.PartitionId; -import com.google.datastore.v1.Query; -import com.google.datastore.v1.QueryResultBatch; -import com.google.datastore.v1.RunQueryRequest; -import com.google.datastore.v1.RunQueryResponse; -import com.google.datastore.v1.client.Datastore; -import com.google.datastore.v1.client.QuerySplitter; -import com.google.protobuf.Int32Value; - import org.junit.Before; import org.junit.Rule; import org.junit.Test; @@ -96,12 +99,6 @@ import org.mockito.invocation.InvocationOnMock; import org.mockito.stubbing.Answer; -import java.util.ArrayList; -import java.util.HashSet; -import java.util.LinkedList; -import java.util.List; -import java.util.Set; - /** * Tests for {@link DatastoreV1}. */ diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/V1ReadIT.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/V1ReadIT.java index 8fedc774f661c..99988331f775b 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/V1ReadIT.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/V1ReadIT.java @@ -23,6 +23,11 @@ import static org.apache.beam.sdk.io.gcp.datastore.V1TestUtil.makeAncestorKey; import static org.apache.beam.sdk.io.gcp.datastore.V1TestUtil.makeEntity; +import com.google.datastore.v1.Entity; +import com.google.datastore.v1.Key; +import com.google.datastore.v1.Query; +import com.google.datastore.v1.client.Datastore; +import java.util.UUID; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.io.gcp.datastore.V1TestUtil.UpsertMutationBuilder; import org.apache.beam.sdk.io.gcp.datastore.V1TestUtil.V1TestWriter; @@ -31,20 +36,12 @@ import org.apache.beam.sdk.testing.TestPipeline; import org.apache.beam.sdk.transforms.Count; import org.apache.beam.sdk.values.PCollection; - -import com.google.datastore.v1.Entity; -import com.google.datastore.v1.Key; -import com.google.datastore.v1.Query; -import com.google.datastore.v1.client.Datastore; - import org.junit.After; import org.junit.Before; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.util.UUID; - /** * End-to-end tests for Datastore DatastoreV1.Read. */ diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/V1TestOptions.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/V1TestOptions.java index 360855fee0f2c..9ebba3a4c5092 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/V1TestOptions.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/V1TestOptions.java @@ -18,12 +18,11 @@ package org.apache.beam.sdk.io.gcp.datastore; +import javax.annotation.Nullable; import org.apache.beam.sdk.options.Default; import org.apache.beam.sdk.options.Description; import org.apache.beam.sdk.testing.TestPipelineOptions; -import javax.annotation.Nullable; - /** * DatastoreV1 Datastore related pipeline options. */ diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/V1TestUtil.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/V1TestUtil.java index 1e323ec06a28c..a596bb3b76868 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/V1TestUtil.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/V1TestUtil.java @@ -25,12 +25,6 @@ import static com.google.datastore.v1.client.DatastoreHelper.makeUpsert; import static com.google.datastore.v1.client.DatastoreHelper.makeValue; -import org.apache.beam.sdk.options.GcpOptions; -import org.apache.beam.sdk.options.PipelineOptions; -import org.apache.beam.sdk.transforms.DoFn; -import org.apache.beam.sdk.util.AttemptBoundedExponentialBackOff; -import org.apache.beam.sdk.util.RetryHttpRequestInitializer; - import com.google.api.client.auth.oauth2.Credential; import com.google.api.client.util.BackOff; import com.google.api.client.util.BackOffUtils; @@ -51,16 +45,19 @@ import com.google.datastore.v1.client.DatastoreFactory; import com.google.datastore.v1.client.DatastoreOptions; import com.google.protobuf.Int32Value; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - import java.io.IOException; import java.util.ArrayList; import java.util.Iterator; import java.util.List; import java.util.UUID; import javax.annotation.Nullable; +import org.apache.beam.sdk.options.GcpOptions; +import org.apache.beam.sdk.options.PipelineOptions; +import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.util.AttemptBoundedExponentialBackOff; +import org.apache.beam.sdk.util.RetryHttpRequestInitializer; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; class V1TestUtil { private static final Logger LOG = LoggerFactory.getLogger(V1TestUtil.class); diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/V1WriteIT.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/V1WriteIT.java index b97c05c8c6c4c..fa7c1402a5b14 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/V1WriteIT.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/V1WriteIT.java @@ -22,21 +22,19 @@ import static org.apache.beam.sdk.io.gcp.datastore.V1TestUtil.deleteAllEntities; import static org.junit.Assert.assertEquals; +import java.util.UUID; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.io.CountingInput; import org.apache.beam.sdk.io.gcp.datastore.V1TestUtil.CreateEntityFn; import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.testing.TestPipeline; import org.apache.beam.sdk.transforms.ParDo; - import org.junit.After; import org.junit.Before; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.util.UUID; - /** * End-to-end tests for Datastore DatastoreV1.Write. */ diff --git a/sdks/java/io/hdfs/src/main/java/org/apache/beam/sdk/io/hdfs/AvroHDFSFileSource.java b/sdks/java/io/hdfs/src/main/java/org/apache/beam/sdk/io/hdfs/AvroHDFSFileSource.java index 9dc926bf66dac..26299952daf55 100644 --- a/sdks/java/io/hdfs/src/main/java/org/apache/beam/sdk/io/hdfs/AvroHDFSFileSource.java +++ b/sdks/java/io/hdfs/src/main/java/org/apache/beam/sdk/io/hdfs/AvroHDFSFileSource.java @@ -17,30 +17,27 @@ */ package org.apache.beam.sdk.io.hdfs; -import org.apache.beam.sdk.coders.AvroCoder; -import org.apache.beam.sdk.coders.Coder; -import org.apache.beam.sdk.coders.KvCoder; -import org.apache.beam.sdk.io.BoundedSource; -import org.apache.beam.sdk.options.PipelineOptions; -import org.apache.beam.sdk.util.CoderUtils; -import org.apache.beam.sdk.values.KV; - import com.google.common.base.Function; import com.google.common.collect.ImmutableList; import com.google.common.collect.Lists; - +import java.io.IOException; +import java.util.List; +import javax.annotation.Nullable; import org.apache.avro.Schema; import org.apache.avro.mapred.AvroKey; import org.apache.avro.mapreduce.AvroJob; import org.apache.avro.mapreduce.AvroKeyInputFormat; +import org.apache.beam.sdk.coders.AvroCoder; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.coders.KvCoder; +import org.apache.beam.sdk.io.BoundedSource; +import org.apache.beam.sdk.options.PipelineOptions; +import org.apache.beam.sdk.util.CoderUtils; +import org.apache.beam.sdk.values.KV; import org.apache.hadoop.io.NullWritable; import org.apache.hadoop.mapreduce.InputSplit; import org.apache.hadoop.mapreduce.lib.input.FileInputFormat; -import java.io.IOException; -import java.util.List; -import javax.annotation.Nullable; - /** * A {@code BoundedSource} for reading Avro files resident in a Hadoop filesystem. * diff --git a/sdks/java/io/hdfs/src/main/java/org/apache/beam/sdk/io/hdfs/AvroWrapperCoder.java b/sdks/java/io/hdfs/src/main/java/org/apache/beam/sdk/io/hdfs/AvroWrapperCoder.java index a831afec8110c..c1340c05169de 100644 --- a/sdks/java/io/hdfs/src/main/java/org/apache/beam/sdk/io/hdfs/AvroWrapperCoder.java +++ b/sdks/java/io/hdfs/src/main/java/org/apache/beam/sdk/io/hdfs/AvroWrapperCoder.java @@ -19,22 +19,20 @@ import static com.google.common.base.Preconditions.checkArgument; -import org.apache.beam.sdk.coders.AvroCoder; -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.util.CloudObject; -import org.apache.beam.sdk.util.PropertyNames; - import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; -import org.apache.avro.mapred.AvroWrapper; - import java.io.IOException; import java.io.InputStream; import java.io.OutputStream; import java.util.Collections; import java.util.List; +import org.apache.avro.mapred.AvroWrapper; +import org.apache.beam.sdk.coders.AvroCoder; +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.util.CloudObject; +import org.apache.beam.sdk.util.PropertyNames; /** * A {@code AvroWrapperCoder} is a {@link Coder} for a Java class that implements {@link diff --git a/sdks/java/io/hdfs/src/main/java/org/apache/beam/sdk/io/hdfs/HDFSFileSink.java b/sdks/java/io/hdfs/src/main/java/org/apache/beam/sdk/io/hdfs/HDFSFileSink.java index 688447aea6567..0b538b355bc18 100644 --- a/sdks/java/io/hdfs/src/main/java/org/apache/beam/sdk/io/hdfs/HDFSFileSink.java +++ b/sdks/java/io/hdfs/src/main/java/org/apache/beam/sdk/io/hdfs/HDFSFileSink.java @@ -19,16 +19,18 @@ import static com.google.common.base.Preconditions.checkState; +import com.google.api.client.util.Maps; +import com.google.common.collect.Lists; +import com.google.common.collect.Sets; +import java.io.IOException; +import java.util.Map; +import java.util.Random; +import java.util.Set; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.StringUtf8Coder; import org.apache.beam.sdk.io.Sink; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.values.KV; - -import com.google.api.client.util.Maps; -import com.google.common.collect.Lists; -import com.google.common.collect.Sets; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; @@ -47,11 +49,6 @@ import org.apache.hadoop.mapreduce.task.JobContextImpl; import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl; -import java.io.IOException; -import java.util.Map; -import java.util.Random; -import java.util.Set; - /** * A {@code Sink} for writing records to a Hadoop filesystem using a Hadoop file-based output * format. diff --git a/sdks/java/io/hdfs/src/main/java/org/apache/beam/sdk/io/hdfs/HDFSFileSource.java b/sdks/java/io/hdfs/src/main/java/org/apache/beam/sdk/io/hdfs/HDFSFileSource.java index de68565fabfd9..3a4d01fe77466 100644 --- a/sdks/java/io/hdfs/src/main/java/org/apache/beam/sdk/io/hdfs/HDFSFileSource.java +++ b/sdks/java/io/hdfs/src/main/java/org/apache/beam/sdk/io/hdfs/HDFSFileSource.java @@ -20,6 +20,19 @@ import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.base.Preconditions.checkNotNull; +import com.google.common.base.Function; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.Lists; +import java.io.Externalizable; +import java.io.IOException; +import java.io.ObjectInput; +import java.io.ObjectOutput; +import java.lang.reflect.InvocationTargetException; +import java.lang.reflect.Method; +import java.util.List; +import java.util.ListIterator; +import java.util.NoSuchElementException; +import javax.annotation.Nullable; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.KvCoder; import org.apache.beam.sdk.coders.VoidCoder; @@ -27,11 +40,6 @@ import org.apache.beam.sdk.io.Read; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.values.KV; - -import com.google.common.base.Function; -import com.google.common.collect.ImmutableList; -import com.google.common.collect.Lists; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.Path; @@ -46,18 +54,6 @@ import org.apache.hadoop.mapreduce.lib.input.FileInputFormat; import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl; -import java.io.Externalizable; -import java.io.IOException; -import java.io.ObjectInput; -import java.io.ObjectOutput; -import java.lang.reflect.InvocationTargetException; -import java.lang.reflect.Method; -import java.util.List; -import java.util.ListIterator; -import java.util.NoSuchElementException; - -import javax.annotation.Nullable; - /** * A {@code BoundedSource} for reading files resident in a Hadoop filesystem (HDFS) using a * Hadoop file-based input format. diff --git a/sdks/java/io/hdfs/src/main/java/org/apache/beam/sdk/io/hdfs/WritableCoder.java b/sdks/java/io/hdfs/src/main/java/org/apache/beam/sdk/io/hdfs/WritableCoder.java index 4e913ede2d645..f3569eafaf6e3 100644 --- a/sdks/java/io/hdfs/src/main/java/org/apache/beam/sdk/io/hdfs/WritableCoder.java +++ b/sdks/java/io/hdfs/src/main/java/org/apache/beam/sdk/io/hdfs/WritableCoder.java @@ -17,20 +17,20 @@ */ package org.apache.beam.sdk.io.hdfs; -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.util.CloudObject; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; -import org.apache.hadoop.io.NullWritable; -import org.apache.hadoop.io.Writable; import java.io.DataInputStream; import java.io.DataOutputStream; import java.io.IOException; import java.io.InputStream; import java.io.OutputStream; import java.util.List; +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.util.CloudObject; +import org.apache.hadoop.io.NullWritable; +import org.apache.hadoop.io.Writable; /** * A {@code WritableCoder} is a {@link Coder} for a Java class that implements {@link Writable}. diff --git a/sdks/java/io/hdfs/src/main/java/org/apache/beam/sdk/io/hdfs/simpleauth/SimpleAuthAvroHDFSFileSource.java b/sdks/java/io/hdfs/src/main/java/org/apache/beam/sdk/io/hdfs/simpleauth/SimpleAuthAvroHDFSFileSource.java index 5dd9673025056..d37ced9b84afe 100644 --- a/sdks/java/io/hdfs/src/main/java/org/apache/beam/sdk/io/hdfs/simpleauth/SimpleAuthAvroHDFSFileSource.java +++ b/sdks/java/io/hdfs/src/main/java/org/apache/beam/sdk/io/hdfs/simpleauth/SimpleAuthAvroHDFSFileSource.java @@ -17,18 +17,16 @@ */ package org.apache.beam.sdk.io.hdfs.simpleauth; -import org.apache.beam.sdk.coders.AvroCoder; -import org.apache.beam.sdk.io.hdfs.AvroHDFSFileSource; -import org.apache.beam.sdk.io.hdfs.HDFSFileSource; -import org.apache.beam.sdk.options.PipelineOptions; - import com.google.common.base.Function; import com.google.common.collect.ImmutableList; import com.google.common.collect.Lists; -import org.apache.hadoop.mapreduce.InputSplit; - import java.util.List; import javax.annotation.Nullable; +import org.apache.beam.sdk.coders.AvroCoder; +import org.apache.beam.sdk.io.hdfs.AvroHDFSFileSource; +import org.apache.beam.sdk.io.hdfs.HDFSFileSource; +import org.apache.beam.sdk.options.PipelineOptions; +import org.apache.hadoop.mapreduce.InputSplit; /** * Source for Avros on Hadoop/HDFS with Simple Authentication. diff --git a/sdks/java/io/hdfs/src/main/java/org/apache/beam/sdk/io/hdfs/simpleauth/SimpleAuthHDFSFileSink.java b/sdks/java/io/hdfs/src/main/java/org/apache/beam/sdk/io/hdfs/simpleauth/SimpleAuthHDFSFileSink.java index d0fd8b629085b..e2c2c907f1aa8 100644 --- a/sdks/java/io/hdfs/src/main/java/org/apache/beam/sdk/io/hdfs/simpleauth/SimpleAuthHDFSFileSink.java +++ b/sdks/java/io/hdfs/src/main/java/org/apache/beam/sdk/io/hdfs/simpleauth/SimpleAuthHDFSFileSink.java @@ -17,6 +17,7 @@ */ package org.apache.beam.sdk.io.hdfs.simpleauth; +import java.security.PrivilegedExceptionAction; import org.apache.beam.sdk.io.Sink; import org.apache.beam.sdk.io.hdfs.HDFSFileSink; import org.apache.beam.sdk.options.PipelineOptions; @@ -25,8 +26,6 @@ import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat; import org.apache.hadoop.security.UserGroupInformation; -import java.security.PrivilegedExceptionAction; - /** * A {@code Sink} for writing records to a Hadoop filesystem using a Hadoop file-based output * format with Simple Authentication. diff --git a/sdks/java/io/hdfs/src/main/java/org/apache/beam/sdk/io/hdfs/simpleauth/SimpleAuthHDFSFileSource.java b/sdks/java/io/hdfs/src/main/java/org/apache/beam/sdk/io/hdfs/simpleauth/SimpleAuthHDFSFileSource.java index 5b768fc2bc76b..6fb340ebd34ff 100644 --- a/sdks/java/io/hdfs/src/main/java/org/apache/beam/sdk/io/hdfs/simpleauth/SimpleAuthHDFSFileSource.java +++ b/sdks/java/io/hdfs/src/main/java/org/apache/beam/sdk/io/hdfs/simpleauth/SimpleAuthHDFSFileSource.java @@ -17,21 +17,19 @@ */ package org.apache.beam.sdk.io.hdfs.simpleauth; +import com.google.common.base.Function; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.Lists; +import java.util.List; +import javax.annotation.Nullable; import org.apache.beam.sdk.io.BoundedSource; import org.apache.beam.sdk.io.Read; import org.apache.beam.sdk.io.hdfs.HDFSFileSource; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.values.KV; - -import com.google.common.base.Function; -import com.google.common.collect.ImmutableList; -import com.google.common.collect.Lists; import org.apache.hadoop.mapreduce.InputSplit; import org.apache.hadoop.mapreduce.lib.input.FileInputFormat; -import java.util.List; -import javax.annotation.Nullable; - /** * Source for Hadoop/HDFS with Simple Authentication. * diff --git a/sdks/java/io/hdfs/src/test/java/org/apache/beam/sdk/io/hdfs/AvroWrapperCoderTest.java b/sdks/java/io/hdfs/src/test/java/org/apache/beam/sdk/io/hdfs/AvroWrapperCoderTest.java index 85cbd46178f73..6ebea3a2baf3c 100644 --- a/sdks/java/io/hdfs/src/test/java/org/apache/beam/sdk/io/hdfs/AvroWrapperCoderTest.java +++ b/sdks/java/io/hdfs/src/test/java/org/apache/beam/sdk/io/hdfs/AvroWrapperCoderTest.java @@ -17,11 +17,10 @@ */ package org.apache.beam.sdk.io.hdfs; -import org.apache.beam.sdk.coders.AvroCoder; -import org.apache.beam.sdk.testing.CoderProperties; - import org.apache.avro.mapred.AvroKey; import org.apache.avro.mapred.AvroValue; +import org.apache.beam.sdk.coders.AvroCoder; +import org.apache.beam.sdk.testing.CoderProperties; import org.junit.Test; /** diff --git a/sdks/java/io/hdfs/src/test/java/org/apache/beam/sdk/io/hdfs/HDFSFileSourceTest.java b/sdks/java/io/hdfs/src/test/java/org/apache/beam/sdk/io/hdfs/HDFSFileSourceTest.java index 67df7bcb4bb41..6145952af4ad1 100644 --- a/sdks/java/io/hdfs/src/test/java/org/apache/beam/sdk/io/hdfs/HDFSFileSourceTest.java +++ b/sdks/java/io/hdfs/src/test/java/org/apache/beam/sdk/io/hdfs/HDFSFileSourceTest.java @@ -24,13 +24,17 @@ import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; +import java.io.File; +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.Random; import org.apache.beam.sdk.io.BoundedSource; import org.apache.beam.sdk.io.Source; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.testing.SourceTestUtils; import org.apache.beam.sdk.values.KV; - import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; import org.apache.hadoop.io.IntWritable; @@ -42,12 +46,6 @@ import org.junit.Test; import org.junit.rules.TemporaryFolder; -import java.io.File; -import java.io.IOException; -import java.util.ArrayList; -import java.util.List; -import java.util.Random; - /** * Tests for HDFSFileSource. */ diff --git a/sdks/java/io/hdfs/src/test/java/org/apache/beam/sdk/io/hdfs/WritableCoderTest.java b/sdks/java/io/hdfs/src/test/java/org/apache/beam/sdk/io/hdfs/WritableCoderTest.java index ac32c336f998d..e78f850c66357 100644 --- a/sdks/java/io/hdfs/src/test/java/org/apache/beam/sdk/io/hdfs/WritableCoderTest.java +++ b/sdks/java/io/hdfs/src/test/java/org/apache/beam/sdk/io/hdfs/WritableCoderTest.java @@ -18,7 +18,6 @@ package org.apache.beam.sdk.io.hdfs; import org.apache.beam.sdk.testing.CoderProperties; - import org.apache.hadoop.io.IntWritable; import org.apache.hadoop.io.NullWritable; import org.junit.Test; diff --git a/sdks/java/io/jms/src/main/java/org/apache/beam/sdk/io/jms/JmsCheckpointMark.java b/sdks/java/io/jms/src/main/java/org/apache/beam/sdk/io/jms/JmsCheckpointMark.java index 81c2b826ce8d4..ba8fba9d00aee 100644 --- a/sdks/java/io/jms/src/main/java/org/apache/beam/sdk/io/jms/JmsCheckpointMark.java +++ b/sdks/java/io/jms/src/main/java/org/apache/beam/sdk/io/jms/JmsCheckpointMark.java @@ -17,18 +17,15 @@ */ package org.apache.beam.sdk.io.jms; +import java.util.ArrayList; +import java.util.List; +import javax.jms.Message; import org.apache.beam.sdk.coders.AvroCoder; import org.apache.beam.sdk.coders.DefaultCoder; import org.apache.beam.sdk.io.UnboundedSource; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; - import org.joda.time.Instant; -import java.util.ArrayList; -import java.util.List; - -import javax.jms.Message; - /** * Checkpoint for an unbounded JmsIO.Read. Consists of * JMS destination name, and the latest message ID consumed so far. 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 557fe13a3a372..f92dbd456b79f 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,22 @@ import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.base.Preconditions.checkNotNull; +import com.google.common.annotations.VisibleForTesting; +import java.io.IOException; +import java.util.ArrayList; +import java.util.Enumeration; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.NoSuchElementException; +import javax.annotation.Nullable; +import javax.jms.Connection; +import javax.jms.ConnectionFactory; +import javax.jms.Destination; +import javax.jms.MessageConsumer; +import javax.jms.MessageProducer; +import javax.jms.Session; +import javax.jms.TextMessage; import org.apache.beam.sdk.coders.AvroCoder; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.SerializableCoder; @@ -36,31 +52,11 @@ import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PDone; import org.apache.beam.sdk.values.PInput; - -import com.google.common.annotations.VisibleForTesting; - import org.joda.time.Duration; import org.joda.time.Instant; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.io.IOException; -import java.util.ArrayList; -import java.util.Enumeration; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.NoSuchElementException; - -import javax.annotation.Nullable; -import javax.jms.Connection; -import javax.jms.ConnectionFactory; -import javax.jms.Destination; -import javax.jms.MessageConsumer; -import javax.jms.MessageProducer; -import javax.jms.Session; -import javax.jms.TextMessage; - /** * An unbounded source for JMS destinations (queues or topics). * 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 aa0c472e631a6..65a9189d2cea8 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,7 +20,6 @@ import java.io.Serializable; import java.util.Map; import java.util.Objects; - import javax.jms.Destination; /** 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 020794c7cfbd6..4c3be6d0bae2c 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 @@ -17,6 +17,17 @@ */ package org.apache.beam.sdk.io.jms; +import java.util.ArrayList; +import javax.jms.Connection; +import javax.jms.ConnectionFactory; +import javax.jms.Message; +import javax.jms.MessageConsumer; +import javax.jms.MessageProducer; +import javax.jms.Session; +import javax.jms.TextMessage; +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; @@ -24,10 +35,6 @@ import org.apache.beam.sdk.transforms.Count; import org.apache.beam.sdk.transforms.Create; import org.apache.beam.sdk.values.PCollection; - -import org.apache.activemq.ActiveMQConnectionFactory; -import org.apache.activemq.broker.BrokerService; -import org.apache.activemq.store.memory.MemoryPersistenceAdapter; import org.junit.After; import org.junit.Assert; import org.junit.Before; @@ -36,16 +43,6 @@ import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.util.ArrayList; - -import javax.jms.Connection; -import javax.jms.ConnectionFactory; -import javax.jms.Message; -import javax.jms.MessageConsumer; -import javax.jms.MessageProducer; -import javax.jms.Session; -import javax.jms.TextMessage; - /** * Tests of {@link JmsIO}. */ diff --git a/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaCheckpointMark.java b/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaCheckpointMark.java index 4b6b976fa54dc..664bb6f766ee5 100644 --- a/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaCheckpointMark.java +++ b/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaCheckpointMark.java @@ -17,16 +17,14 @@ */ package org.apache.beam.sdk.io.kafka; +import java.io.IOException; +import java.io.Serializable; +import java.util.List; import org.apache.beam.sdk.coders.DefaultCoder; import org.apache.beam.sdk.coders.SerializableCoder; import org.apache.beam.sdk.io.UnboundedSource; - import org.apache.kafka.common.TopicPartition; -import java.io.IOException; -import java.io.Serializable; -import java.util.List; - /** * Checkpoint for an unbounded KafkaIO.Read. Consists of Kafka topic name, partition id, * and the latest offset consumed so far. 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 6cc5d6a597d62..885d5d10b4b7b 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 @@ -21,6 +21,34 @@ 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.base.Function; +import com.google.common.base.Joiner; +import com.google.common.base.Optional; +import com.google.common.collect.ComparisonChain; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Iterators; +import com.google.common.collect.Lists; +import com.google.common.io.Closeables; +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.Comparator; +import java.util.HashMap; +import java.util.Iterator; +import java.util.LinkedList; +import java.util.List; +import java.util.Map; +import java.util.NoSuchElementException; +import java.util.Random; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.SynchronousQueue; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; +import javax.annotation.Nullable; import org.apache.beam.sdk.coders.ByteArrayCoder; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.CoderException; @@ -46,18 +74,6 @@ import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PDone; import org.apache.beam.sdk.values.PInput; - -import com.google.common.annotations.VisibleForTesting; -import com.google.common.base.Function; -import com.google.common.base.Joiner; -import com.google.common.base.Optional; -import com.google.common.collect.ComparisonChain; -import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableMap; -import com.google.common.collect.Iterators; -import com.google.common.collect.Lists; -import com.google.common.io.Closeables; - import org.apache.kafka.clients.consumer.Consumer; import org.apache.kafka.clients.consumer.ConsumerConfig; import org.apache.kafka.clients.consumer.ConsumerRecord; @@ -79,25 +95,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.io.IOException; -import java.util.ArrayList; -import java.util.Collections; -import java.util.Comparator; -import java.util.HashMap; -import java.util.Iterator; -import java.util.LinkedList; -import java.util.List; -import java.util.Map; -import java.util.NoSuchElementException; -import java.util.Random; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.Executors; -import java.util.concurrent.ScheduledExecutorService; -import java.util.concurrent.SynchronousQueue; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicBoolean; -import javax.annotation.Nullable; - /** * An unbounded source and a sink for Kafka topics. * Kafka version 0.9 and above are supported. diff --git a/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaRecord.java b/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaRecord.java index 76e688b178521..fa202e103e921 100644 --- a/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaRecord.java +++ b/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaRecord.java @@ -17,10 +17,9 @@ */ package org.apache.beam.sdk.io.kafka; -import org.apache.beam.sdk.values.KV; - import java.io.Serializable; import java.util.Arrays; +import org.apache.beam.sdk.values.KV; /** * KafkaRecord contains key and value of the record as well as metadata for the record (topic name, diff --git a/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaRecordCoder.java b/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaRecordCoder.java index 8a3e7f51441d9..736a752f7741f 100644 --- a/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaRecordCoder.java +++ b/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaRecordCoder.java @@ -17,6 +17,12 @@ */ package org.apache.beam.sdk.io.kafka; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.util.List; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.CoderException; import org.apache.beam.sdk.coders.KvCoder; @@ -27,14 +33,6 @@ import org.apache.beam.sdk.util.PropertyNames; import org.apache.beam.sdk.values.KV; -import com.fasterxml.jackson.annotation.JsonCreator; -import com.fasterxml.jackson.annotation.JsonProperty; - -import java.io.IOException; -import java.io.InputStream; -import java.io.OutputStream; -import java.util.List; - /** * {@link Coder} for {@link KafkaRecord}. */ 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 9a89c3621bcd3..772efe1ec2941 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 @@ -21,6 +21,21 @@ import static org.junit.Assert.assertNull; import static org.junit.Assert.assertTrue; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.Collections; +import java.util.Comparator; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; +import javax.annotation.Nullable; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.Pipeline.PipelineExecutionException; import org.apache.beam.sdk.coders.BigEndianIntegerCoder; @@ -45,10 +60,6 @@ import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionList; - -import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableMap; - import org.apache.kafka.clients.consumer.Consumer; import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.clients.consumer.MockConsumer; @@ -66,20 +77,6 @@ import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.io.IOException; -import java.nio.ByteBuffer; -import java.util.ArrayList; -import java.util.Collections; -import java.util.Comparator; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.Executors; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicBoolean; -import javax.annotation.Nullable; - /** * Tests of {@link KafkaSource}. */ 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 132247b2474bf..98d99cefd93dc 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 @@ -21,6 +21,8 @@ import static org.hamcrest.Matchers.empty; import static org.hamcrest.Matchers.not; +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; @@ -28,9 +30,6 @@ import org.apache.beam.sdk.util.SerializableUtils; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; - -import com.google.common.collect.Iterables; - import org.junit.Assume; import org.junit.Rule; import org.junit.Test; @@ -38,8 +37,6 @@ import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.io.Serializable; - /** * Java 8 Tests for {@link Combine}. */ 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 3c83be28ca668..afd1c8b48ca5a 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 @@ -17,13 +17,13 @@ */ 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; import org.apache.beam.sdk.testing.TestPipeline; import org.apache.beam.sdk.values.PCollection; - import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -31,8 +31,6 @@ import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.io.Serializable; - /** * Java 8 Tests for {@link Filter}. */ 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 5ee10d1083373..70cc04da9aff3 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 @@ -17,23 +17,20 @@ */ package org.apache.beam.sdk.transforms; +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; import org.apache.beam.sdk.values.TypeDescriptor; - -import com.google.common.collect.ImmutableList; - import org.junit.Rule; import org.junit.Test; import org.junit.rules.ExpectedException; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.io.Serializable; -import java.util.List; - /** * Java 8 Tests for {@link 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 339e431be27aa..9b556b9556863 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 @@ -17,18 +17,16 @@ */ 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.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.io.Serializable; - /** * Java 8 tests for {@link 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 c8283dbf04114..0aeb41ffdc926 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 @@ -19,20 +19,18 @@ 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; import org.apache.beam.sdk.values.PCollectionList; - import org.junit.Rule; import org.junit.Test; import org.junit.rules.ExpectedException; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.io.Serializable; - /** * Java 8 Tests for {@link Filter}. */ diff --git a/sdks/java/java8tests/src/test/java/org/apache/beam/sdk/transforms/RemoveDuplicatesJava8Test.java b/sdks/java/java8tests/src/test/java/org/apache/beam/sdk/transforms/RemoveDuplicatesJava8Test.java index aa4c01548c823..99266d487cdde 100644 --- a/sdks/java/java8tests/src/test/java/org/apache/beam/sdk/transforms/RemoveDuplicatesJava8Test.java +++ b/sdks/java/java8tests/src/test/java/org/apache/beam/sdk/transforms/RemoveDuplicatesJava8Test.java @@ -23,23 +23,20 @@ import static org.hamcrest.Matchers.not; import static org.junit.Assert.assertThat; +import com.google.common.collect.HashMultimap; +import com.google.common.collect.Multimap; +import java.util.HashSet; +import java.util.Set; 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 com.google.common.collect.HashMultimap; -import com.google.common.collect.Multimap; - import org.junit.Rule; import org.junit.Test; import org.junit.rules.ExpectedException; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.util.HashSet; -import java.util.Set; - /** * Java 8 tests for {@link RemoveDuplicates}. */ 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 a89e281e68307..a5b9cb1239a0f 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 @@ -25,7 +25,6 @@ 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; 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 03aa64719fc92..5f1e74bd2ddd9 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 @@ -17,20 +17,18 @@ */ package org.apache.beam.sdk.transforms; +import java.io.Serializable; 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.joda.time.Instant; import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.io.Serializable; - /** * Java 8 tests for {@link WithTimestamps}. */ 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 index bc55c062fabe1..e9f433396e6e5 100644 --- 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 @@ -18,6 +18,9 @@ package ${package}; import ${package}.WordCount; +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; @@ -30,15 +33,9 @@ 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; -import java.util.Arrays; -import java.util.List; -import java.util.regex.Pattern; - - /** * An example that verifies word counts in Shakespeare and includes Dataflow best practices. * 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 index ffe8b881ca5a4..17bf7ca6154be 100644 --- 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 @@ -17,8 +17,14 @@ */ 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 ${package}.common.DataflowExampleUtils; - +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.PipelineResult; import org.apache.beam.sdk.io.PubsubIO; @@ -33,22 +39,11 @@ import org.apache.beam.sdk.transforms.windowing.Window; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; - -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 org.joda.time.Duration; import org.joda.time.Instant; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.io.IOException; -import java.util.ArrayList; -import java.util.List; - - /** * An example that counts words in text, and can run over either unbounded or bounded input * collections. diff --git a/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/common/DataflowExampleUtils.java b/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/common/DataflowExampleUtils.java index fa29fddcdf999..9e6be788d0e4f 100644 --- a/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/common/DataflowExampleUtils.java +++ b/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/common/DataflowExampleUtils.java @@ -17,12 +17,6 @@ */ package ${package}.common; -import org.apache.beam.runners.dataflow.BlockingDataflowRunner; -import org.apache.beam.runners.dataflow.DataflowPipelineJob; -import org.apache.beam.runners.dataflow.DataflowRunner; -import org.apache.beam.runners.dataflow.options.DataflowPipelineOptions; -import org.apache.beam.runners.dataflow.util.MonitoringUtil; - import com.google.api.client.googleapis.json.GoogleJsonResponseException; import com.google.api.client.googleapis.services.AbstractGoogleClientRequest; import com.google.api.services.bigquery.Bigquery; @@ -36,21 +30,24 @@ import com.google.api.services.dataflow.Dataflow; import com.google.api.services.pubsub.Pubsub; import com.google.api.services.pubsub.model.Topic; -import org.apache.beam.sdk.Pipeline; -import org.apache.beam.sdk.PipelineResult; -import org.apache.beam.sdk.io.TextIO; -import org.apache.beam.sdk.options.BigQueryOptions; -import org.apache.beam.sdk.transforms.IntraBundleParallelization; -import org.apache.beam.sdk.util.Transport; import com.google.common.collect.Lists; import com.google.common.collect.Sets; - import java.io.IOException; import java.util.Collection; import java.util.List; import java.util.Set; - import javax.servlet.http.HttpServletResponse; +import org.apache.beam.runners.dataflow.BlockingDataflowRunner; +import org.apache.beam.runners.dataflow.DataflowPipelineJob; +import org.apache.beam.runners.dataflow.DataflowRunner; +import org.apache.beam.runners.dataflow.options.DataflowPipelineOptions; +import org.apache.beam.runners.dataflow.util.MonitoringUtil; +import org.apache.beam.sdk.Pipeline; +import org.apache.beam.sdk.PipelineResult; +import org.apache.beam.sdk.io.TextIO; +import org.apache.beam.sdk.options.BigQueryOptions; +import org.apache.beam.sdk.transforms.IntraBundleParallelization; +import org.apache.beam.sdk.util.Transport; /** * The utility class that sets up and tears down external resources, starts the Google Cloud Pub/Sub 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 index 279f2e098dde1..79fa865eb5299 100644 --- 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 @@ -17,14 +17,13 @@ */ package ${package}.common; +import com.google.api.services.bigquery.model.TableSchema; import org.apache.beam.runners.dataflow.options.DataflowPipelineOptions; 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 com.google.api.services.bigquery.model.TableSchema; - /** * Options that can be used to configure BigQuery tables in Dataflow examples. * The project defaults to the project being used to run the example. diff --git a/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/common/PubsubFileInjector.java b/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/common/PubsubFileInjector.java index 9b347da6089ba..58e082148a4a3 100644 --- a/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/common/PubsubFileInjector.java +++ b/sdks/java/maven-archetypes/examples/src/main/resources/archetype-resources/src/main/java/common/PubsubFileInjector.java @@ -17,6 +17,12 @@ */ package ${package}.common; +import com.google.api.services.pubsub.Pubsub; +import com.google.api.services.pubsub.model.PublishRequest; +import com.google.api.services.pubsub.model.PubsubMessage; +import com.google.common.collect.ImmutableMap; +import java.io.IOException; +import java.util.Arrays; import org.apache.beam.runners.dataflow.options.DataflowPipelineOptions; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.io.TextIO; @@ -28,14 +34,6 @@ import org.apache.beam.sdk.transforms.IntraBundleParallelization; import org.apache.beam.sdk.util.Transport; -import com.google.api.services.pubsub.Pubsub; -import com.google.api.services.pubsub.model.PublishRequest; -import com.google.api.services.pubsub.model.PubsubMessage; -import com.google.common.collect.ImmutableMap; - -import java.io.IOException; -import java.util.Arrays; - /** * A batch Dataflow pipeline for injecting a set of GCS files into * a PubSub topic line by line. Empty lines are skipped. 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 index 4d2e324c1be53..dfa1a75d4135e 100644 --- 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 @@ -18,16 +18,14 @@ package ${package}; import com.google.common.io.Files; - +import java.io.File; +import java.nio.charset.StandardCharsets; import org.junit.Rule; import org.junit.Test; import org.junit.rules.TemporaryFolder; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.io.File; -import java.nio.charset.StandardCharsets; - /** * Tests for {@link DebuggingWordCount}. */ 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 index debfc78623a2e..875d3d757a79c 100644 --- 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 @@ -20,6 +20,8 @@ import ${package}.WordCount.CountWords; import ${package}.WordCount.ExtractWordsFn; import ${package}.WordCount.FormatAsTextFn; +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; @@ -29,7 +31,6 @@ import org.apache.beam.sdk.transforms.DoFnTester; import org.apache.beam.sdk.transforms.ParDo; import org.apache.beam.sdk.values.PCollection; - import org.hamcrest.CoreMatchers; import org.junit.Assert; import org.junit.Test; @@ -37,8 +38,6 @@ import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.util.Arrays; -import java.util.List; /** * Tests of WordCount. 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 6a1c41b0dc87a..0b21aa650a8df 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 @@ -22,7 +22,6 @@ import org.apache.beam.sdk.transforms.Create; import org.apache.beam.sdk.transforms.OldDoFn; import org.apache.beam.sdk.transforms.ParDo; - import org.slf4j.Logger; import org.slf4j.LoggerFactory; 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 7c13350aee522..b332442cadcd7 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 @@ -22,7 +22,6 @@ import org.apache.beam.sdk.transforms.Create; import org.apache.beam.sdk.transforms.OldDoFn; import org.apache.beam.sdk.transforms.ParDo; - import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/sdks/java/microbenchmarks/src/main/java/org/apache/beam/sdk/microbenchmarks/coders/AvroCoderBenchmark.java b/sdks/java/microbenchmarks/src/main/java/org/apache/beam/sdk/microbenchmarks/coders/AvroCoderBenchmark.java index 39b31ef309087..35d5add1471ae 100644 --- a/sdks/java/microbenchmarks/src/main/java/org/apache/beam/sdk/microbenchmarks/coders/AvroCoderBenchmark.java +++ b/sdks/java/microbenchmarks/src/main/java/org/apache/beam/sdk/microbenchmarks/coders/AvroCoderBenchmark.java @@ -17,9 +17,10 @@ */ package org.apache.beam.sdk.microbenchmarks.coders; +import java.io.IOException; +import java.util.Arrays; import org.apache.beam.sdk.coders.AvroCoder; import org.apache.beam.sdk.coders.DefaultCoder; - import org.openjdk.jmh.annotations.Benchmark; import org.openjdk.jmh.annotations.Fork; import org.openjdk.jmh.annotations.Param; @@ -28,9 +29,6 @@ import org.openjdk.jmh.annotations.State; import org.openjdk.jmh.annotations.Warmup; -import java.io.IOException; -import java.util.Arrays; - /** * Benchmarks for {@link AvroCoder}. */ diff --git a/sdks/java/microbenchmarks/src/main/java/org/apache/beam/sdk/microbenchmarks/coders/ByteArrayCoderBenchmark.java b/sdks/java/microbenchmarks/src/main/java/org/apache/beam/sdk/microbenchmarks/coders/ByteArrayCoderBenchmark.java index df20a15d4ec21..78909752a5d46 100644 --- a/sdks/java/microbenchmarks/src/main/java/org/apache/beam/sdk/microbenchmarks/coders/ByteArrayCoderBenchmark.java +++ b/sdks/java/microbenchmarks/src/main/java/org/apache/beam/sdk/microbenchmarks/coders/ByteArrayCoderBenchmark.java @@ -17,8 +17,9 @@ */ package org.apache.beam.sdk.microbenchmarks.coders; +import java.io.IOException; +import java.util.Arrays; import org.apache.beam.sdk.coders.ByteArrayCoder; - import org.openjdk.jmh.annotations.Benchmark; import org.openjdk.jmh.annotations.Fork; import org.openjdk.jmh.annotations.Param; @@ -27,9 +28,6 @@ import org.openjdk.jmh.annotations.State; import org.openjdk.jmh.annotations.Warmup; -import java.io.IOException; -import java.util.Arrays; - /** * Benchmarks for {@link ByteArrayCoder}. */ diff --git a/sdks/java/microbenchmarks/src/main/java/org/apache/beam/sdk/microbenchmarks/coders/CoderBenchmarking.java b/sdks/java/microbenchmarks/src/main/java/org/apache/beam/sdk/microbenchmarks/coders/CoderBenchmarking.java index 8523cb2a6a1f4..c92215d0afc54 100644 --- a/sdks/java/microbenchmarks/src/main/java/org/apache/beam/sdk/microbenchmarks/coders/CoderBenchmarking.java +++ b/sdks/java/microbenchmarks/src/main/java/org/apache/beam/sdk/microbenchmarks/coders/CoderBenchmarking.java @@ -17,11 +17,10 @@ */ package org.apache.beam.sdk.microbenchmarks.coders; +import java.io.IOException; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.util.CoderUtils; -import java.io.IOException; - /** * Utilities for writing coder benchmarks. */ diff --git a/sdks/java/microbenchmarks/src/main/java/org/apache/beam/sdk/microbenchmarks/coders/StringUtf8CoderBenchmark.java b/sdks/java/microbenchmarks/src/main/java/org/apache/beam/sdk/microbenchmarks/coders/StringUtf8CoderBenchmark.java index c0bcb453944b9..540c95899c21d 100644 --- a/sdks/java/microbenchmarks/src/main/java/org/apache/beam/sdk/microbenchmarks/coders/StringUtf8CoderBenchmark.java +++ b/sdks/java/microbenchmarks/src/main/java/org/apache/beam/sdk/microbenchmarks/coders/StringUtf8CoderBenchmark.java @@ -17,8 +17,9 @@ */ package org.apache.beam.sdk.microbenchmarks.coders; +import java.io.IOException; +import java.util.Arrays; import org.apache.beam.sdk.coders.StringUtf8Coder; - import org.openjdk.jmh.annotations.Benchmark; import org.openjdk.jmh.annotations.Fork; import org.openjdk.jmh.annotations.Param; @@ -27,9 +28,6 @@ import org.openjdk.jmh.annotations.State; import org.openjdk.jmh.annotations.Warmup; -import java.io.IOException; -import java.util.Arrays; - /** * Benchmarks for {@link StringUtf8Coder}. */ From 8d3219686169b0bf798b576c7e5f30b01b6d6da8 Mon Sep 17 00:00:00 2001 From: Dan Halperin Date: Wed, 24 Aug 2016 09:49:46 -0700 Subject: [PATCH 067/346] BigQueryIO.Write: raise size limit to 11 TiB BigQuery has changed their total size quota to 12 TiB. https://cloud.google.com/bigquery/quota-policy#import --- .../java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java | 4 ++-- 1 file changed, 2 insertions(+), 2 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 04fb041badd13..01a8a1c5cd4e8 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 @@ -1420,8 +1420,8 @@ public static class Bound extends PTransform, PDone> { // Maximum number of files in a single partition. static final int MAX_NUM_FILES = 10000; - // Maximum number of bytes in a single partition. - static final long MAX_SIZE_BYTES = 3 * (1L << 40); + // Maximum number of bytes in a single partition -- 11 TiB just under BQ's 12 TiB limit. + static final long MAX_SIZE_BYTES = 11 * (1L << 40); // The maximum number of retry jobs. static final int MAX_RETRY_JOBS = 3; From 186fe28f330664ab5237ab103923d4372c5eb37d Mon Sep 17 00:00:00 2001 From: Pei He Date: Wed, 24 Aug 2016 14:47:11 -0700 Subject: [PATCH 068/346] Cleanup some javadoc that referring Dataflow --- .../java/org/apache/beam/examples/DebuggingWordCount.java | 2 +- .../main/java/org/apache/beam/examples/WindowedWordCount.java | 3 +-- .../java/org/apache/beam/examples/common/ExampleOptions.java | 2 +- .../src/main/java/org/apache/beam/examples/complete/README.md | 4 ++-- .../apache/beam/examples/complete/StreamingWordExtract.java | 2 +- .../apache/beam/examples/complete/TopWikipediaSessions.java | 2 +- .../org/apache/beam/examples/complete/TrafficMaxLaneFlow.java | 4 ++-- .../java/org/apache/beam/examples/complete/TrafficRoutes.java | 4 ++-- 8 files changed, 11 insertions(+), 12 deletions(-) diff --git a/examples/java/src/main/java/org/apache/beam/examples/DebuggingWordCount.java b/examples/java/src/main/java/org/apache/beam/examples/DebuggingWordCount.java index 4a9aba9da5e8c..5a0930cd35f5b 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/DebuggingWordCount.java +++ b/examples/java/src/main/java/org/apache/beam/examples/DebuggingWordCount.java @@ -37,7 +37,7 @@ /** - * An example that verifies word counts in Shakespeare and includes Dataflow best practices. + * 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} 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 6d69f1488526d..5f60524209c19 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 @@ -183,8 +183,7 @@ public static interface Options extends WordCount.WordCountOptions, public static void main(String[] args) throws IOException { Options options = PipelineOptionsFactory.fromArgs(args).withValidation().as(Options.class); options.setBigQuerySchema(getSchema()); - // DataflowExampleUtils creates the necessary input sources to simplify execution of this - // Pipeline. + // ExampleUtils creates the necessary input sources to simplify execution of this Pipeline. ExampleUtils exampleUtils = new ExampleUtils(options); exampleUtils.setup(); diff --git a/examples/java/src/main/java/org/apache/beam/examples/common/ExampleOptions.java b/examples/java/src/main/java/org/apache/beam/examples/common/ExampleOptions.java index a7dcc7c175176..8b7ed073f3593 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/common/ExampleOptions.java +++ b/examples/java/src/main/java/org/apache/beam/examples/common/ExampleOptions.java @@ -33,7 +33,7 @@ * Options that can be used to configure the Beam examples. */ public interface ExampleOptions extends PipelineOptions { - @Description("Whether to keep jobs running on the Dataflow service after local process exit") + @Description("Whether to keep jobs running after local process exit") @Default.Boolean(false) boolean getKeepJobsRunning(); void setKeepJobsRunning(boolean keepJobsRunning); 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 99c93ef4b82a5..b98be7a723960 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 @@ -43,14 +43,14 @@ This directory contains end-to-end example pipelines that perform complex data p Windowing to perform time-based aggregations of data.

      • TrafficMaxLaneFlow - — A streaming Cloud Dataflow example using BigQuery output in the + — 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 - — A streaming Cloud Dataflow example using BigQuery output in the + — 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 Pub/Sub topic ingestion. diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/StreamingWordExtract.java b/examples/java/src/main/java/org/apache/beam/examples/complete/StreamingWordExtract.java index c4ad35bcc8189..348bab84b996d 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/complete/StreamingWordExtract.java +++ b/examples/java/src/main/java/org/apache/beam/examples/complete/StreamingWordExtract.java @@ -37,7 +37,7 @@ import org.apache.beam.sdk.transforms.ParDo; /** - * A streaming Dataflow Example using BigQuery output. + * A streaming Beam Example using BigQuery output. * *

        This pipeline example reads lines of the input text file, splits each line * into individual words, capitalizes those words, and writes the output to 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 4c5472bf6f687..1b2064ad068ee 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 @@ -181,7 +181,7 @@ public void processElement(ProcessContext c) { /** * Options supported by this class. * - *

        Inherits standard Dataflow configuration options. + *

        Inherits standard Beam configuration options. */ private static interface Options extends PipelineOptions { @Description( 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 f7ff035dbf50e..1b27e650f03e9 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 @@ -53,7 +53,7 @@ import org.joda.time.format.DateTimeFormatter; /** - * A Dataflow Example that runs in both batch and streaming modes with traffic sensor data. + * A Beam Example that runs in both batch and streaming modes with traffic sensor data. * You can configure the running mode by setting {@literal --streaming} to true or false. * *

        Concepts: The batch and streaming runners, sliding windows, @@ -329,7 +329,7 @@ public static void main(String[] args) throws IOException { .withValidation() .as(TrafficMaxLaneFlowOptions.class); options.setBigQuerySchema(FormatMaxesFn.getSchema()); - // Using DataflowExampleUtils to set up required resources. + // Using ExampleUtils to set up required resources. ExampleUtils exampleUtils = new ExampleUtils(options); exampleUtils.setup(); 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 12eaeaa972579..f3c2d3936ee74 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 @@ -57,7 +57,7 @@ import org.joda.time.format.DateTimeFormatter; /** - * A Dataflow Example that runs in both batch and streaming modes with traffic sensor data. + * A Beam Example that runs in both batch and streaming modes with traffic sensor data. * You can configure the running mode by setting {@literal --streaming} to true or false. * *

        Concepts: The batch and streaming runners, GroupByKey, sliding windows. @@ -340,7 +340,7 @@ public static void main(String[] args) throws IOException { .as(TrafficRoutesOptions.class); options.setBigQuerySchema(FormatStatsFn.getSchema()); - // Using DataflowExampleUtils to set up required resources. + // Using ExampleUtils to set up required resources. ExampleUtils exampleUtils = new ExampleUtils(options); exampleUtils.setup(); From 433842b5ca4a8167488574492f6d1ecbe6330d91 Mon Sep 17 00:00:00 2001 From: Pei He Date: Wed, 24 Aug 2016 17:26:50 -0700 Subject: [PATCH 069/346] Move the samples data to gs://apache-beam-samples/ --- .../java/org/apache/beam/examples/DebuggingWordCount.java | 4 ++-- .../main/java/org/apache/beam/examples/MinimalWordCount.java | 2 +- .../java/org/apache/beam/examples/WindowedWordCount.java | 2 +- .../src/main/java/org/apache/beam/examples/WordCount.java | 4 ++-- .../apache/beam/examples/complete/StreamingWordExtract.java | 2 +- .../main/java/org/apache/beam/examples/complete/TfIdf.java | 4 ++-- .../apache/beam/examples/complete/TopWikipediaSessions.java | 5 +++-- .../apache/beam/examples/complete/TrafficMaxLaneFlow.java | 2 +- .../org/apache/beam/examples/complete/TrafficRoutes.java | 2 +- .../apache/beam/examples/cookbook/DatastoreWordCount.java | 2 +- .../java/org/apache/beam/examples/cookbook/DeDupExample.java | 4 ++-- .../org/apache/beam/examples/cookbook/TriggerExample.java | 2 +- .../java/org/apache/beam/examples/MinimalWordCountJava8.java | 2 +- .../apache/beam/examples/complete/game/HourlyTeamScore.java | 2 +- .../org/apache/beam/examples/complete/game/UserScore.java | 2 +- .../org/apache/beam/examples/MinimalWordCountJava8Test.java | 2 +- 16 files changed, 22 insertions(+), 21 deletions(-) diff --git a/examples/java/src/main/java/org/apache/beam/examples/DebuggingWordCount.java b/examples/java/src/main/java/org/apache/beam/examples/DebuggingWordCount.java index 5a0930cd35f5b..be3aa419b5379 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/DebuggingWordCount.java +++ b/examples/java/src/main/java/org/apache/beam/examples/DebuggingWordCount.java @@ -100,8 +100,8 @@ * that changing the default worker log level to TRACE or DEBUG will significantly increase * the amount of logs output. * - *

        The input file defaults to {@code gs://dataflow-samples/shakespeare/kinglear.txt} and can be - * overridden with {@code --inputFile}. + *

        The input file defaults to {@code gs://apache-beam-samples/shakespeare/kinglear.txt} + * and can be overridden with {@code --inputFile}. */ public class DebuggingWordCount { /** A DoFn that filters for a specific key based upon a regular expression. */ diff --git a/examples/java/src/main/java/org/apache/beam/examples/MinimalWordCount.java b/examples/java/src/main/java/org/apache/beam/examples/MinimalWordCount.java index df725e3f15858..f28a20cf94491 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/MinimalWordCount.java +++ b/examples/java/src/main/java/org/apache/beam/examples/MinimalWordCount.java @@ -87,7 +87,7 @@ public static void main(String[] args) { // 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). - p.apply(TextIO.Read.from("gs://dataflow-samples/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 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 5f60524209c19..7af354cee0fab 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 @@ -82,7 +82,7 @@ * *

        Optionally specify the input file path via: * {@code --inputFile=gs://INPUT_PATH}, - * which defaults to {@code gs://dataflow-samples/shakespeare/kinglear.txt}. + * 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 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 d42d6214973d3..0275651bf288b 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 @@ -89,8 +89,8 @@ * --output=gs://YOUR_OUTPUT_PREFIX * } * - *

        The input file defaults to {@code gs://dataflow-samples/shakespeare/kinglear.txt} and can be - * overridden with {@code --inputFile}. + *

        The input file defaults to {@code gs://apache-beam-samples/shakespeare/kinglear.txt} + * and can be overridden with {@code --inputFile}. */ public class WordCount { diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/StreamingWordExtract.java b/examples/java/src/main/java/org/apache/beam/examples/complete/StreamingWordExtract.java index 348bab84b996d..869ea69b8ae47 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/complete/StreamingWordExtract.java +++ b/examples/java/src/main/java/org/apache/beam/examples/complete/StreamingWordExtract.java @@ -104,7 +104,7 @@ static TableSchema getSchema() { private interface StreamingWordExtractOptions extends ExampleOptions, ExampleBigQueryTableOptions, StreamingOptions { @Description("Path of the file to read from") - @Default.String("gs://dataflow-samples/shakespeare/kinglear.txt") + @Default.String("gs://apache-beam-samples/shakespeare/kinglear.txt") String getInputFile(); void setInputFile(String value); } 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 a5a939263ee4e..87023edac29ac 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 @@ -83,7 +83,7 @@ * --output=gs://YOUR_OUTPUT_PREFIX * } * - *

        The default input is {@code gs://dataflow-samples/shakespeare/} and can be overridden with + *

        The default input is {@code gs://apache-beam-samples/shakespeare/} and can be overridden with * {@code --input}. */ public class TfIdf { @@ -94,7 +94,7 @@ public class TfIdf { */ private static interface Options extends PipelineOptions { @Description("Path to the directory or GCS prefix containing files to read from") - @Default.String("gs://dataflow-samples/shakespeare/") + @Default.String("gs://apache-beam-samples/shakespeare/") String getInput(); void setInput(String value); 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 1b2064ad068ee..d597258d3b923 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 @@ -64,7 +64,7 @@ * --output=gs://YOUR_OUTPUT_PREFIX * } * - *

        The default input is {@code gs://dataflow-samples/wikipedia_edits/*.json} and can be + *

        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) @@ -77,7 +77,8 @@ * 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 = "gs://dataflow-samples/wikipedia_edits/*.json"; + private static final String EXPORTED_WIKI_TABLE = + "gs://apache-beam-samples/wikipedia_edits/*.json"; /** * Extracts user and timestamp from a TableRow representing a Wikipedia edit. 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 1b27e650f03e9..e4569600c05c6 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 @@ -303,7 +303,7 @@ public PCollection apply(PBegin begin) { */ private interface TrafficMaxLaneFlowOptions extends ExampleOptions, ExampleBigQueryTableOptions { @Description("Path of the file to read from") - @Default.String("gs://dataflow-samples/traffic_sensor/" + @Default.String("gs://apache-beam-samples/traffic_sensor/" + "Freeways-5Minaa2010-01-01_to_2010-02-15_test2.csv") String getInputFile(); void setInputFile(String value); 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 f3c2d3936ee74..95336c644a79f 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 @@ -313,7 +313,7 @@ public PCollection apply(PBegin begin) { */ private interface TrafficRoutesOptions extends ExampleOptions, ExampleBigQueryTableOptions { @Description("Path of the file to read from") - @Default.String("gs://dataflow-samples/traffic_sensor/" + @Default.String("gs://apache-beam-samples/traffic_sensor/" + "Freeways-5Minaa2010-01-01_to_2010-02-15_test2.csv") String getInputFile(); void setInputFile(String value); diff --git a/examples/java/src/main/java/org/apache/beam/examples/cookbook/DatastoreWordCount.java b/examples/java/src/main/java/org/apache/beam/examples/cookbook/DatastoreWordCount.java index 9a9e79968670e..eb2165f91e19b 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/cookbook/DatastoreWordCount.java +++ b/examples/java/src/main/java/org/apache/beam/examples/cookbook/DatastoreWordCount.java @@ -151,7 +151,7 @@ public void processElement(ProcessContext c) { */ public static interface Options extends PipelineOptions { @Description("Path of the file to read from and store to Datastore") - @Default.String("gs://dataflow-samples/shakespeare/kinglear.txt") + @Default.String("gs://apache-beam-samples/shakespeare/kinglear.txt") String getInput(); void setInput(String value); diff --git a/examples/java/src/main/java/org/apache/beam/examples/cookbook/DeDupExample.java b/examples/java/src/main/java/org/apache/beam/examples/cookbook/DeDupExample.java index d573bcd9d295e..57917109addd1 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/cookbook/DeDupExample.java +++ b/examples/java/src/main/java/org/apache/beam/examples/cookbook/DeDupExample.java @@ -47,7 +47,7 @@ * and an output prefix on GCS: * --output=gs://YOUR_OUTPUT_PREFIX * - *

        The input defaults to {@code gs://dataflow-samples/shakespeare/*} and can be + *

        The input defaults to {@code gs://apache-beam-samples/shakespeare/*} and can be * overridden with {@code --input}. */ public class DeDupExample { @@ -59,7 +59,7 @@ public class DeDupExample { */ private static interface Options extends PipelineOptions { @Description("Path to the directory or GCS prefix containing files to read from") - @Default.String("gs://dataflow-samples/shakespeare/*") + @Default.String("gs://apache-beam-samples/shakespeare/*") String getInput(); void setInput(String value); 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 db59435555458..263054138dd64 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 @@ -420,7 +420,7 @@ public interface TrafficFlowOptions extends ExampleOptions, ExampleBigQueryTableOptions, StreamingOptions { @Description("Input file to read from") - @Default.String("gs://dataflow-samples/traffic_sensor/" + @Default.String("gs://apache-beam-samples/traffic_sensor/" + "Freeways-5Minaa2010-01-01_to_2010-02-15.csv") String getInput(); void setInput(String value); diff --git a/examples/java8/src/main/java/org/apache/beam/examples/MinimalWordCountJava8.java b/examples/java8/src/main/java/org/apache/beam/examples/MinimalWordCountJava8.java index a49da7bdfbb65..24dd6f9b2c483 100644 --- a/examples/java8/src/main/java/org/apache/beam/examples/MinimalWordCountJava8.java +++ b/examples/java8/src/main/java/org/apache/beam/examples/MinimalWordCountJava8.java @@ -55,7 +55,7 @@ public static void main(String[] args) { Pipeline p = Pipeline.create(options); - p.apply(TextIO.Read.from("gs://dataflow-samples/shakespeare/*")) + p.apply(TextIO.Read.from("gs://apache-beam-samples/shakespeare/*")) .apply(FlatMapElements.via((String word) -> Arrays.asList(word.split("[^a-zA-Z']+"))) .withOutputType(TypeDescriptors.strings())) .apply(Filter.by((String word) -> !word.isEmpty())) diff --git a/examples/java8/src/main/java/org/apache/beam/examples/complete/game/HourlyTeamScore.java b/examples/java8/src/main/java/org/apache/beam/examples/complete/game/HourlyTeamScore.java index d408e2132dabf..cf1389981fc5b 100644 --- a/examples/java8/src/main/java/org/apache/beam/examples/complete/game/HourlyTeamScore.java +++ b/examples/java8/src/main/java/org/apache/beam/examples/complete/game/HourlyTeamScore.java @@ -70,7 +70,7 @@ * timestamped after 23:59 PST on 2015-10-18 should not be included in the analysis. * To indicate a time before which data should be filtered out, include the {@code --startMin} arg. * If you're using the default input specified in {@link UserScore}, - * "gs://dataflow-samples/game/gaming_data*.csv", then + * "gs://apache-beam-samples/game/gaming_data*.csv", then * {@code --startMin=2015-11-16-16-10 --stopMin=2015-11-17-16-10} are good values. */ public class HourlyTeamScore extends UserScore { 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 65036cee6b922..f05879f93106f 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 @@ -182,7 +182,7 @@ public static interface Options extends PipelineOptions { @Description("Path to the data file(s) containing game data.") // The default maps to two large Google Cloud Storage files (each ~12GB) holding two subsequent // day's worth (roughly) of data. - @Default.String("gs://dataflow-samples/game/gaming_data*.csv") + @Default.String("gs://apache-beam-samples/game/gaming_data*.csv") String getInput(); void setInput(String value); 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 85841a7801474..181921920b002 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 @@ -60,7 +60,7 @@ public void testMinimalWordCountJava8() throws Exception { Pipeline p = TestPipeline.create(); p.getOptions().as(GcsOptions.class).setGcsUtil(buildMockGcsUtil()); - p.apply(TextIO.Read.from("gs://dataflow-samples/shakespeare/*")) + p.apply(TextIO.Read.from("gs://apache-beam-samples/shakespeare/*")) .apply(FlatMapElements.via((String word) -> Arrays.asList(word.split("[^a-zA-Z']+"))) .withOutputType(TypeDescriptors.strings())) .apply(Filter.by((String word) -> !word.isEmpty())) From 32928c31aa665f9052039b7ed884530290df1034 Mon Sep 17 00:00:00 2001 From: Pei He Date: Wed, 10 Aug 2016 15:21:41 -0700 Subject: [PATCH 070/346] [BEAM-545] Promote JobName to PipelineOptions --- .../common/ExampleBigQueryTableOptions.java | 3 +- .../beam/examples/common/ExampleOptions.java | 43 ----------------- ...mplePubsubTopicAndSubscriptionOptions.java | 2 +- .../common/ExamplePubsubTopicOptions.java | 2 +- .../runners/flink/FlinkPipelineOptions.java | 34 ------------- .../options/DataflowPipelineOptions.java | 48 ------------------- .../options/DataflowPipelineOptionsTest.java | 42 +++++++++++++--- .../beam/sdk/options/PipelineOptions.java | 42 ++++++++++++++++ 8 files changed, 80 insertions(+), 136 deletions(-) diff --git a/examples/java/src/main/java/org/apache/beam/examples/common/ExampleBigQueryTableOptions.java b/examples/java/src/main/java/org/apache/beam/examples/common/ExampleBigQueryTableOptions.java index 2eef525faed0b..5d815c72cc09d 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/common/ExampleBigQueryTableOptions.java +++ b/examples/java/src/main/java/org/apache/beam/examples/common/ExampleBigQueryTableOptions.java @@ -49,8 +49,7 @@ public interface ExampleBigQueryTableOptions extends GcpOptions { static class BigQueryTableFactory implements DefaultValueFactory { @Override public String create(PipelineOptions options) { - return options.as(ExampleOptions.class).getNormalizedUniqueName() - .replace('-', '_'); + return options.getJobName().replace('-', '_'); } } } diff --git a/examples/java/src/main/java/org/apache/beam/examples/common/ExampleOptions.java b/examples/java/src/main/java/org/apache/beam/examples/common/ExampleOptions.java index 8b7ed073f3593..85643e41f352f 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/common/ExampleOptions.java +++ b/examples/java/src/main/java/org/apache/beam/examples/common/ExampleOptions.java @@ -17,17 +17,9 @@ */ package org.apache.beam.examples.common; -import com.google.common.base.MoreObjects; -import java.util.concurrent.ThreadLocalRandom; -import org.apache.beam.sdk.options.ApplicationNameOptions; 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.joda.time.DateTimeUtils; -import org.joda.time.DateTimeZone; -import org.joda.time.format.DateTimeFormat; -import org.joda.time.format.DateTimeFormatter; /** * Options that can be used to configure the Beam examples. @@ -42,39 +34,4 @@ public interface ExampleOptions extends PipelineOptions { @Default.Integer(1) int getInjectorNumWorkers(); void setInjectorNumWorkers(int numWorkers); - - @Description("A normalized unique name that is used to name anything related to the pipeline." - + "It defaults to ApplicationName-UserName-Date-RandomInteger") - @Default.InstanceFactory(NormalizedUniqueNameFactory.class) - String getNormalizedUniqueName(); - void setNormalizedUniqueName(String numWorkers); - - /** - * Returns a normalized unique name constructed from {@link ApplicationNameOptions#getAppName()}, - * the local system user name (if available), the current time, and a random integer. - * - *

        The normalization makes sure that the name matches the pattern of - * [a-z]([-a-z0-9]*[a-z0-9])?. - */ - public static class NormalizedUniqueNameFactory implements DefaultValueFactory { - private static final DateTimeFormatter FORMATTER = - DateTimeFormat.forPattern("MMddHHmmss").withZone(DateTimeZone.UTC); - - @Override - public String create(PipelineOptions options) { - String appName = options.as(ApplicationNameOptions.class).getAppName(); - String normalizedAppName = appName == null || appName.length() == 0 ? "BeamApp" - : appName.toLowerCase() - .replaceAll("[^a-z0-9]", "0") - .replaceAll("^[^a-z]", "a"); - String userName = MoreObjects.firstNonNull(System.getProperty("user.name"), ""); - String normalizedUserName = userName.toLowerCase() - .replaceAll("[^a-z0-9]", "0"); - String datePart = FORMATTER.print(DateTimeUtils.currentTimeMillis()); - - String randomPart = Integer.toHexString(ThreadLocalRandom.current().nextInt()); - return String.format("%s-%s-%s-%s", - normalizedAppName, normalizedUserName, datePart, randomPart); - } - } } diff --git a/examples/java/src/main/java/org/apache/beam/examples/common/ExamplePubsubTopicAndSubscriptionOptions.java b/examples/java/src/main/java/org/apache/beam/examples/common/ExamplePubsubTopicAndSubscriptionOptions.java index 36893a3a69909..7f954a1c18509 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/common/ExamplePubsubTopicAndSubscriptionOptions.java +++ b/examples/java/src/main/java/org/apache/beam/examples/common/ExamplePubsubTopicAndSubscriptionOptions.java @@ -39,7 +39,7 @@ static class PubsubSubscriptionFactory implements DefaultValueFactory { @Override public String create(PipelineOptions options) { return "projects/" + options.as(GcpOptions.class).getProject() - + "/subscriptions/" + options.as(ExampleOptions.class).getNormalizedUniqueName(); + + "/subscriptions/" + options.getJobName(); } } } diff --git a/examples/java/src/main/java/org/apache/beam/examples/common/ExamplePubsubTopicOptions.java b/examples/java/src/main/java/org/apache/beam/examples/common/ExamplePubsubTopicOptions.java index 1c9270b8f0656..71879b7242765 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/common/ExamplePubsubTopicOptions.java +++ b/examples/java/src/main/java/org/apache/beam/examples/common/ExamplePubsubTopicOptions.java @@ -39,7 +39,7 @@ static class PubsubTopicFactory implements DefaultValueFactory { @Override public String create(PipelineOptions options) { return "projects/" + options.as(GcpOptions.class).getProject() - + "/topics/" + options.as(ExampleOptions.class).getNormalizedUniqueName(); + + "/topics/" + options.getJobName(); } } } 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 6d1a8d040f1a5..6561fa5ef996c 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 @@ -22,14 +22,9 @@ import java.util.List; import org.apache.beam.sdk.options.ApplicationNameOptions; 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.StreamingOptions; -import org.joda.time.DateTimeUtils; -import org.joda.time.DateTimeZone; -import org.joda.time.format.DateTimeFormat; -import org.joda.time.format.DateTimeFormatter; /** * Options which can be used to configure a Flink PipelineRunner. @@ -49,15 +44,6 @@ public interface FlinkPipelineOptions extends PipelineOptions, ApplicationNameOp List getFilesToStage(); void setFilesToStage(List value); - /** - * The job name is used to identify jobs running on a Flink cluster. - */ - @Description("Flink job name, to uniquely identify active jobs. " - + "Defaults to using the ApplicationName-UserName-Date.") - @Default.InstanceFactory(JobNameFactory.class) - String getJobName(); - void setJobName(String value); - /** * The url of the Flink JobManager on which to execute pipelines. This can either be * the the address of a cluster JobManager, in the form "host:port" or one of the special @@ -93,24 +79,4 @@ public interface FlinkPipelineOptions extends PipelineOptions, ApplicationNameOp @Default.Long(-1L) Long getExecutionRetryDelay(); void setExecutionRetryDelay(Long delay); - - - class JobNameFactory implements DefaultValueFactory { - private static final DateTimeFormatter FORMATTER = - DateTimeFormat.forPattern("MMddHHmmss").withZone(DateTimeZone.UTC); - - @Override - public String create(PipelineOptions options) { - String appName = options.as(ApplicationNameOptions.class).getAppName(); - String normalizedAppName = appName == null || appName.length() == 0 ? "FlinkRunner" - : appName.toLowerCase() - .replaceAll("[^a-z0-9]", "0") - .replaceAll("^[^a-z]", "a"); - String userName = System.getProperty("user.name", ""); - String normalizedUserName = userName.toLowerCase() - .replaceAll("[^a-z0-9]", "0"); - String datePart = FORMATTER.print(DateTimeUtils.currentTimeMillis()); - return normalizedAppName + "-" + normalizedUserName + "-" + datePart; - } - } } 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 8ef43c5efaf66..9f58f93c9ecc3 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 @@ -20,7 +20,6 @@ import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.base.Strings.isNullOrEmpty; -import com.google.common.base.MoreObjects; import java.io.IOException; import org.apache.beam.runners.dataflow.DataflowRunner; import org.apache.beam.sdk.options.ApplicationNameOptions; @@ -35,10 +34,6 @@ import org.apache.beam.sdk.options.StreamingOptions; import org.apache.beam.sdk.options.Validation; import org.apache.beam.sdk.util.IOChannelUtils; -import org.joda.time.DateTimeUtils; -import org.joda.time.DateTimeZone; -import org.joda.time.format.DateTimeFormat; -import org.joda.time.format.DateTimeFormatter; /** * Options that can be used to configure the {@link DataflowRunner}. @@ -75,21 +70,6 @@ public interface DataflowPipelineOptions String getStagingLocation(); void setStagingLocation(String value); - /** - * The Dataflow job name is used as an idempotence key within the Dataflow service. - * If there is an existing job that is currently active, another active job with the same - * name will not be able to be created. Defaults to using the ApplicationName-UserName-Date. - */ - @Description("The Dataflow job name is used as an idempotence key within the Dataflow service. " - + "For each running job in the same GCP project, jobs with the same name cannot be created " - + "unless the new job is an explicit update of the previous one. Defaults to using " - + "ApplicationName-UserName-Date. The job name must match the regular expression " - + "'[a-z]([-a-z0-9]{0,38}[a-z0-9])?'. The runner will automatically truncate the name of the " - + "job and convert to lower case.") - @Default.InstanceFactory(JobNameFactory.class) - String getJobName(); - void setJobName(String value); - /** * Whether to update the currently running pipeline with the same name as this one. */ @@ -99,34 +79,6 @@ public interface DataflowPipelineOptions boolean isUpdate(); void setUpdate(boolean value); - /** - * Returns a normalized job name constructed from {@link ApplicationNameOptions#getAppName()}, the - * local system user name (if available), and the current time. The normalization makes sure that - * the job name matches the required pattern of [a-z]([-a-z0-9]*[a-z0-9])? and length limit of 40 - * characters. - * - *

        This job name factory is only able to generate one unique name per second per application - * and user combination. - */ - public static class JobNameFactory implements DefaultValueFactory { - private static final DateTimeFormatter FORMATTER = - DateTimeFormat.forPattern("MMddHHmmss").withZone(DateTimeZone.UTC); - - @Override - public String create(PipelineOptions options) { - String appName = options.as(ApplicationNameOptions.class).getAppName(); - String normalizedAppName = appName == null || appName.length() == 0 ? "dataflow" - : appName.toLowerCase() - .replaceAll("[^a-z0-9]", "0") - .replaceAll("^[^a-z]", "a"); - String userName = MoreObjects.firstNonNull(System.getProperty("user.name"), ""); - String normalizedUserName = userName.toLowerCase() - .replaceAll("[^a-z0-9]", "0"); - String datePart = FORMATTER.print(DateTimeUtils.currentTimeMillis()); - return normalizedAppName + "-" + normalizedUserName + "-" + datePart; - } - } - /** * Returns a default staging location under {@link GcpOptions#getGcpTempLocation}. */ 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 61d39923ae5dd..202d04b1e9a85 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 @@ -53,7 +53,13 @@ public void testUserNameIsNotSet() { System.getProperties().remove("user.name"); DataflowPipelineOptions options = PipelineOptionsFactory.as(DataflowPipelineOptions.class); options.setAppName("TestApplication"); - assertEquals("testapplication--1208190706", options.getJobName()); + String[] nameComponents = options.getJobName().split("-"); + assertEquals(4, nameComponents.length); + assertEquals("testapplication", nameComponents[0]); + assertEquals("", nameComponents[1]); + assertEquals("1208190706", nameComponents[2]); + // Verify the last component is a hex integer (unsigned). + Long.parseLong(nameComponents[3], 16); assertTrue(options.getJobName().length() <= 40); } @@ -63,9 +69,13 @@ public void testAppNameAndUserNameAreLong() { System.getProperties().put("user.name", "abcdeabcdeabcdeabcdeabcdeabcde"); DataflowPipelineOptions options = PipelineOptionsFactory.as(DataflowPipelineOptions.class); options.setAppName("1234567890123456789012345678901234567890"); - assertEquals( - "a234567890123456789012345678901234567890-abcdeabcdeabcdeabcdeabcdeabcde-1208190706", - options.getJobName()); + String[] nameComponents = options.getJobName().split("-"); + assertEquals(4, nameComponents.length); + assertEquals("a234567890123456789012345678901234567890", nameComponents[0]); + assertEquals("abcdeabcdeabcdeabcdeabcdeabcde", nameComponents[1]); + assertEquals("1208190706", nameComponents[2]); + // Verify the last component is a hex integer (unsigned). + Long.parseLong(nameComponents[3], 16); } @Test @@ -74,7 +84,13 @@ public void testAppNameIsLong() { System.getProperties().put("user.name", "abcde"); DataflowPipelineOptions options = PipelineOptionsFactory.as(DataflowPipelineOptions.class); options.setAppName("1234567890123456789012345678901234567890"); - assertEquals("a234567890123456789012345678901234567890-abcde-1208190706", options.getJobName()); + String[] nameComponents = options.getJobName().split("-"); + assertEquals(4, nameComponents.length); + assertEquals("a234567890123456789012345678901234567890", nameComponents[0]); + assertEquals("abcde", nameComponents[1]); + assertEquals("1208190706", nameComponents[2]); + // Verify the last component is a hex integer (unsigned). + Long.parseLong(nameComponents[3], 16); } @Test @@ -83,7 +99,13 @@ public void testUserNameIsLong() { System.getProperties().put("user.name", "abcdeabcdeabcdeabcdeabcdeabcde"); DataflowPipelineOptions options = PipelineOptionsFactory.as(DataflowPipelineOptions.class); options.setAppName("1234567890"); - assertEquals("a234567890-abcdeabcdeabcdeabcdeabcdeabcde-1208190706", options.getJobName()); + String[] nameComponents = options.getJobName().split("-"); + assertEquals(4, nameComponents.length); + assertEquals("a234567890", nameComponents[0]); + assertEquals("abcdeabcdeabcdeabcdeabcdeabcde", nameComponents[1]); + assertEquals("1208190706", nameComponents[2]); + // Verify the last component is a hex integer (unsigned). + Long.parseLong(nameComponents[3], 16); } @Test @@ -92,7 +114,13 @@ public void testUtf8UserNameAndApplicationNameIsNormalized() { System.getProperties().put("user.name", "ði ıntəˈnæʃənəl "); DataflowPipelineOptions options = PipelineOptionsFactory.as(DataflowPipelineOptions.class); options.setAppName("fəˈnɛtık əsoʊsiˈeıʃn"); - assertEquals("f00n0t0k00so0si0e00n-0i00nt00n000n0l0-1208190706", options.getJobName()); + String[] nameComponents = options.getJobName().split("-"); + assertEquals(4, nameComponents.length); + assertEquals("f00n0t0k00so0si0e00n", nameComponents[0]); + assertEquals("0i00nt00n000n0l0", nameComponents[1]); + assertEquals("1208190706", nameComponents[2]); + // Verify the last component is a hex integer (unsigned). + Long.parseLong(nameComponents[3], 16); } @Test 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 db54d0ac76b67..701ae70eafde4 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 @@ -22,8 +22,10 @@ import com.fasterxml.jackson.databind.annotation.JsonDeserialize; import com.fasterxml.jackson.databind.annotation.JsonSerialize; import com.google.auto.service.AutoService; +import com.google.common.base.MoreObjects; import java.lang.reflect.Proxy; import java.util.ServiceLoader; +import java.util.concurrent.ThreadLocalRandom; import javax.annotation.concurrent.ThreadSafe; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.options.GoogleApiDebugOptions.GoogleApiTracer; @@ -33,6 +35,10 @@ import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.DoFn.Context; import org.apache.beam.sdk.transforms.display.HasDisplayData; +import org.joda.time.DateTimeUtils; +import org.joda.time.DateTimeZone; +import org.joda.time.format.DateTimeFormat; +import org.joda.time.format.DateTimeFormatter; /** * PipelineOptions are used to configure Pipelines. You can extend {@link PipelineOptions} @@ -258,6 +264,13 @@ public static enum CheckEnabled { String getTempLocation(); void setTempLocation(String value); + @Description("Name of the pipeline execution." + + "It must match the regular expression '[a-z]([-a-z0-9]{0,38}[a-z0-9])?'." + + "It defaults to ApplicationName-UserName-Date-RandomInteger") + @Default.InstanceFactory(JobNameFactory.class) + String getJobName(); + void setJobName(String numWorkers); + /** * A {@link DefaultValueFactory} that obtains the class of the {@code DirectRunner} if it exists * on the classpath, and throws an exception otherwise. @@ -284,4 +297,33 @@ public Class create(PipelineOptions options) { } } } + + /** + * Returns a normalized job name constructed from {@link ApplicationNameOptions#getAppName()}, + * the local system user name (if available), the current time, and a random integer. + * + *

        The normalization makes sure that the name matches the pattern of + * [a-z]([-a-z0-9]*[a-z0-9])?. + */ + static class JobNameFactory implements DefaultValueFactory { + private static final DateTimeFormatter FORMATTER = + DateTimeFormat.forPattern("MMddHHmmss").withZone(DateTimeZone.UTC); + + @Override + public String create(PipelineOptions options) { + String appName = options.as(ApplicationNameOptions.class).getAppName(); + String normalizedAppName = appName == null || appName.length() == 0 ? "BeamApp" + : appName.toLowerCase() + .replaceAll("[^a-z0-9]", "0") + .replaceAll("^[^a-z]", "a"); + String userName = MoreObjects.firstNonNull(System.getProperty("user.name"), ""); + String normalizedUserName = userName.toLowerCase() + .replaceAll("[^a-z0-9]", "0"); + String datePart = FORMATTER.print(DateTimeUtils.currentTimeMillis()); + + String randomPart = Integer.toHexString(ThreadLocalRandom.current().nextInt()); + return String.format("%s-%s-%s-%s", + normalizedAppName, normalizedUserName, datePart, randomPart); + } + } } From f05fbe732068f74192853c4cdeb268e848915e66 Mon Sep 17 00:00:00 2001 From: Scott Wegner Date: Wed, 17 Aug 2016 14:38:36 -0700 Subject: [PATCH 071/346] Update DoFn javadocs to remove references to OldDoFn and Dataflow --- .../examples/common/PubsubFileInjector.java | 2 +- .../apache/beam/sdk/util/DoFnRunnerBase.java | 16 +- .../beam/sdk/util/GroupAlsoByWindowsDoFn.java | 2 +- .../apache/beam/sdk/util/ReduceFnRunner.java | 5 +- .../beam/sdk/util/SimpleDoFnRunner.java | 4 +- .../ImmutabilityCheckingBundleFactory.java | 4 +- .../direct/TransformEvaluatorFactory.java | 3 +- .../beam/runners/dataflow/util/DoFnInfo.java | 7 +- .../translation/MultiOutputWordCountTest.java | 2 +- .../spark/translation/SerializationTest.java | 4 +- .../org/apache/beam/sdk/AggregatorValues.java | 4 +- .../beam/sdk/transforms/Aggregator.java | 14 +- .../beam/sdk/transforms/CombineFns.java | 18 +- .../org/apache/beam/sdk/transforms/DoFn.java | 23 +- .../beam/sdk/transforms/DoFnTester.java | 62 ++-- .../beam/sdk/transforms/GroupByKey.java | 7 +- .../beam/sdk/transforms/PTransform.java | 2 +- .../org/apache/beam/sdk/transforms/ParDo.java | 306 +++++++++--------- .../beam/sdk/transforms/SimpleFunction.java | 6 +- .../sdk/transforms/windowing/PaneInfo.java | 10 +- .../beam/sdk/util/BaseExecutionContext.java | 4 +- .../util/ReifyTimestampAndWindowsDoFn.java | 4 +- .../beam/sdk/util/SerializableUtils.java | 2 +- .../beam/sdk/util/SystemDoFnInternal.java | 7 +- .../beam/sdk/util/WindowingInternals.java | 3 +- .../DoFnDelegatingAggregatorTest.java | 2 +- .../beam/sdk/transforms/DoFnTesterTest.java | 3 +- .../beam/sdk/transforms/NoOpOldDoFn.java | 2 +- 28 files changed, 263 insertions(+), 265 deletions(-) diff --git a/examples/java/src/main/java/org/apache/beam/examples/common/PubsubFileInjector.java b/examples/java/src/main/java/org/apache/beam/examples/common/PubsubFileInjector.java index e6a1495e545d5..4634159826d35 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/common/PubsubFileInjector.java +++ b/examples/java/src/main/java/org/apache/beam/examples/common/PubsubFileInjector.java @@ -69,7 +69,7 @@ public Bound publish(String outputTopic) { } } - /** A OldDoFn that publishes non-empty lines to Google Cloud PubSub. */ + /** A {@link OldDoFn} that publishes non-empty lines to Google Cloud PubSub. */ public static class Bound extends OldDoFn { private final String outputTopic; private final String timestampLabelKey; diff --git a/runners/core-java/src/main/java/org/apache/beam/sdk/util/DoFnRunnerBase.java b/runners/core-java/src/main/java/org/apache/beam/sdk/util/DoFnRunnerBase.java index 8a0f6bf868d91..04a0978b60301 100644 --- a/runners/core-java/src/main/java/org/apache/beam/sdk/util/DoFnRunnerBase.java +++ b/runners/core-java/src/main/java/org/apache/beam/sdk/util/DoFnRunnerBase.java @@ -58,10 +58,10 @@ */ public abstract class DoFnRunnerBase implements DoFnRunner { - /** The OldDoFn being run. */ + /** The {@link OldDoFn} being run. */ public final OldDoFn fn; - /** The context used for running the OldDoFn. */ + /** The context used for running the {@link OldDoFn}. */ public final DoFnContext context; protected DoFnRunnerBase( @@ -164,8 +164,8 @@ public void finishBundle() { /** * A concrete implementation of {@code OldDoFn.Context} used for running a {@link OldDoFn}. * - * @param the type of the OldDoFn's (main) input elements - * @param the type of the OldDoFn's (main) output elements + * @param the type of the {@link OldDoFn} (main) input elements + * @param the type of the {@link OldDoFn} (main) output elements */ private static class DoFnContext extends OldDoFn.Context { @@ -350,7 +350,7 @@ protected Aggregator createAggreg } /** - * Returns a new {@code OldDoFn.ProcessContext} for the given element. + * Returns a new {@link OldDoFn.ProcessContext} for the given element. */ protected OldDoFn.ProcessContext createProcessContext( WindowedValue elem) { @@ -366,11 +366,11 @@ private boolean isSystemDoFn() { } /** - * A concrete implementation of {@code OldDoFn.ProcessContext} used for + * A concrete implementation of {@link OldDoFn.ProcessContext} used for * running a {@link OldDoFn} over a single element. * - * @param the type of the OldDoFn's (main) input elements - * @param the type of the OldDoFn's (main) output elements + * @param the type of the {@link OldDoFn} (main) input elements + * @param the type of the {@link OldDoFn} (main) output elements */ static class DoFnProcessContext extends OldDoFn.ProcessContext { diff --git a/runners/core-java/src/main/java/org/apache/beam/sdk/util/GroupAlsoByWindowsDoFn.java b/runners/core-java/src/main/java/org/apache/beam/sdk/util/GroupAlsoByWindowsDoFn.java index f82e5dfe32e7b..f386dfba12201 100644 --- a/runners/core-java/src/main/java/org/apache/beam/sdk/util/GroupAlsoByWindowsDoFn.java +++ b/runners/core-java/src/main/java/org/apache/beam/sdk/util/GroupAlsoByWindowsDoFn.java @@ -26,7 +26,7 @@ import org.apache.beam.sdk.values.KV; /** - * OldDoFn that merges windows and groups elements in those windows, optionally + * {@link OldDoFn} that merges windows and groups elements in those windows, optionally * combining values. * * @param key type diff --git a/runners/core-java/src/main/java/org/apache/beam/sdk/util/ReduceFnRunner.java b/runners/core-java/src/main/java/org/apache/beam/sdk/util/ReduceFnRunner.java index 61e5b21ebfd93..7c3e4d749a1f3 100644 --- a/runners/core-java/src/main/java/org/apache/beam/sdk/util/ReduceFnRunner.java +++ b/runners/core-java/src/main/java/org/apache/beam/sdk/util/ReduceFnRunner.java @@ -33,7 +33,6 @@ import javax.annotation.Nullable; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.Aggregator; -import org.apache.beam.sdk.transforms.OldDoFn; import org.apache.beam.sdk.transforms.windowing.AfterWatermark; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.GlobalWindow; @@ -177,8 +176,8 @@ public class ReduceFnRunner { * Store the previously emitted pane (if any) for each window. * *

          - *
        • State: The previous {@link PaneInfo} passed to the user's {@link OldDoFn#processElement}, - * if any. + *
        • State: The previous {@link PaneInfo} passed to the user's {@code DoFn.ProcessElement} + * method, if any. *
        • Style style: DIRECT *
        • Merging: Always keyed by actual window, so does not depend on {@link #activeWindows}. * Cleared when window is merged away. diff --git a/runners/core-java/src/main/java/org/apache/beam/sdk/util/SimpleDoFnRunner.java b/runners/core-java/src/main/java/org/apache/beam/sdk/util/SimpleDoFnRunner.java index 6c1cf451d61a6..1ebe5a874e74a 100644 --- a/runners/core-java/src/main/java/org/apache/beam/sdk/util/SimpleDoFnRunner.java +++ b/runners/core-java/src/main/java/org/apache/beam/sdk/util/SimpleDoFnRunner.java @@ -28,8 +28,8 @@ /** * Runs a {@link OldDoFn} by constructing the appropriate contexts and passing them in. * - * @param the type of the OldDoFn's (main) input elements - * @param the type of the OldDoFn's (main) output elements + * @param the type of the {@link OldDoFn} (main) input elements + * @param the type of the {@link OldDoFn} (main) output elements */ public class SimpleDoFnRunner extends DoFnRunnerBase{ 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 d5c0f0c354e60..71bd8b4e50053 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 @@ -25,7 +25,7 @@ import org.apache.beam.runners.direct.DirectRunner.UncommittedBundle; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.CoderException; -import org.apache.beam.sdk.transforms.OldDoFn; +import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.util.IllegalMutationException; import org.apache.beam.sdk.util.MutationDetector; import org.apache.beam.sdk.util.MutationDetectors; @@ -40,7 +40,7 @@ * elements added to the bundle will be encoded by the {@link Coder} of the underlying * {@link PCollection}. * - *

          This catches errors during the execution of a {@link OldDoFn} caused by modifying an element + *

          This catches errors during the execution of a {@link DoFn} caused by modifying an element * after it is added to an output {@link PCollection}. */ class ImmutabilityCheckingBundleFactory implements BundleFactory { diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TransformEvaluatorFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TransformEvaluatorFactory.java index e9fa06b107f70..ecf2da8f67a85 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TransformEvaluatorFactory.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TransformEvaluatorFactory.java @@ -22,7 +22,6 @@ import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.io.Read; import org.apache.beam.sdk.transforms.AppliedPTransform; -import org.apache.beam.sdk.transforms.OldDoFn; import org.apache.beam.sdk.transforms.PTransform; /** @@ -37,7 +36,7 @@ public interface TransformEvaluatorFactory { * Create a new {@link TransformEvaluator} for the application of the {@link PTransform}. * *

          Any work that must be done before input elements are processed (such as calling - * {@link OldDoFn#startBundle(OldDoFn.Context)}) must be done before the + * {@code DoFn.StartBundle}) must be done before the * {@link TransformEvaluator} is made available to the caller. * *

          May return null if the application cannot produce an evaluator (for example, it is a diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/DoFnInfo.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/DoFnInfo.java index 139db9da0ab98..949c381489936 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/DoFnInfo.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/DoFnInfo.java @@ -24,10 +24,10 @@ import org.apache.beam.sdk.values.PCollectionView; /** - * Wrapper class holding the necessary information to serialize a OldDoFn. + * Wrapper class holding the necessary information to serialize a {@link OldDoFn}. * - * @param the type of the (main) input elements of the OldDoFn - * @param the type of the (main) output elements of the OldDoFn + * @param the type of the (main) input elements of the {@link OldDoFn} + * @param the type of the (main) output elements of the {@link OldDoFn} */ public class DoFnInfo implements Serializable { private final OldDoFn doFn; @@ -66,3 +66,4 @@ public Coder getInputCoder() { return inputCoder; } } + diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/MultiOutputWordCountTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/MultiOutputWordCountTest.java index 517596aab7822..acfa3dfae2b80 100644 --- a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/MultiOutputWordCountTest.java +++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/MultiOutputWordCountTest.java @@ -100,7 +100,7 @@ public void testRun() throws Exception { } /** - * A OldDoFn that tokenizes lines of text into individual words. + * A {@link DoFn} that tokenizes lines of text into individual words. */ static class ExtractWordsFn extends DoFn { diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/SerializationTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/SerializationTest.java index 0e9121ca87f97..22a40cd29e2ca 100644 --- a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/SerializationTest.java +++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/SerializationTest.java @@ -142,7 +142,7 @@ public void testRun() throws Exception { } /** - * A OldDoFn that tokenizes lines of text into individual words. + * A {@link DoFn} that tokenizes lines of text into individual words. */ static class ExtractWordsFn extends DoFn { private static final Pattern WORD_BOUNDARY = Pattern.compile("[^a-zA-Z']+"); @@ -170,7 +170,7 @@ public void processElement(ProcessContext c) { } /** - * A OldDoFn that converts a Word and Count into a printable string. + * A {@link DoFn} that converts a Word and Count into a printable string. */ private static class FormatCountsFn extends DoFn, StringHolder> { @ProcessElement diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/AggregatorValues.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/AggregatorValues.java index 6297085319e2a..1fd034a02cc01 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/AggregatorValues.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/AggregatorValues.java @@ -21,11 +21,11 @@ import java.util.Map; import org.apache.beam.sdk.transforms.Aggregator; import org.apache.beam.sdk.transforms.Combine.CombineFn; -import org.apache.beam.sdk.transforms.OldDoFn; +import org.apache.beam.sdk.transforms.DoFn; /** * A collection of values associated with an {@link Aggregator}. Aggregators declared in a - * {@link OldDoFn} are emitted on a per-{@code OldDoFn}-application basis. + * {@link DoFn} are emitted on a per-{@link DoFn}-application basis. * * @param the output type of the aggregator */ 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 db4ab33ff1b0e..67d399fbc8207 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 @@ -25,8 +25,8 @@ * to be combined across all bundles. * *

          Aggregators are created by calling - * {@link OldDoFn#createAggregator OldDoFn.createAggregatorForDoFn}, - * typically from the {@link OldDoFn} constructor. Elements can be added to the + * {@link DoFn#createAggregator DoFn.createAggregatorForDoFn}, + * typically from the {@link DoFn} constructor. Elements can be added to the * {@code Aggregator} by calling {@link Aggregator#addValue}. * *

          Aggregators are visible in the monitoring UI, when the pipeline is run @@ -37,14 +37,14 @@ * *

          Example: *

           {@code
          - * class MyDoFn extends OldDoFn {
          + * class MyDoFn extends DoFn {
            *   private Aggregator myAggregator;
            *
            *   public MyDoFn() {
            *     myAggregator = createAggregatorForDoFn("myAggregator", new Sum.SumIntegerFn());
            *   }
            *
          - *   @Override
          + *   @ProcessElement
            *   public void processElement(ProcessContext c) {
            *     myAggregator.addValue(1);
            *   }
          @@ -79,8 +79,8 @@ interface AggregatorFactory {
               /**
                * Create an aggregator with the given {@code name} and {@link CombineFn}.
                *
          -     *  

          This method is called to create an aggregator for a {@link OldDoFn}. It receives the - * class of the {@link OldDoFn} being executed and the context of the step it is being + *

          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( @@ -90,7 +90,7 @@ Aggregator createAggregatorForDoFn( // TODO: Consider the following additional API conveniences: // - In addition to createAggregatorForDoFn(), consider adding getAggregator() to - // avoid the need to store the aggregator locally in a OldDoFn, i.e., create + // avoid the need to store the aggregator locally in a DoFn, i.e., create // if not already present. // - Add a shortcut for the most common aggregator: // c.createAggregatorForDoFn("name", new Sum.SumIntegerFn()). diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/CombineFns.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/CombineFns.java index 9fa8ded6b2409..6f059932f5f47 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/CombineFns.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/CombineFns.java @@ -67,7 +67,7 @@ public class CombineFns { *

          The same {@link TupleTag} cannot be used in a composition multiple times. * *

          Example: - *

          { @code
          +   * 
          
              * PCollection> latencies = ...;
              *
              * TupleTag maxLatencyTag = new TupleTag();
          @@ -75,7 +75,7 @@ public class CombineFns {
              *
              * SimpleFunction identityFn =
              *     new SimpleFunction() {
          -   *       @Override
          +   *      {@literal @}Override
              *       public Integer apply(Integer input) {
              *           return input;
              *       }};
          @@ -87,8 +87,8 @@ public class CombineFns {
              *
              * PCollection finalResultCollection = maxAndMean
              *     .apply(ParDo.of(
          -   *         new OldDoFn, T>() {
          -   *           @Override
          +   *         new DoFn, T>() {
          +   *          {@literal @}ProcessElement
              *           public void processElement(ProcessContext c) throws Exception {
              *             KV e = c.element();
              *             Integer maxLatency = e.getValue().get(maxLatencyTag);
          @@ -97,7 +97,7 @@ public class CombineFns {
              *             c.output(...some T...);
              *           }
              *         }));
          -   * } 
          + *
          */ public static ComposeKeyedCombineFnBuilder composeKeyed() { return new ComposeKeyedCombineFnBuilder(); @@ -110,7 +110,7 @@ public static ComposeKeyedCombineFnBuilder composeKeyed() { *

          The same {@link TupleTag} cannot be used in a composition multiple times. * *

          Example: - *

          { @code
          +   * 
          
              * PCollection globalLatencies = ...;
              *
              * TupleTag maxLatencyTag = new TupleTag();
          @@ -130,8 +130,8 @@ public static ComposeKeyedCombineFnBuilder composeKeyed() {
              *
              * PCollection finalResultCollection = maxAndMean
              *     .apply(ParDo.of(
          -   *         new OldDoFn() {
          -   *           @Override
          +   *         new DoFn() {
          +   *          {@literal @}ProcessElement
              *           public void processElement(ProcessContext c) throws Exception {
              *             CoCombineResult e = c.element();
              *             Integer maxLatency = e.get(maxLatencyTag);
          @@ -140,7 +140,7 @@ public static ComposeKeyedCombineFnBuilder composeKeyed() {
              *             c.output(...some T...);
              *           }
              *         }));
          -   * } 
          + *
          */ public static ComposeCombineFnBuilder compose() { return new ComposeCombineFnBuilder(); 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 9f898261c7bb3..59c832323686f 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 @@ -63,8 +63,6 @@ * that satisfies the requirements described there. See the {@link ProcessElement} * for details. * - *

          This functionality is experimental and likely to change. - * *

          Example usage: * *

           {@code
          @@ -123,7 +121,7 @@ public abstract class Context {
                *
                * 

          If invoked from {@link ProcessElement}), the timestamp * must not be older than the input element's timestamp minus - * {@link OldDoFn#getAllowedTimestampSkew}. The output element will + * {@link DoFn#getAllowedTimestampSkew}. The output element will * be in the same windows as the input element. * *

          If invoked from {@link StartBundle} or {@link FinishBundle}, @@ -172,7 +170,7 @@ public abstract class Context { * *

          If invoked from {@link ProcessElement}), the timestamp * must not be older than the input element's timestamp minus - * {@link OldDoFn#getAllowedTimestampSkew}. The output element will + * {@link DoFn#getAllowedTimestampSkew}. The output element will * be in the same windows as the input element. * *

          If invoked from {@link StartBundle} or {@link FinishBundle}, @@ -190,7 +188,7 @@ public abstract void sideOutputWithTimestamp( } /** - * Information accessible when running {@link OldDoFn#processElement}. + * Information accessible when running a {@link DoFn.ProcessElement} method. */ public abstract class ProcessContext extends Context { @@ -359,9 +357,14 @@ public OutputReceiver outputReceiver() { * Annotation for the method to use to prepare an instance for processing a batch of elements. * The method annotated with this must satisfy the following constraints: *

            - *
          • It must have at least one argument. + *
          • It must have exactly one argument. *
          • Its first (and only) argument must be a {@link DoFn.Context}. *
          + * + *

          A simple method declaration would look like: + * + * public void setup(DoFn.Context c) { .. } + * */ @Documented @Retention(RetentionPolicy.RUNTIME) @@ -414,13 +417,13 @@ public OutputReceiver outputReceiver() { /** * Returns an {@link Aggregator} with aggregation logic specified by the * {@link CombineFn} argument. The name provided must be unique across - * {@link Aggregator}s created within the OldDoFn. Aggregators can only be created + * {@link Aggregator}s created within the {@link DoFn}. Aggregators can only be created * during pipeline construction. * * @param name the name of the aggregator * @param combiner the {@link CombineFn} to use in the aggregator * @return an aggregator for the provided name and combiner in the scope of - * this OldDoFn + * this {@link DoFn} * @throws NullPointerException if the name or combiner is null * @throws IllegalArgumentException if the given name collides with another * aggregator in this scope @@ -447,13 +450,13 @@ public OutputReceiver outputReceiver() { /** * Returns an {@link Aggregator} with the aggregation logic specified by the * {@link SerializableFunction} argument. The name provided must be unique - * across {@link Aggregator}s created within the OldDoFn. Aggregators can only be + * across {@link Aggregator}s created within the {@link DoFn}. Aggregators can only be * created during pipeline construction. * * @param name the name of the aggregator * @param combiner the {@link SerializableFunction} to use in the aggregator * @return an aggregator for the provided name and combiner in the scope of - * this OldDoFn + * this {@link DoFn} * @throws NullPointerException if the name or combiner is null * @throws IllegalArgumentException if the given name collides with another * aggregator in this scope 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 82c12938fa684..6801768c10967 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 @@ -50,12 +50,12 @@ import org.joda.time.Instant; /** - * A harness for unit-testing a {@link OldDoFn}. + * A harness for unit-testing a {@link DoFn}. * *

          For example: * *

           {@code
          - * OldDoFn fn = ...;
          + * DoFn fn = ...;
            *
            * DoFnTester fnTester = DoFnTester.of(fn);
            *
          @@ -72,17 +72,17 @@
            * Assert.assertThat(fnTester.processBundle(i1, i2, ...), Matchers.hasItems(...));
            * } 
          * - * @param the type of the {@code OldDoFn}'s (main) input elements - * @param the type of the {@code OldDoFn}'s (main) output elements + * @param the type of the {@link DoFn}'s (main) input elements + * @param the type of the {@link DoFn}'s (main) output elements */ public class DoFnTester { /** * Returns a {@code DoFnTester} supporting unit-testing of the given - * {@link OldDoFn}. + * {@link DoFn}. */ @SuppressWarnings("unchecked") - public static DoFnTester of(OldDoFn fn) { - return new DoFnTester(fn); + public static DoFnTester of(DoFn fn) { + return new DoFnTester<>(DoFnAdapters.toOldDoFn(fn)); } /** @@ -90,19 +90,19 @@ public static DoFnTester of(OldDoFn DoFnTester - of(DoFn fn) { - return new DoFnTester(DoFnAdapters.toOldDoFn(fn)); + public static DoFnTester + of(OldDoFn fn) { + return new DoFnTester<>(fn); } /** * Registers the tuple of values of the side input {@link PCollectionView}s to - * pass to the {@link OldDoFn} under test. + * pass to the {@link DoFn} under test. * *

          Resets the state of this {@link DoFnTester}. * *

          If this isn't called, {@code DoFnTester} assumes the - * {@link OldDoFn} takes no side inputs. + * {@link DoFn} takes no side inputs. */ public void setSideInputs(Map, Map> sideInputs) { this.sideInputs = sideInputs; @@ -110,7 +110,7 @@ public void setSideInputs(Map, Map> sideInp } /** - * Registers the values of a side input {@link PCollectionView} to pass to the {@link OldDoFn} + * Registers the values of a side input {@link PCollectionView} to pass to the {@link DoFn} * under test. * *

          The provided value is the final value of the side input in the specified window, not @@ -129,7 +129,7 @@ public void setSideInput(PCollectionView sideInput, BoundedWindow window, } /** - * Whether or not a {@link DoFnTester} should clone the {@link OldDoFn} under test. + * Whether or not a {@link DoFnTester} should clone the {@link DoFn} under test. */ public enum CloningBehavior { CLONE, @@ -137,14 +137,14 @@ public enum CloningBehavior { } /** - * Instruct this {@link DoFnTester} whether or not to clone the {@link OldDoFn} under test. + * Instruct this {@link DoFnTester} whether or not to clone the {@link DoFn} under test. */ public void setCloningBehavior(CloningBehavior newValue) { this.cloningBehavior = newValue; } /** - * Indicates whether this {@link DoFnTester} will clone the {@link OldDoFn} under test. + * Indicates whether this {@link DoFnTester} will clone the {@link DoFn} under test. */ public CloningBehavior getCloningBehavior() { return cloningBehavior; @@ -166,7 +166,7 @@ public List processBundle(Iterable inputElements) th } /** - * A convenience method for testing {@link OldDoFn DoFns} with bundles of elements. + * A convenience method for testing {@link DoFn DoFns} with bundles of elements. * Logic proceeds as follows: * *

            @@ -182,9 +182,9 @@ public final List processBundle(InputT... inputElements) throws Excepti } /** - * Calls {@link OldDoFn#startBundle} on the {@code OldDoFn} under test. + * Calls the {@link DoFn.StartBundle} method on the {@link DoFn} under test. * - *

            If needed, first creates a fresh instance of the OldDoFn under test. + *

            If needed, first creates a fresh instance of the {@link DoFn} under test. */ public void startBundle() throws Exception { resetState(); @@ -210,14 +210,14 @@ private static void unwrapUserCodeException(UserCodeException e) throws Exceptio } /** - * Calls {@link OldDoFn#processElement} on the {@code OldDoFn} under test, in a - * context where {@link OldDoFn.ProcessContext#element} returns the + * Calls the {@link DoFn.ProcessElement} method on the {@link DoFn} under test, in a + * context where {@link DoFn.ProcessContext#element} returns the * given element. * *

            Will call {@link #startBundle} automatically, if it hasn't * already been called. * - * @throws IllegalStateException if the {@code OldDoFn} under test has already + * @throws IllegalStateException if the {@code DoFn} under test has already * been finished */ public void processElement(InputT element) throws Exception { @@ -235,12 +235,12 @@ public void processElement(InputT element) throws Exception { } /** - * Calls {@link OldDoFn#finishBundle} of the {@code OldDoFn} under test. + * Calls the {@link DoFn.FinishBundle} method of the {@link DoFn} under test. * *

            Will call {@link #startBundle} automatically, if it hasn't * already been called. * - * @throws IllegalStateException if the {@code OldDoFn} under test has already + * @throws IllegalStateException if the {@link DoFn} under test has already * been finished */ public void finishBundle() throws Exception { @@ -674,7 +674,7 @@ protected Aggregator createAggreg ///////////////////////////////////////////////////////////////////////////// - /** The possible states of processing a OldDoFn. */ + /** The possible states of processing a {@link DoFn}. */ enum State { UNSTARTED, STARTED, @@ -683,23 +683,23 @@ enum State { private final PipelineOptions options = PipelineOptionsFactory.create(); - /** The original OldDoFn under test. */ + /** The original {@link OldDoFn} under test. */ private final OldDoFn origFn; /** - * Whether to clone the original {@link OldDoFn} or just use it as-is. + * Whether to clone the original {@link DoFn} or just use it as-is. * - *

            Worker-side {@link OldDoFn DoFns} may not be serializable, and are not required to be. + *

            Worker-side {@link DoFn DoFns} may not be serializable, and are not required to be. */ private CloningBehavior cloningBehavior = CloningBehavior.CLONE; - /** The side input values to provide to the OldDoFn under test. */ + /** The side input values to provide to the {@link DoFn} under test. */ private Map, Map> sideInputs = new HashMap<>(); private Map accumulators; - /** The output tags used by the OldDoFn under test. */ + /** The output tags used by the {@link DoFn} under test. */ private TupleTag mainOutputTag = new TupleTag<>(); /** The original OldDoFn under test, if started. */ @@ -708,7 +708,7 @@ enum State { /** The ListOutputManager to examine the outputs. */ private Map, List>> outputs; - /** The state of processing of the OldDoFn under test. */ + /** The state of processing of the {@link DoFn} under test. */ private State state; private DoFnTester(OldDoFn origFn) { 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 ed7f411aec11c..3a3da65e077d6 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 @@ -63,18 +63,19 @@ * {@code Coder} of the values of the input. * *

            Example of use: - *

             {@code
            + * 
            
              * PCollection> urlDocPairs = ...;
              * PCollection>> urlToDocs =
              *     urlDocPairs.apply(GroupByKey.create());
              * PCollection results =
            - *     urlToDocs.apply(ParDo.of(new OldDoFn>, R>() {
            + *     urlToDocs.apply(ParDo.of(new DoFn>, R>() {
            + *      {@literal @}ProcessElement
              *       public void processElement(ProcessContext c) {
              *         String url = c.element().getKey();
              *         Iterable docsWithThatUrl = c.element().getValue();
              *         ... process all docs having that url ...
              *       }}));
            - * } 
            + *
            * *

            {@code GroupByKey} is a key primitive in data-parallel * processing, since it is the main way to efficiently bring 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 19abef90cea1b..4a58141ab503f 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 @@ -146,7 +146,7 @@ * implementing {@code Serializable}. * *

            {@code PTransform} is marked {@code Serializable} solely - * because it is common for an anonymous {@code OldDoFn}, + * because it is common for an anonymous {@link DoFn}, * instance to be created within an * {@code apply()} method of a composite {@code PTransform}. * 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 5efbe9f190240..f9cb557c1e5df 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 @@ -22,7 +22,6 @@ import java.util.Arrays; import java.util.List; import org.apache.beam.sdk.Pipeline; -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.runners.PipelineRunner; @@ -50,13 +49,12 @@ *

            The {@link ParDo} processing style is similar to what happens inside * the "Mapper" or "Reducer" class of a MapReduce-style algorithm. * - *

            {@link OldDoFn DoFns}

            + *

            {@link DoFn DoFns}

            * *

            The function to use to process each element is specified by a - * {@link OldDoFn OldDoFn<InputT, OutputT>}, primarily via its - * {@link OldDoFn#processElement processElement} method. The {@link OldDoFn} may also - * override the default implementations of {@link OldDoFn#startBundle startBundle} - * and {@link OldDoFn#finishBundle finishBundle}. + * {@link DoFn DoFn<InputT, OutputT>}, primarily via its + * {@link DoFn.ProcessElement ProcessElement} method. The {@link DoFn} may also + * provide a {@link DoFn.StartBundle StartBundle} and {@link DoFn.FinishBundle finishBundle} method. * *

            Conceptually, when a {@link ParDo} transform is executed, the * elements of the input {@link PCollection} are first divided up @@ -66,39 +64,38 @@ * *

              *
            1. If required, a fresh instance of the argument {@link DoFn} is created - * on a worker, and {@link DoFn#setup()} is called on this instance. This may be through - * deserialization or other means. A {@link PipelineRunner} may reuse {@link DoFn} instances for - * multiple bundles. A {@link DoFn} that has terminated abnormally (by throwing an + * on a worker, and the {@link DoFn.Setup} method is called on this instance. This may be + * through deserialization or other means. A {@link PipelineRunner} may reuse {@link DoFn} + * instances for multiple bundles. A {@link DoFn} that has terminated abnormally (by throwing an * {@link Exception}) will never be reused.
            2. - *
            3. The {@link OldDoFn OldDoFn's} {@link OldDoFn#startBundle} method is called to - * initialize it. If this method is not overridden, the call may be optimized - * away.
            4. - *
            5. The {@link OldDoFn OldDoFn's} {@link OldDoFn#processElement} method + *
            6. The {@link DoFn DoFn's} {@link DoFn.StartBundle} method, if provided, is called to + * initialize it.
            7. + *
            8. The {@link DoFn DoFn's} {@link DoFn.ProcessElement} method * is called on each of the input elements in the bundle.
            9. - *
            10. The {@link OldDoFn OldDoFn's} {@link OldDoFn#finishBundle} method is called - * to complete its work. After {@link OldDoFn#finishBundle} is called, the - * framework will not again invoke {@link OldDoFn#processElement} or - * {@link OldDoFn#finishBundle} - * until a new call to {@link OldDoFn#startBundle} has occurred. - * If this method is not overridden, this call may be optimized away.
            11. - *
            12. If any of {@link DoFn#setup}, {@link DoFn#startBundle}, {@link DoFn#processElement} or - * {@link DoFn#finishBundle} throw an exception, {@link DoFn#teardown} will be called on the - * {@link DoFn} instance.
            13. - *
            14. If a runner will no longer use a {@link DoFn}, {@link DoFn#teardown()} will be called on - * the discarded instance.
            15. + *
            16. The {@link DoFn DoFn's} {@link DoFn.FinishBundle} method, if provided, is called + * to complete its work. After {@link DoFn.FinishBundle} is called, the + * framework will not again invoke {@link DoFn.ProcessElement} or + * {@link DoFn.FinishBundle} + * until a new call to {@link DoFn.StartBundle} has occurred.
            17. + *
            18. If any of {@link DoFn.Setup}, {@link DoFn.StartBundle}, {@link DoFn.ProcessElement} or + * {@link DoFn.FinishBundle} methods throw an exception, the {@link DoFn.Teardown} method, if + * provided, will be called on the {@link DoFn} instance.
            19. + *
            20. If a runner will no longer use a {@link DoFn}, the {@link DoFn.Teardown} method, if + * provided, will be called on the discarded instance.
            21. *
            * - * Each of the calls to any of the {@link OldDoFn OldDoFn's} processing + * Each of the calls to any of the {@link DoFn DoFn's} processing * methods can produce zero or more output elements. All of the - * of output elements from all of the {@link OldDoFn} instances + * of output elements from all of the {@link DoFn} instances * are included in the output {@link PCollection}. * *

            For example: * - *

             {@code
            + * 
            
              * PCollection lines = ...;
              * PCollection words =
            - *     lines.apply(ParDo.of(new OldDoFn() {
            + *     lines.apply(ParDo.of(new DoFn() {
            + *        {@literal @}ProcessElement
              *         public void processElement(ProcessContext c) {
              *           String line = c.element();
              *           for (String word : line.split("[^a-zA-Z']+")) {
            @@ -106,13 +103,14 @@
              *           }
              *         }}));
              * PCollection wordLengths =
            - *     words.apply(ParDo.of(new OldDoFn() {
            + *     words.apply(ParDo.of(new DoFn() {
            + *        {@literal @}ProcessElement
              *         public void processElement(ProcessContext c) {
              *           String word = c.element();
              *           Integer length = word.length();
              *           c.output(length);
              *         }}));
            - * } 
            + *
            * *

            Each output element has the same timestamp and is in the same windows * as its corresponding input element, and the output {@code PCollection} @@ -131,9 +129,9 @@ * *

             {@code
              * PCollection words =
            - *     lines.apply("ExtractWords", ParDo.of(new OldDoFn() { ... }));
            + *     lines.apply("ExtractWords", ParDo.of(new DoFn() { ... }));
              * PCollection wordLengths =
            - *     words.apply("ComputeWordLengths", ParDo.of(new OldDoFn() { ... }));
            + *     words.apply("ComputeWordLengths", ParDo.of(new DoFn() { ... }));
              * } 
            * *

            Side Inputs

            @@ -145,17 +143,18 @@ * {@link PCollection PCollections} computed by earlier pipeline operations, * passed in to the {@link ParDo} transform using * {@link #withSideInputs}, and their contents accessible to each of - * the {@link OldDoFn} operations via {@link OldDoFn.ProcessContext#sideInput sideInput}. + * the {@link DoFn} operations via {@link DoFn.ProcessContext#sideInput sideInput}. * For example: * - *
             {@code
            + * 
            
              * PCollection words = ...;
              * PCollection maxWordLengthCutOff = ...; // Singleton PCollection
              * final PCollectionView maxWordLengthCutOffView =
              *     maxWordLengthCutOff.apply(View.asSingleton());
              * PCollection wordsBelowCutOff =
              *     words.apply(ParDo.withSideInputs(maxWordLengthCutOffView)
            - *                      .of(new OldDoFn() {
            + *                      .of(new DoFn() {
            + *        {@literal @}ProcessElement
              *         public void processElement(ProcessContext c) {
              *           String word = c.element();
              *           int lengthCutOff = c.sideInput(maxWordLengthCutOffView);
            @@ -163,7 +162,7 @@
              *             c.output(word);
              *           }
              *         }}));
            - * } 
            + *
            * *

            Side Outputs

            * @@ -174,13 +173,13 @@ * and bundled in a {@link PCollectionTuple}. The {@link TupleTag TupleTags} * to be used for the output {@link PCollectionTuple} are specified by * invoking {@link #withOutputTags}. Unconsumed side outputs do not - * necessarily need to be explicitly specified, even if the {@link OldDoFn} - * generates them. Within the {@link OldDoFn}, an element is added to the + * necessarily need to be explicitly specified, even if the {@link DoFn} + * generates them. Within the {@link DoFn}, an element is added to the * main output {@link PCollection} as normal, using - * {@link OldDoFn.Context#output}, while an element is added to a side output - * {@link PCollection} using {@link OldDoFn.Context#sideOutput}. For example: + * {@link DoFn.Context#output}, while an element is added to a side output + * {@link PCollection} using {@link DoFn.Context#sideOutput}. For example: * - *
             {@code
            + * 
            
              * PCollection words = ...;
              * // Select words whose length is below a cut off,
              * // plus the lengths of words that are above the cut off.
            @@ -201,10 +200,11 @@
              *         .withOutputTags(wordsBelowCutOffTag,
              *                         TupleTagList.of(wordLengthsAboveCutOffTag)
              *                                     .and(markedWordsTag))
            - *         .of(new OldDoFn() {
            + *         .of(new DoFn() {
              *             // Create a tag for the unconsumed side output.
              *             final TupleTag specialWordsTag =
              *                 new TupleTag(){};
            + *            {@literal @}ProcessElement
              *             public void processElement(ProcessContext c) {
              *               String word = c.element();
              *               if (word.length() <= wordLengthCutOff) {
            @@ -230,14 +230,13 @@
              *     results.get(wordLengthsAboveCutOffTag);
              * PCollection markedWords =
              *     results.get(markedWordsTag);
            - * } 
            + *
            * *

            Properties May Be Specified In Any Order

            * *

            Several properties can be specified for a {@link ParDo} - * {@link PTransform}, including name, side inputs, side output tags, - * and {@link OldDoFn} to invoke. Only the {@link OldDoFn} is required; the - * name is encouraged but not required, and side inputs and side + * {@link PTransform}, including side inputs, side output tags, + * and {@link DoFn} to invoke. Only the {@link DoFn} is required; side inputs and side * output tags are only specified when they're needed. These * properties can be specified in any order, as long as they're * specified before the {@link ParDo} {@link PTransform} is applied. @@ -250,23 +249,23 @@ * {@link ParDo.Bound} nested classes, each of which offer * property setter instance methods to enable setting additional * properties. {@link ParDo.Bound} is used for {@link ParDo} - * transforms whose {@link OldDoFn} is specified and whose input and + * transforms whose {@link DoFn} is specified and whose input and * output static types have been bound. {@link ParDo.Unbound ParDo.Unbound} is used * for {@link ParDo} transforms that have not yet had their - * {@link OldDoFn} specified. Only {@link ParDo.Bound} instances can be + * {@link DoFn} specified. Only {@link ParDo.Bound} instances can be * applied. * *

            Another benefit of this approach is that it reduces the number * of type parameters that need to be specified manually. In * particular, the input and output types of the {@link ParDo} * {@link PTransform} are inferred automatically from the type - * parameters of the {@link OldDoFn} argument passed to {@link ParDo#of}. + * parameters of the {@link DoFn} argument passed to {@link ParDo#of}. * *

            Output Coders

            * *

            By default, the {@link Coder Coder<OutputT>} for the * elements of the main output {@link PCollection PCollection<OutputT>} is - * inferred from the concrete type of the {@link OldDoFn OldDoFn<InputT, OutputT>}. + * inferred from the concrete type of the {@link DoFn DoFn<InputT, OutputT>}. * *

            By default, the {@link Coder Coder<SideOutputT>} for the elements of * a side output {@link PCollection PCollection<SideOutputT>} is inferred @@ -286,74 +285,74 @@ * This style of {@code TupleTag} instantiation is used in the example of * multiple side outputs, above. * - *

            Serializability of {@link OldDoFn DoFns}

            + *

            Serializability of {@link DoFn DoFns}

            * - *

            A {@link OldDoFn} passed to a {@link ParDo} transform must be - * {@link Serializable}. This allows the {@link OldDoFn} instance + *

            A {@link DoFn} passed to a {@link ParDo} transform must be + * {@link Serializable}. This allows the {@link DoFn} instance * created in this "main program" to be sent (in serialized form) to * remote worker machines and reconstituted for bundles of elements - * of the input {@link PCollection} being processed. A {@link OldDoFn} + * of the input {@link PCollection} being processed. A {@link DoFn} * can have instance variable state, and non-transient instance * variable state will be serialized in the main program and then * deserialized on remote worker machines for some number of bundles * of elements to process. * - *

            {@link OldDoFn DoFns} expressed as anonymous inner classes can be + *

            {@link DoFn DoFns} expressed as anonymous inner classes can be * convenient, but due to a quirk in Java's rules for serializability, * non-static inner or nested classes (including anonymous inner * classes) automatically capture their enclosing class's instance in * their serialized state. This can lead to including much more than - * intended in the serialized state of a {@link OldDoFn}, or even things + * intended in the serialized state of a {@link DoFn}, or even things * that aren't {@link Serializable}. * *

            There are two ways to avoid unintended serialized state in a - * {@link OldDoFn}: + * {@link DoFn}: * *

              * - *
            • Define the {@link OldDoFn} as a named, static class. + *
            • Define the {@link DoFn} as a named, static class. * - *
            • Define the {@link OldDoFn} as an anonymous inner class inside of + *
            • Define the {@link DoFn} as an anonymous inner class inside of * a static method. * *
            * *

            Both of these approaches ensure that there is no implicit enclosing - * instance serialized along with the {@link OldDoFn} instance. + * instance serialized along with the {@link DoFn} instance. * *

            Prior to Java 8, any local variables of the enclosing * method referenced from within an anonymous inner class need to be - * marked as {@code final}. If defining the {@link OldDoFn} as a named + * marked as {@code final}. If defining the {@link DoFn} as a named * static class, such variables would be passed as explicit * constructor arguments and stored in explicit instance variables. * *

            There are three main ways to initialize the state of a - * {@link OldDoFn} instance processing a bundle: + * {@link DoFn} instance processing a bundle: * *

              * *
            • Define instance variable state (including implicit instance * variables holding final variables captured by an anonymous inner - * class), initialized by the {@link OldDoFn}'s constructor (which is + * class), initialized by the {@link DoFn}'s constructor (which is * implicit for an anonymous inner class). This state will be - * automatically serialized and then deserialized in the {@code OldDoFn} + * automatically serialized and then deserialized in the {@link DoFn} * instances created for bundles. This method is good for state - * known when the original {@code OldDoFn} is created in the main + * known when the original {@link DoFn} is created in the main * program, if it's not overly large. This is not suitable for any - * state which must only be used for a single bundle, as {@link OldDoFn OldDoFn's} + * state which must only be used for a single bundle, as {@link DoFn DoFn's} * may be used to process multiple bundles. * *
            • Compute the state as a singleton {@link PCollection} and pass it - * in as a side input to the {@link OldDoFn}. This is good if the state + * in as a side input to the {@link DoFn}. This is good if the state * needs to be computed by the pipeline, or if the state is very large * and so is best read from file(s) rather than sent as part of the - * {@code OldDoFn}'s serialized state. + * {@link DoFn DoFn's} serialized state. * - *
            • Initialize the state in each {@link OldDoFn} instance, in - * {@link OldDoFn#startBundle}. This is good if the initialization + *
            • Initialize the state in each {@link DoFn} instance, in a + * {@link DoFn.StartBundle} method. This is good if the initialization * doesn't depend on any information known only by the main program or * computed by earlier pipeline operations, but is the same for all - * instances of this {@link OldDoFn} for all program executions, say + * instances of this {@link DoFn} for all program executions, say * setting up empty caches or initializing constant data. * *
            @@ -363,16 +362,16 @@ *

            {@link ParDo} operations are intended to be able to run in * parallel across multiple worker machines. This precludes easy * sharing and updating mutable state across those machines. There is - * no support in the Google Cloud Dataflow system for communicating + * no support in the Beam model for communicating * and synchronizing updates to shared state across worker machines, * so programs should not access any mutable static variable state in - * their {@link OldDoFn}, without understanding that the Java processes + * their {@link DoFn}, without understanding that the Java processes * for the main program and workers will each have its own independent * copy of such state, and there won't be any automatic copying of * that state across Java processes. All information should be - * communicated to {@link OldDoFn} instances via main and side inputs and + * communicated to {@link DoFn} instances via main and side inputs and * serialized state, and all output should be communicated from a - * {@link OldDoFn} instance via main and side outputs, in the absence of + * {@link DoFn} instance via main and side outputs, in the absence of * external communication mechanisms written by user code. * *

            Fault Tolerance

            @@ -380,29 +379,28 @@ *

            In a distributed system, things can fail: machines can crash, * machines can be unable to communicate across the network, etc. * While individual failures are rare, the larger the job, the greater - * the chance that something, somewhere, will fail. The Google Cloud - * Dataflow service strives to mask such failures automatically, - * principally by retrying failed {@link OldDoFn} bundle. This means - * that a {@code OldDoFn} instance might process a bundle partially, then - * crash for some reason, then be rerun (often on a different worker - * machine) on that same bundle and on the same elements as before. - * Sometimes two or more {@link OldDoFn} instances will be running on the + * the chance that something, somewhere, will fail. Beam runners may strive + * to mask such failures by retrying failed {@link DoFn} bundle. This means + * that a {@link DoFn} instance might process a bundle partially, then + * crash for some reason, then be rerun (often in a new JVM) on that + * same bundle and on the same elements as before. + * Sometimes two or more {@link DoFn} instances will be running on the * same bundle simultaneously, with the system taking the results of * the first instance to complete successfully. Consequently, the - * code in a {@link OldDoFn} needs to be written such that these + * code in a {@link DoFn} needs to be written such that these * duplicate (sequential or concurrent) executions do not cause - * problems. If the outputs of a {@link OldDoFn} are a pure function of + * problems. If the outputs of a {@link DoFn} are a pure function of * its inputs, then this requirement is satisfied. However, if a - * {@link OldDoFn OldDoFn's} execution has external side-effects, such as performing - * updates to external HTTP services, then the {@link OldDoFn OldDoFn's} code + * {@link DoFn DoFn's} execution has external side-effects, such as performing + * updates to external HTTP services, then the {@link DoFn DoFn's} code * needs to take care to ensure that those updates are idempotent and * that concurrent updates are acceptable. This property can be * difficult to achieve, so it is advisable to strive to keep - * {@link OldDoFn DoFns} as pure functions as much as possible. + * {@link DoFn DoFns} as pure functions as much as possible. * *

            Optimization

            * - *

            The Google Cloud Dataflow service automatically optimizes a + *

            Beam runners may choose to apply optimizations to a * pipeline before it is executed. A key optimization, fusion, * relates to {@link ParDo} operations. If one {@link ParDo} operation produces a * {@link PCollection} that is then consumed as the main input of another @@ -419,18 +417,16 @@ * written to disk, saving all the I/O and space expense of * constructing it. * - *

            The Google Cloud Dataflow service applies fusion as much as - * possible, greatly reducing the cost of executing pipelines. As a - * result, it is essentially "free" to write {@link ParDo} operations in a + *

            When Beam runners apply fusion optimization, it is essentially "free" + * to write {@link ParDo} operations in a * very modular, composable style, each {@link ParDo} operation doing one * clear task, and stringing together sequences of {@link ParDo} operations to * get the desired overall effect. Such programs can be easier to * understand, easier to unit-test, easier to extend and evolve, and * easier to reuse in new programs. The predefined library of - * PTransforms that come with Google Cloud Dataflow makes heavy use of - * this modular, composable style, trusting to the Google Cloud - * Dataflow service's optimizer to "flatten out" all the compositions - * into highly optimized stages. + * PTransforms that come with Beam makes heavy use of + * this modular, composable style, trusting to the runner to + * "flatten out" all the compositions into highly optimized stages. * * @see the web * documentation for ParDo @@ -443,15 +439,15 @@ public class ParDo { * *

            Side inputs are {@link PCollectionView PCollectionViews}, whose contents are * computed during pipeline execution and then made accessible to - * {@link OldDoFn} code via {@link OldDoFn.ProcessContext#sideInput sideInput}. Each - * invocation of the {@link OldDoFn} receives the same values for these + * {@link DoFn} code via {@link DoFn.ProcessContext#sideInput sideInput}. Each + * invocation of the {@link DoFn} receives the same values for these * side inputs. * *

            See the discussion of Side Inputs above for more explanation. * *

            The resulting {@link PTransform} is incomplete, and its * input/output types are not yet bound. Use - * {@link ParDo.Unbound#of} to specify the {@link OldDoFn} to + * {@link ParDo.Unbound#of} to specify the {@link DoFn} to * invoke, which will also bind the input/output types of this * {@link PTransform}. */ @@ -464,13 +460,13 @@ public static Unbound withSideInputs(PCollectionView... sideInputs) { * *

            Side inputs are {@link PCollectionView}s, whose contents are * computed during pipeline execution and then made accessible to - * {@code OldDoFn} code via {@link OldDoFn.ProcessContext#sideInput sideInput}. + * {@link DoFn} code via {@link DoFn.ProcessContext#sideInput sideInput}. * *

            See the discussion of Side Inputs above for more explanation. * *

            The resulting {@link PTransform} is incomplete, and its * input/output types are not yet bound. Use - * {@link ParDo.Unbound#of} to specify the {@link OldDoFn} to + * {@link ParDo.Unbound#of} to specify the {@link DoFn} to * invoke, which will also bind the input/output types of this * {@link PTransform}. */ @@ -486,11 +482,11 @@ public static Unbound withSideInputs( * *

            {@link TupleTag TupleTags} are used to name (with its static element * type {@code T}) each main and side output {@code PCollection}. - * This {@link PTransform PTransform's} {@link OldDoFn} emits elements to the main + * This {@link PTransform PTransform's} {@link DoFn} emits elements to the main * output {@link PCollection} as normal, using - * {@link OldDoFn.Context#output}. The {@link OldDoFn} emits elements to + * {@link DoFn.Context#output}. The {@link DoFn} emits elements to * a side output {@code PCollection} using - * {@link OldDoFn.Context#sideOutput}, passing that side output's tag + * {@link DoFn.Context#sideOutput}, passing that side output's tag * as an argument. The result of invoking this {@link PTransform} * will be a {@link PCollectionTuple}, and any of the the main and * side output {@code PCollection}s can be retrieved from it via @@ -501,7 +497,7 @@ public static Unbound withSideInputs( * *

            The resulting {@link PTransform} is incomplete, and its input * type is not yet bound. Use {@link ParDo.UnboundMulti#of} - * to specify the {@link OldDoFn} to invoke, which will also bind the + * to specify the {@link DoFn} to invoke, which will also bind the * input type of this {@link PTransform}. */ public static UnboundMulti withOutputTags( @@ -510,6 +506,20 @@ public static UnboundMulti withOutputTags( return new Unbound().withOutputTags(mainOutputTag, sideOutputTags); } + /** + * Creates a {@link ParDo} {@link PTransform} that will invoke the + * given {@link DoFn} 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 DoFn}. It is ready to be applied, or further + * properties can be set on it first. + */ + public static Bound of(DoFn fn) { + return of(adapt(fn), fn.getClass()); + } + /** * Creates a {@link ParDo} {@link PTransform} that will invoke the * given {@link OldDoFn} function. @@ -537,29 +547,11 @@ private static Bound of( return DoFnAdapters.toOldDoFn(fn); } - /** - * Creates a {@link ParDo} {@link PTransform} that will invoke the - * given {@link DoFn} 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. - * - *

            {@link DoFn} is an experimental alternative to - * {@link OldDoFn} which simplifies accessing the window of the element. - */ - @Experimental - public static Bound of(DoFn fn) { - return of(adapt(fn), fn.getClass()); - } - /** * An incomplete {@link ParDo} transform, with unbound input/output types. * *

            Before being applied, {@link ParDo.Unbound#of} must be - * invoked to specify the {@link OldDoFn} to invoke, which will also + * invoked to specify the {@link DoFn} to invoke, which will also * bind the input/output types of this {@link PTransform}. */ public static class Unbound { @@ -619,6 +611,18 @@ public UnboundMulti withOutputTags(TupleTag mainOutp name, sideInputs, mainOutputTag, sideOutputTags); } + /** + * Returns a new {@link ParDo} {@link PTransform} that's like this + * transform but which will invoke the given {@link DoFn} + * function, and which 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. + */ + public Bound of(DoFn fn) { + return of(adapt(fn), fn.getClass()); + } + /** * Returns a new {@link ParDo} {@link PTransform} that's like this * transform but that will invoke the given {@link OldDoFn} @@ -638,24 +642,11 @@ private Bound of( OldDoFn fn, Class fnClass) { return new Bound<>(name, sideInputs, fn, fnClass); } - - - /** - * Returns a new {@link ParDo} {@link PTransform} that's like this - * transform but which will invoke the given {@link DoFn} - * function, and which 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. - */ - public Bound of(DoFn fn) { - return of(adapt(fn), fn.getClass()); - } } /** * A {@link PTransform} that, when applied to a {@code PCollection}, - * invokes a user-specified {@code OldDoFn} on all its elements, + * invokes a user-specified {@code DoFn} on all its elements, * with all its outputs collected into an output * {@code PCollection}. * @@ -756,9 +747,9 @@ protected String getKindString() { /** * {@inheritDoc} * - *

            {@link ParDo} registers its internal {@link OldDoFn} as a subcomponent for display data. - * {@link OldDoFn} implementations can register display data by overriding - * {@link OldDoFn#populateDisplayData}. + *

            {@link ParDo} registers its internal {@link DoFn} as a subcomponent for display data. + * {@link DoFn} implementations can register display data by overriding + * {@link DoFn#populateDisplayData}. */ @Override public void populateDisplayData(Builder builder) { @@ -780,7 +771,7 @@ public List> getSideInputs() { * input type. * *

            Before being applied, {@link ParDo.UnboundMulti#of} must be - * invoked to specify the {@link OldDoFn} to invoke, which will also + * invoked to specify the {@link DoFn} to invoke, which will also * bind the input type of this {@link PTransform}. * * @param the type of the main output {@code PCollection} elements @@ -836,38 +827,41 @@ public UnboundMulti withSideInputs( /** * 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. + * that's like this transform but which will invoke the given + * {@link DoFn} function, and which 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. */ - public BoundMulti of(OldDoFn fn) { - return of(fn, fn.getClass()); - } - - public BoundMulti of(OldDoFn fn, Class fnClass) { - return new BoundMulti<>( - name, sideInputs, mainOutputTag, sideOutputTags, fn, fnClass); + public BoundMulti of(DoFn fn) { + return of(adapt(fn), fn.getClass()); } /** * Returns a new multi-output {@link ParDo} {@link PTransform} - * that's like this transform but which will invoke the given - * {@link DoFn} function, and which has its input type bound. + * 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} */ - public BoundMulti of(DoFn fn) { - return of(adapt(fn), fn.getClass()); + @Deprecated + public BoundMulti of(OldDoFn fn) { + return of(fn, fn.getClass()); + } + + private BoundMulti of(OldDoFn fn, Class fnClass) { + return new BoundMulti<>( + name, sideInputs, mainOutputTag, sideOutputTags, fn, fnClass); } } /** * A {@link PTransform} that, when applied to a * {@code PCollection}, invokes a user-specified - * {@code OldDoFn} on all its elements, which can emit elements + * {@code DoFn} on all its elements, which can emit elements * to any of the {@link PTransform}'s main and side output * {@code PCollection}s, which are bundled into a result * {@code PCollectionTuple}. @@ -939,7 +933,7 @@ public PCollectionTuple apply(PCollection input) { input.isBounded()); // The fn will likely be an instance of an anonymous subclass - // such as OldDoFn { }, thus will have a high-fidelity + // such as DoFn { }, thus will have a high-fidelity // TypeDescriptor for the output type. outputs.get(mainOutputTag).setTypeDescriptorInternal(fn.getOutputTypeDescriptor()); diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/SimpleFunction.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/SimpleFunction.java index bf075f8dcbd29..86046598fb4c4 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/SimpleFunction.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/SimpleFunction.java @@ -37,7 +37,7 @@ SimpleFunction fromSerializableFunctionWithOutputType( /** * Returns a {@link TypeDescriptor} capturing what is known statically - * about the input type of this {@code OldDoFn} instance's most-derived + * about the input type of this {@link SimpleFunction} instance's most-derived * class. * *

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

            In the normal case of a concrete {@code OldDoFn} subclass with + *

            In the normal case of a concrete {@link SimpleFunction} 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 diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/PaneInfo.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/PaneInfo.java index 0c87e2271ec92..727a4925cb31a 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/PaneInfo.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/windowing/PaneInfo.java @@ -29,15 +29,15 @@ 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.transforms.DoFn; import org.apache.beam.sdk.transforms.GroupByKey; -import org.apache.beam.sdk.transforms.OldDoFn; import org.apache.beam.sdk.util.VarInt; /** * Provides information about the pane an element belongs to. Every pane is implicitly associated * with a window. Panes are observable only via the - * {@link OldDoFn.ProcessContext#pane} method of the context - * passed to a {@link OldDoFn#processElement} overridden method. + * {@link DoFn.ProcessContext#pane} method of the context + * passed to a {@link DoFn.ProcessElement} method. * *

            Note: This does not uniquely identify a pane, and should not be used for comparisons. */ @@ -72,8 +72,8 @@ public final class PaneInfo { * definitions: *

              *
            1. We'll call a pipeline 'simple' if it does not use - * {@link OldDoFn.Context#outputWithTimestamp} in - * any {@code OldDoFn}, and it uses the same + * {@link DoFn.Context#outputWithTimestamp} in + * any {@link DoFn}, and it uses the same * {@link org.apache.beam.sdk.transforms.windowing.Window.Bound#withAllowedLateness} * argument value on all windows (or uses the default of {@link org.joda.time.Duration#ZERO}). *
            2. We'll call an element 'locally late', from the point of view of a computation on a diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/BaseExecutionContext.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/BaseExecutionContext.java index dead76eb3d001..9ee55ad3a4f0b 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/BaseExecutionContext.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/BaseExecutionContext.java @@ -106,7 +106,7 @@ public Collection getAllStepContexts() { /** * Hook for subclasses to implement that will be called whenever - * {@link OldDoFn.Context#output} + * {@code DoFn.Context#output} * is called. */ @Override @@ -114,7 +114,7 @@ public void noteOutput(WindowedValue output) {} /** * Hook for subclasses to implement that will be called whenever - * {@link OldDoFn.Context#sideOutput} + * {@code DoFn.Context#sideOutput} * is called. */ @Override diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ReifyTimestampAndWindowsDoFn.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ReifyTimestampAndWindowsDoFn.java index 2808ca90568e3..8f3f540e66ae4 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ReifyTimestampAndWindowsDoFn.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/ReifyTimestampAndWindowsDoFn.java @@ -21,8 +21,8 @@ import org.apache.beam.sdk.values.KV; /** - * OldDoFn that makes timestamps and window assignments explicit in the value part of each key/value - * pair. + * {@link OldDoFn} that makes timestamps and window assignments explicit in the value part of each + * key/value pair. * * @param the type of the keys of the input and output {@code PCollection}s * @param the type of the values of the input {@code PCollection} diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/SerializableUtils.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/SerializableUtils.java index 354aa5d91182d..6b3218ecf199c 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/SerializableUtils.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/SerializableUtils.java @@ -102,7 +102,7 @@ public static T clone(T value) { */ public static CloudObject ensureSerializable(Coder coder) { // Make sure that Coders are java serializable as well since - // they are regularly captured within OldDoFn's. + // they are regularly captured within DoFn's. Coder copy = (Coder) ensureSerializable((Serializable) coder); CloudObject cloudObject = copy.asCloudObject(); diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/SystemDoFnInternal.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/SystemDoFnInternal.java index e9904b2d7660a..004496b8319b3 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/SystemDoFnInternal.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/SystemDoFnInternal.java @@ -22,15 +22,14 @@ import java.lang.annotation.Retention; import java.lang.annotation.RetentionPolicy; import java.lang.annotation.Target; -import org.apache.beam.sdk.transforms.OldDoFn; /** - * Annotation to mark {@link OldDoFn DoFns} as an internal component of the Dataflow SDK. + * Annotation to mark {@code DoFns} as an internal component of the Beam SDK. * *

              Currently, the only effect of this is to mark any aggregators reported by an annotated - * {@code OldDoFn} as a system counter (as opposed to a user counter). + * {@code DoFn} as a system counter (as opposed to a user counter). * - *

              This is internal to the Dataflow SDK. + *

              This is internal to the Beam SDK. */ @Documented @Retention(RetentionPolicy.RUNTIME) 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 54158d24e5208..016276cb4bf34 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 @@ -20,6 +20,7 @@ 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; import org.apache.beam.sdk.util.state.StateInternals; @@ -28,7 +29,7 @@ import org.joda.time.Instant; /** - * Interface that may be required by some (internal) {@code OldDoFn}s to implement windowing. It + * Interface that may be required by some (internal) {@link DoFn}s to implement windowing. It * should not be necessary for general user code to interact with this at all. * *

              This interface should be provided by runner implementors to support windowing on their runner. diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/DoFnDelegatingAggregatorTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/DoFnDelegatingAggregatorTest.java index 25b909aabbaf3..c072fd7c47db4 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/DoFnDelegatingAggregatorTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/DoFnDelegatingAggregatorTest.java @@ -35,7 +35,7 @@ import org.mockito.MockitoAnnotations; /** - * Tests for OldDoFn.DelegatingAggregator. + * Tests for {@link OldDoFn.DelegatingAggregator}. */ @RunWith(JUnit4.class) public class DoFnDelegatingAggregatorTest { 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 2f1519cc42764..2649be5aadf30 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 @@ -276,7 +276,8 @@ public void processElement(ProcessContext c) throws Exception { } /** - * A OldDoFn that adds values to an aggregator and converts input to String in processElement. + * An {@link OldDoFn} that adds values to an aggregator and converts input to String in + * {@link OldDoFn#processElement). */ private static class CounterDoFn extends OldDoFn { Aggregator agg = createAggregator("ctr", new Sum.SumLongFn()); 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 c73251000650a..302b66aa7c4eb 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 @@ -28,7 +28,7 @@ /** * A {@link OldDoFn} that does nothing with provided elements. Used for testing - * methods provided by the OldDoFn abstract class. + * methods provided by the {@link OldDoFn} abstract class. * * @param unused. * @param unused. From 4ec73d80e3d30ff4a3b5738e9a8711e857381015 Mon Sep 17 00:00:00 2001 From: Thomas Groh Date: Thu, 25 Aug 2016 14:58:56 -0700 Subject: [PATCH 072/346] Make WriteTest more resilient to Randomness In the worst case scenario for random key assignment in Write.ApplyShardingKey, the chance of the number of records per output shard was too high. This makes the test significantly less likely to flake. --- .../src/test/java/org/apache/beam/sdk/io/WriteTest.java | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) 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 28651884d2982..997566ae619b8 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 @@ -192,11 +192,12 @@ public void testShardedWriteBalanced() { inputs.add(String.format("elt%04d", i)); } + int numShards = 10; runShardedWrite( inputs, new WindowAndReshuffle<>( Window.into(Sessions.withGapDuration(Duration.millis(1)))), - Optional.of(10)); + Optional.of(numShards)); // Check that both the min and max number of results per shard are close to the expected. int min = Integer.MAX_VALUE; @@ -205,7 +206,9 @@ public void testShardedWriteBalanced() { min = Math.min(min, i); max = Math.max(max, i); } - assertThat((double) min, Matchers.greaterThanOrEqualTo(max * 0.9)); + double expected = numElements / (double) numShards; + assertThat((double) min, Matchers.greaterThanOrEqualTo(expected * 0.6)); + assertThat((double) max, Matchers.lessThanOrEqualTo(expected * 1.4)); } /** From b3be7b70ac1606edd58a9ca1d4861e9d9e2a07a9 Mon Sep 17 00:00:00 2001 From: Dan Halperin Date: Tue, 23 Aug 2016 18:54:23 -0700 Subject: [PATCH 073/346] checkstyle: prohibit API client repackaged Guava Apparently the IllegalImport check only blocks packages, so we had to move to Regexp to get individual classes. As a bonus, this enforcement let us remove two bogus dependencies. Smaller JARs for the win! --- runners/spark/pom.xml | 4 ---- .../streaming/StreamingTransformTranslator.java | 6 +++--- .../build-tools/src/main/resources/beam/checkstyle.xml | 10 +++++++++- .../beam/sdk/io/gcp/bigquery/BigQueryIOTest.java | 2 +- sdks/java/io/hdfs/pom.xml | 5 ----- .../java/org/apache/beam/sdk/io/hdfs/HDFSFileSink.java | 2 +- 6 files changed, 14 insertions(+), 15 deletions(-) diff --git a/runners/spark/pom.xml b/runners/spark/pom.xml index a6311b5dcf1f6..a5e99a06e80b9 100644 --- a/runners/spark/pom.xml +++ b/runners/spark/pom.xml @@ -154,10 +154,6 @@ auto-service true - - com.google.http-client - google-http-client - com.fasterxml.jackson.core jackson-core 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 43160f5500669..5f35ebb201862 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 @@ -17,9 +17,9 @@ */ package org.apache.beam.runners.spark.translation.streaming; -import com.google.api.client.util.Lists; -import com.google.api.client.util.Maps; -import com.google.api.client.util.Sets; +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; +import com.google.common.collect.Sets; import com.google.common.reflect.TypeToken; import java.lang.reflect.ParameterizedType; import java.lang.reflect.Type; diff --git a/sdks/java/build-tools/src/main/resources/beam/checkstyle.xml b/sdks/java/build-tools/src/main/resources/beam/checkstyle.xml index 47ddc5b2e37ad..c7d9b2c2a8132 100644 --- a/sdks/java/build-tools/src/main/resources/beam/checkstyle.xml +++ b/sdks/java/build-tools/src/main/resources/beam/checkstyle.xml @@ -100,7 +100,15 @@ page at http://checkstyle.sourceforge.net/config.html --> - + + + + + + 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 a6d7e2f5e957c..7a7575bfdd24c 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 @@ -33,7 +33,6 @@ import static org.mockito.Mockito.when; import com.google.api.client.util.Data; -import com.google.api.client.util.Strings; import com.google.api.services.bigquery.model.ErrorProto; import com.google.api.services.bigquery.model.Job; import com.google.api.services.bigquery.model.JobConfigurationExtract; @@ -50,6 +49,7 @@ import com.google.api.services.bigquery.model.TableReference; import com.google.api.services.bigquery.model.TableRow; import com.google.api.services.bigquery.model.TableSchema; +import com.google.common.base.Strings; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Lists; diff --git a/sdks/java/io/hdfs/pom.xml b/sdks/java/io/hdfs/pom.xml index 07b9eb6b4bfe7..0ec542ccf9d80 100644 --- a/sdks/java/io/hdfs/pom.xml +++ b/sdks/java/io/hdfs/pom.xml @@ -81,11 +81,6 @@ annotations - - com.google.http-client - google-http-client - - org.apache.avro avro diff --git a/sdks/java/io/hdfs/src/main/java/org/apache/beam/sdk/io/hdfs/HDFSFileSink.java b/sdks/java/io/hdfs/src/main/java/org/apache/beam/sdk/io/hdfs/HDFSFileSink.java index 0b538b355bc18..6d30307f2e07f 100644 --- a/sdks/java/io/hdfs/src/main/java/org/apache/beam/sdk/io/hdfs/HDFSFileSink.java +++ b/sdks/java/io/hdfs/src/main/java/org/apache/beam/sdk/io/hdfs/HDFSFileSink.java @@ -19,8 +19,8 @@ import static com.google.common.base.Preconditions.checkState; -import com.google.api.client.util.Maps; import com.google.common.collect.Lists; +import com.google.common.collect.Maps; import com.google.common.collect.Sets; import java.io.IOException; import java.util.Map; From 1f8b5344ce1cad4f82c70c11503fef90b27e493f Mon Sep 17 00:00:00 2001 From: Ian Zhou Date: Mon, 15 Aug 2016 15:39:56 -0700 Subject: [PATCH 074/346] Modified BigtableIO to use DoFn setup/tearDown methods instead of startBundle/finishBundle --- .../beam/sdk/io/gcp/bigtable/BigtableIO.java | 17 +++++++++++++---- .../sdk/io/gcp/bigtable/BigtableService.java | 9 ++++++++- .../io/gcp/bigtable/BigtableServiceImpl.java | 8 ++++++++ .../sdk/io/gcp/bigtable/BigtableIOTest.java | 3 +++ 4 files changed, 32 insertions(+), 5 deletions(-) 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 3a9ffce487f6f..67dde507313d8 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 @@ -515,9 +515,13 @@ public BigtableWriterFn(String tableId, BigtableService bigtableService) { this.failures = new ConcurrentLinkedQueue<>(); } - @StartBundle - public void startBundle(Context c) throws Exception { + @Setup + public void setup() throws Exception { bigtableWriter = bigtableService.openForWriting(tableId); + } + + @StartBundle + public void startBundle(Context c) { recordsWritten = 0; } @@ -531,12 +535,17 @@ public void processElement(ProcessContext c) throws Exception { @FinishBundle public void finishBundle(Context c) throws Exception { - bigtableWriter.close(); - bigtableWriter = null; + bigtableWriter.flush(); checkForFailures(); logger.info("Wrote {} records", recordsWritten); } + @Teardown + public void tearDown() throws Exception { + bigtableWriter.close(); + bigtableWriter = null; + } + @Override public void populateDisplayData(DisplayData.Builder builder) { Write.this.populateDisplayData(builder); diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableService.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableService.java index ecd38a7468e4b..c656bbbf3fd99 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableService.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableService.java @@ -50,10 +50,17 @@ ListenableFuture writeRecord(KV writeRecord( return Futures.immediateFuture(MutateRowResponse.getDefaultInstance()); } + @Override + public void flush() {} + @Override public void close() {} } From 5b425ac5020aefff6f6474610f5ad35fad54d306 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Jean-Baptiste=20Onofr=C3=A9?= Date: Thu, 25 Aug 2016 14:32:20 +0200 Subject: [PATCH 075/346] [BEAM-294] Rename dataflow references to beam --- runners/spark/pom.xml | 2 +- .../java/org/apache/beam/runners/spark/SparkRunner.java | 2 +- .../org/apache/beam/runners/spark/TestSparkRunner.java | 2 +- .../beam/runners/spark/aggregators/NamedAggregators.java | 2 +- .../org/apache/beam/runners/spark/examples/WordCount.java | 2 +- .../runners/spark/io/hadoop/ShardNameTemplateHelper.java | 6 +++--- .../beam/runners/spark/translation/DoFnFunction.java | 2 +- .../spark/translation/SparkPipelineTranslator.java | 2 +- .../runners/spark/translation/SparkRuntimeContext.java | 2 +- .../runners/spark/translation/TransformTranslator.java | 4 ++-- .../streaming/StreamingTransformTranslator.java | 8 ++++---- .../apache/beam/runners/spark/util/BroadcastHelper.java | 4 ++-- .../spark/translation/TransformTranslatorTest.java | 2 +- .../spark/translation/streaming/KafkaStreamingTest.java | 2 +- 14 files changed, 21 insertions(+), 21 deletions(-) diff --git a/runners/spark/pom.xml b/runners/spark/pom.xml index a5e99a06e80b9..b924cb8e3ea88 100644 --- a/runners/spark/pom.xml +++ b/runners/spark/pom.xml @@ -330,7 +330,7 @@ - com.google.common 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 2ce1ff697e42a..fa85a2e25e262 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 @@ -50,7 +50,7 @@ /** * 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 dataflow pipeline with the default options of a single threaded spark instance in local mode, + * a Beam pipeline with the default options of a single threaded spark instance in local mode, * we would do the following: * * {@code 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 50ed5f3f5ce8c..376b80ff953b2 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 @@ -28,7 +28,7 @@ /** * 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 dataflow + * by Spark, and then submitting the job to Spark to be executed. 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: * diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/aggregators/NamedAggregators.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/aggregators/NamedAggregators.java index c15e276c43dbb..e2cd9632e37b7 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/aggregators/NamedAggregators.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/aggregators/NamedAggregators.java @@ -32,7 +32,7 @@ /** * This class wraps a map of named aggregators. Spark expects that all accumulators be declared - * before a job is launched. Dataflow allows aggregators to be used and incremented on the fly. + * before a job is launched. Beam allows aggregators to be used and incremented on the fly. * We create a map of named aggregators and instantiate in the the spark context before the job * is launched. We can then add aggregators on the fly in Spark. */ 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 06770305f2c14..1af84add67ecb 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 @@ -110,7 +110,7 @@ public PCollection> apply(PCollection lines) { */ public static interface WordCountOptions extends PipelineOptions { @Description("Path of the file to read from") - @Default.String("gs://dataflow-samples/shakespeare/kinglear.txt") + @Default.String("gs://beam-samples/shakespeare/kinglear.txt") String getInputFile(); void setInputFile(String value); diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/io/hadoop/ShardNameTemplateHelper.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/io/hadoop/ShardNameTemplateHelper.java index 7f8e2978c439d..4a7058bfd521d 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/io/hadoop/ShardNameTemplateHelper.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/io/hadoop/ShardNameTemplateHelper.java @@ -36,9 +36,9 @@ public final class ShardNameTemplateHelper { private static final Logger LOG = LoggerFactory.getLogger(ShardNameTemplateHelper.class); - public static final String OUTPUT_FILE_PREFIX = "spark.dataflow.fileoutputformat.prefix"; - public static final String OUTPUT_FILE_TEMPLATE = "spark.dataflow.fileoutputformat.template"; - public static final String OUTPUT_FILE_SUFFIX = "spark.dataflow.fileoutputformat.suffix"; + public static final String OUTPUT_FILE_PREFIX = "spark.beam.fileoutputformat.prefix"; + public static final String OUTPUT_FILE_TEMPLATE = "spark.beam.fileoutputformat.template"; + public static final String OUTPUT_FILE_SUFFIX = "spark.beam.fileoutputformat.suffix"; private ShardNameTemplateHelper() { } diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/DoFnFunction.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/DoFnFunction.java index 800d614e5e964..454b7607e9df2 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/DoFnFunction.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/DoFnFunction.java @@ -31,7 +31,7 @@ import org.slf4j.LoggerFactory; /** - * Dataflow's Do functions correspond to Spark's FlatMap functions. + * Beam's Do functions correspond to Spark's FlatMap functions. * * @param Input element type. * @param Output element type. diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkPipelineTranslator.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkPipelineTranslator.java index 997940bb99de6..1f7ccf1e005b7 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkPipelineTranslator.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkPipelineTranslator.java @@ -20,7 +20,7 @@ import org.apache.beam.sdk.transforms.PTransform; /** - * Translator to support translation between Dataflow transformations and Spark transformations. + * Translator to support translation between Beam transformations and Spark transformations. */ public interface SparkPipelineTranslator { 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 4bc0c00973f8b..2634c65b24484 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 @@ -57,7 +57,7 @@ public class SparkRuntimeContext implements Serializable { private final String serializedPipelineOptions; /** - * Map fo names to dataflow aggregators. + * Map fo names to Beam aggregators. */ private final Map> aggregators = new HashMap<>(); private transient CoderRegistry coderRegistry; 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 08e3fda7a36b5..eaceb852ce344 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 @@ -94,7 +94,7 @@ import scala.Tuple2; /** - * Supports translation between a DataFlow transform, and Spark's operations on RDDs. + * Supports translation between a Beam transform, and Spark's operations on RDDs. */ public final class TransformTranslator { @@ -895,7 +895,7 @@ private static Map, BroadcastHelper> getSideInputs( } /** - * Translator matches Dataflow transformation with the appropriate evaluator. + * Translator matches Beam transformation with the appropriate evaluator. */ public static class Translator implements SparkPipelineTranslator { 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 5f35ebb201862..43dcef665fb2a 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 @@ -68,7 +68,7 @@ /** - * Supports translation between a DataFlow transform, and Spark's operations on DStreams. + * Supports translation between a Beam transform, and Spark's operations on DStreams. */ public final class StreamingTransformTranslator { @@ -349,13 +349,13 @@ public void evaluate(Window.Bound transform, EvaluationContext context) { (TransformEvaluator) EVALUATORS.get(clazz); if (transform == null) { if (UNSUPPORTED_EVALUATORS.contains(clazz)) { - throw new UnsupportedOperationException("Dataflow transformation " + clazz + throw new UnsupportedOperationException("Beam transformation " + clazz .getCanonicalName() + " is currently unsupported by the Spark streaming pipeline"); } // DStream transformations will transform an RDD into another RDD // Actions will create output - // In Dataflow it depends on the PTransform's Input and Output class + // In Beam it depends on the PTransform's Input and Output class Class pTOutputClazz = getPTransformOutputClazz(clazz); if (PDone.class.equals(pTOutputClazz)) { return foreachRDD(rddTranslator); @@ -373,7 +373,7 @@ public void evaluate(Window.Bound transform, EvaluationContext context) { } /** - * Translator matches Dataflow transformation with the appropriate Spark streaming evaluator. + * Translator matches Beam transformation with the appropriate Spark streaming evaluator. * rddTranslator uses Spark evaluators in transform/foreachRDD to evaluate the transformation */ public static class Translator implements SparkPipelineTranslator { diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/util/BroadcastHelper.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/util/BroadcastHelper.java index 29c2dd995b6e6..5f0c79576e595 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/util/BroadcastHelper.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/util/BroadcastHelper.java @@ -34,12 +34,12 @@ public abstract class BroadcastHelper implements Serializable { /** - * If the property {@code dataflow.spark.directBroadcast} is set to + * If the property {@code beam.spark.directBroadcast} is set to * {@code true} then Spark serialization (Kryo) will be used to broadcast values * in View objects. By default this property is not set, and values are coded using * the appropriate {@link Coder}. */ - public static final String DIRECT_BROADCAST = "dataflow.spark.directBroadcast"; + public static final String DIRECT_BROADCAST = "beam.spark.directBroadcast"; private static final Logger LOG = LoggerFactory.getLogger(BroadcastHelper.class); diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/TransformTranslatorTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/TransformTranslatorTest.java index f61ad1c9a9fda..f72eba7ea6cac 100644 --- a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/TransformTranslatorTest.java +++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/TransformTranslatorTest.java @@ -56,7 +56,7 @@ public class TransformTranslatorTest { /** * Builds a simple pipeline with TextIO.Read and TextIO.Write, runs the pipeline - * in DirectRunner and on SparkRunner, with the mapped dataflow-to-spark + * in DirectRunner and on SparkRunner, with the mapped beam-to-spark * transforms. Finally it makes sure that the results are the same for both runs. */ @Test 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 27d6f5ea60689..ac77922bc57c3 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 @@ -56,7 +56,7 @@ public class KafkaStreamingTest { new EmbeddedKafkaCluster.EmbeddedZookeeper(); private static final EmbeddedKafkaCluster EMBEDDED_KAFKA_CLUSTER = new EmbeddedKafkaCluster(EMBEDDED_ZOOKEEPER.getConnection(), new Properties()); - private static final String TOPIC = "kafka_dataflow_test_topic"; + private static final String TOPIC = "kafka_beam_test_topic"; private static final Map KAFKA_MESSAGES = ImmutableMap.of( "k1", "v1", "k2", "v2", "k3", "v3", "k4", "v4" ); From ef312e98c6f671d27e9ca50f9c8511932cca761d Mon Sep 17 00:00:00 2001 From: staslev Date: Fri, 26 Aug 2016 10:26:38 +0300 Subject: [PATCH 076/346] Added support for reporting aggregator values to Spark sinks --- runners/spark/pom.xml | 6 + .../runners/spark/SparkPipelineOptions.java | 7 +- .../spark/aggregators/NamedAggregators.java | 38 +++- .../aggregators/metrics/AggregatorMetric.java | 44 +++++ .../metrics/AggregatorMetricSource.java | 49 +++++ .../metrics/WithNamedAggregatorsSupport.java | 169 ++++++++++++++++++ .../aggregators/metrics/package-info.java | 22 +++ .../aggregators/metrics/sink/CsvSink.java | 39 ++++ .../metrics/sink/GraphiteSink.java | 39 ++++ .../metrics/sink/package-info.java | 23 +++ .../beam/runners/spark/io/ConsoleIO.java | 2 +- .../runners/spark/io/hadoop/HadoopIO.java | 4 +- .../translation/SparkRuntimeContext.java | 29 ++- .../runners/spark/util/BroadcastHelper.java | 4 +- .../spark/InMemoryMetricsSinkRule.java | 32 ++++ .../runners/spark/SimpleWordCountTest.java | 12 ++ .../metrics/sink/InMemoryMetrics.java | 79 ++++++++ .../src/test/resources/metrics.properties | 29 +++ 18 files changed, 611 insertions(+), 16 deletions(-) create mode 100644 runners/spark/src/main/java/org/apache/beam/runners/spark/aggregators/metrics/AggregatorMetric.java create mode 100644 runners/spark/src/main/java/org/apache/beam/runners/spark/aggregators/metrics/AggregatorMetricSource.java create mode 100644 runners/spark/src/main/java/org/apache/beam/runners/spark/aggregators/metrics/WithNamedAggregatorsSupport.java create mode 100644 runners/spark/src/main/java/org/apache/beam/runners/spark/aggregators/metrics/package-info.java create mode 100644 runners/spark/src/main/java/org/apache/beam/runners/spark/aggregators/metrics/sink/CsvSink.java create mode 100644 runners/spark/src/main/java/org/apache/beam/runners/spark/aggregators/metrics/sink/GraphiteSink.java create mode 100644 runners/spark/src/main/java/org/apache/beam/runners/spark/aggregators/metrics/sink/package-info.java create mode 100644 runners/spark/src/test/java/org/apache/beam/runners/spark/InMemoryMetricsSinkRule.java create mode 100644 runners/spark/src/test/java/org/apache/beam/runners/spark/aggregators/metrics/sink/InMemoryMetrics.java create mode 100644 runners/spark/src/test/resources/metrics.properties diff --git a/runners/spark/pom.xml b/runners/spark/pom.xml index b924cb8e3ea88..b928b4452967e 100644 --- a/runners/spark/pom.xml +++ b/runners/spark/pom.xml @@ -37,6 +37,7 @@ 1.6.2 2.2.0 0.8.2.1 + 3.1.2 @@ -231,6 +232,11 @@ + + io.dropwizard.metrics + metrics-core + ${dropwizard.metrics.version} + 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 080ff19a0c25f..be4f7f06a9bc0 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 @@ -38,10 +38,15 @@ public interface SparkPipelineOptions extends PipelineOptions, StreamingOptions, + "execution is stopped") @Default.Long(-1) Long getTimeout(); - void setTimeout(Long batchInterval); + void setTimeout(Long timeoutMillis); @Description("Batch interval for Spark streaming in milliseconds.") @Default.Long(1000) Long getBatchIntervalMillis(); void setBatchIntervalMillis(Long batchInterval); + + @Description("Enable/disable sending aggregator values to Spark's metric sinks") + @Default.Boolean(true) + Boolean getEnableSparkSinks(); + void setEnableSparkSinks(Boolean enableSparkSinks); } diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/aggregators/NamedAggregators.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/aggregators/NamedAggregators.java index e2cd9632e37b7..4e96466706dc9 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/aggregators/NamedAggregators.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/aggregators/NamedAggregators.java @@ -18,13 +18,18 @@ package org.apache.beam.runners.spark.aggregators; +import com.google.common.base.Function; import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Maps; + import java.io.IOException; import java.io.ObjectInputStream; import java.io.ObjectOutputStream; import java.io.Serializable; import java.util.Map; import java.util.TreeMap; + import org.apache.beam.runners.spark.translation.SparkRuntimeContext; import org.apache.beam.sdk.coders.CannotProvideCoderException; import org.apache.beam.sdk.coders.Coder; @@ -69,6 +74,22 @@ public T getValue(String name, Class typeClass) { return typeClass.cast(mNamedAggregators.get(name).render()); } + /** + * @return a map of all the aggregator names and their rendered values + */ + public Map renderAll() { + return + ImmutableMap.copyOf( + Maps.transformValues(mNamedAggregators, + new Function, Object>() { + + @Override + public Object apply(State state) { + return state.render(); + } + })); + } + /** * Merges another NamedAggregators instance with this instance. * @@ -116,6 +137,7 @@ public String toString() { * @param Output data type */ public interface State extends Serializable { + /** * @param element new element to update state */ @@ -133,16 +155,16 @@ public interface State extends Serializable { /** * => combineFunction in data flow. */ - public static class CombineFunctionState - implements State { + public static class CombineFunctionState + implements State { - private Combine.CombineFn combineFn; + private Combine.CombineFn combineFn; private Coder inCoder; private SparkRuntimeContext ctxt; private transient InterT state; public CombineFunctionState( - Combine.CombineFn combineFn, + Combine.CombineFn combineFn, Coder inCoder, SparkRuntimeContext ctxt) { this.combineFn = combineFn; @@ -157,7 +179,7 @@ public void update(InputT element) { } @Override - public State merge(State other) { + public State merge(State other) { this.state = combineFn.mergeAccumulators(ImmutableList.of(current(), other.current())); return this; } @@ -168,12 +190,12 @@ public InterT current() { } @Override - public OutpuT render() { + public OutputT render() { return combineFn.extractOutput(state); } @Override - public Combine.CombineFn getCombineFn() { + public Combine.CombineFn getCombineFn() { return combineFn; } @@ -192,7 +214,7 @@ private void writeObject(ObjectOutputStream oos) throws IOException { @SuppressWarnings("unchecked") private void readObject(ObjectInputStream ois) throws IOException, ClassNotFoundException { ctxt = (SparkRuntimeContext) ois.readObject(); - combineFn = (Combine.CombineFn) ois.readObject(); + combineFn = (Combine.CombineFn) ois.readObject(); inCoder = (Coder) ois.readObject(); try { state = combineFn.getAccumulatorCoder(ctxt.getCoderRegistry(), inCoder) diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/aggregators/metrics/AggregatorMetric.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/aggregators/metrics/AggregatorMetric.java new file mode 100644 index 0000000000000..c07a0697e8f18 --- /dev/null +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/aggregators/metrics/AggregatorMetric.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.runners.spark.aggregators.metrics; + +import com.codahale.metrics.Metric; + +import org.apache.beam.runners.spark.aggregators.NamedAggregators; + +/** + * An adapter between the {@link NamedAggregators} and codahale's {@link Metric} + * interface. + */ +public class AggregatorMetric implements Metric { + + private final NamedAggregators namedAggregators; + + private AggregatorMetric(final NamedAggregators namedAggregators) { + this.namedAggregators = namedAggregators; + } + + public static AggregatorMetric of(final NamedAggregators namedAggregators) { + return new AggregatorMetric(namedAggregators); + } + + NamedAggregators getNamedAggregators() { + return namedAggregators; + } +} diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/aggregators/metrics/AggregatorMetricSource.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/aggregators/metrics/AggregatorMetricSource.java new file mode 100644 index 0000000000000..0658e049a999e --- /dev/null +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/aggregators/metrics/AggregatorMetricSource.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.runners.spark.aggregators.metrics; + +import com.codahale.metrics.MetricRegistry; + +import org.apache.beam.runners.spark.aggregators.NamedAggregators; +import org.apache.spark.metrics.source.Source; + +/** + * A Spark {@link Source} that is tailored to expose an {@link AggregatorMetric}, + * wrapping an underlying {@link NamedAggregators} instance. + */ +public class AggregatorMetricSource implements Source { + + private static final String SOURCE_NAME = "NamedAggregators"; + + private final MetricRegistry metricRegistry = new MetricRegistry(); + + public AggregatorMetricSource(final NamedAggregators aggregators) { + metricRegistry.register(SOURCE_NAME, AggregatorMetric.of(aggregators)); + } + + @Override + public String sourceName() { + return SOURCE_NAME; + } + + @Override + public MetricRegistry metricRegistry() { + return metricRegistry; + } +} 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 new file mode 100644 index 0000000000000..88e2211cf9e1c --- /dev/null +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/aggregators/metrics/WithNamedAggregatorsSupport.java @@ -0,0 +1,169 @@ +/* + * 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.metrics; + +import com.codahale.metrics.Counter; +import com.codahale.metrics.Gauge; +import com.codahale.metrics.Histogram; +import com.codahale.metrics.Meter; +import com.codahale.metrics.Metric; +import com.codahale.metrics.MetricFilter; +import com.codahale.metrics.MetricRegistry; +import com.codahale.metrics.Timer; + +import com.google.common.base.Function; +import com.google.common.base.Optional; +import com.google.common.base.Predicate; +import com.google.common.base.Predicates; +import com.google.common.collect.FluentIterable; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.ImmutableSortedMap; +import com.google.common.collect.Maps; +import com.google.common.collect.Ordering; + +import java.util.Map; +import java.util.SortedMap; + +import org.apache.beam.runners.spark.aggregators.NamedAggregators; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * A {@link MetricRegistry} decorator-like* that supports {@link AggregatorMetric} by exposing + * the underlying * {@link org.apache.beam.runners.spark.aggregators.NamedAggregators}' + * aggregators as {@link Gauge}s. + *

              + * *{@link MetricRegistry} is not an interface, so this is not a by-the-book decorator. + * That said, it delegates all metric related getters to the "decorated" instance. + *

              + */ +public class WithNamedAggregatorsSupport extends MetricRegistry { + + private static final Logger LOG = LoggerFactory.getLogger(WithNamedAggregatorsSupport.class); + + private MetricRegistry internalMetricRegistry; + + private WithNamedAggregatorsSupport(final MetricRegistry internalMetricRegistry) { + this.internalMetricRegistry = internalMetricRegistry; + } + + public static WithNamedAggregatorsSupport forRegistry(final MetricRegistry metricRegistry) { + return new WithNamedAggregatorsSupport(metricRegistry); + } + + @Override + public SortedMap getTimers(final MetricFilter filter) { + return internalMetricRegistry.getTimers(filter); + } + + @Override + public SortedMap getMeters(final MetricFilter filter) { + return internalMetricRegistry.getMeters(filter); + } + + @Override + public SortedMap getHistograms(final MetricFilter filter) { + return internalMetricRegistry.getHistograms(filter); + } + + @Override + public SortedMap getCounters(final MetricFilter filter) { + return internalMetricRegistry.getCounters(filter); + } + + @Override + public SortedMap getGauges(final MetricFilter filter) { + return + new ImmutableSortedMap.Builder( + Ordering.from(String.CASE_INSENSITIVE_ORDER)) + .putAll(internalMetricRegistry.getGauges(filter)) + .putAll(extractGauges(internalMetricRegistry, filter)) + .build(); + } + + private Map extractGauges(final MetricRegistry metricRegistry, + final MetricFilter filter) { + + // find the AggregatorMetric metrics from within all currently registered metrics + final Optional> gauges = + FluentIterable + .from(metricRegistry.getMetrics().entrySet()) + .firstMatch(isAggregatorMetric()) + .transform(toGauges()); + + return + gauges.isPresent() + ? Maps.filterEntries(gauges.get(), matches(filter)) + : ImmutableMap.of(); + } + + private Function, Map> toGauges() { + return new Function, Map>() { + @Override + public Map apply(final Map.Entry entry) { + final NamedAggregators agg = ((AggregatorMetric) entry.getValue()).getNamedAggregators(); + final Map gaugeMap = Maps.transformEntries(agg.renderAll(), toGauge()); + return Maps.filterValues(gaugeMap, Predicates.notNull()); + } + }; + } + + private Maps.EntryTransformer toGauge() { + return new Maps.EntryTransformer() { + + @Override + public Gauge transformEntry(final String name, final Object rawValue) { + return new Gauge() { + + @Override + public Double getValue() { + // at the moment the metric's type is assumed to be + // compatible with Double. While far from perfect, it seems reasonable at + // this point in time + try { + return Double.parseDouble(rawValue.toString()); + } catch (final Exception e) { + LOG.warn("Failed reporting metric with name [{}], of type [{}], since it could not be" + + " converted to double", name, rawValue.getClass().getSimpleName(), e); + return null; + } + } + }; + } + }; + } + + private Predicate> matches(final MetricFilter filter) { + return new Predicate>() { + @Override + public boolean apply(final Map.Entry entry) { + return filter.matches(entry.getKey(), entry.getValue()); + } + }; + } + + private Predicate> isAggregatorMetric() { + return new Predicate>() { + @Override + public boolean apply(final Map.Entry metricEntry) { + return (metricEntry.getValue() instanceof AggregatorMetric); + } + }; + } +} diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/aggregators/metrics/package-info.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/aggregators/metrics/package-info.java new file mode 100644 index 0000000000000..f19f63536001e --- /dev/null +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/aggregators/metrics/package-info.java @@ -0,0 +1,22 @@ +/* + * 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 classes for integrating with Spark's metrics mechanism (Sinks, Sources, etc.). + */ +package org.apache.beam.runners.spark.aggregators.metrics; diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/aggregators/metrics/sink/CsvSink.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/aggregators/metrics/sink/CsvSink.java new file mode 100644 index 0000000000000..af1601aea8be8 --- /dev/null +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/aggregators/metrics/sink/CsvSink.java @@ -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. + */ + +package org.apache.beam.runners.spark.aggregators.metrics.sink; + +import com.codahale.metrics.MetricRegistry; + +import java.util.Properties; + +import org.apache.beam.runners.spark.aggregators.metrics.AggregatorMetric; +import org.apache.beam.runners.spark.aggregators.metrics.WithNamedAggregatorsSupport; +import org.apache.spark.metrics.sink.Sink; + +/** + * A Spark {@link Sink} that is tailored to report {@link AggregatorMetric} metrics + * to a CSV file. + */ +public class CsvSink extends org.apache.spark.metrics.sink.CsvSink { + public CsvSink(final Properties properties, + final MetricRegistry metricRegistry, + final org.apache.spark.SecurityManager securityMgr) { + super(properties, WithNamedAggregatorsSupport.forRegistry(metricRegistry), securityMgr); + } +} diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/aggregators/metrics/sink/GraphiteSink.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/aggregators/metrics/sink/GraphiteSink.java new file mode 100644 index 0000000000000..7a45ef7f48822 --- /dev/null +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/aggregators/metrics/sink/GraphiteSink.java @@ -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. + */ + +package org.apache.beam.runners.spark.aggregators.metrics.sink; + +import com.codahale.metrics.MetricRegistry; + +import java.util.Properties; + +import org.apache.beam.runners.spark.aggregators.metrics.AggregatorMetric; +import org.apache.beam.runners.spark.aggregators.metrics.WithNamedAggregatorsSupport; +import org.apache.spark.metrics.sink.Sink; + +/** + * A Spark {@link Sink} that is tailored to report {@link AggregatorMetric} metrics + * to Graphite. + */ +public class GraphiteSink extends org.apache.spark.metrics.sink.GraphiteSink { + public GraphiteSink(final Properties properties, + final MetricRegistry metricRegistry, + final org.apache.spark.SecurityManager securityMgr) { + super(properties, WithNamedAggregatorsSupport.forRegistry(metricRegistry), securityMgr); + } +} diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/aggregators/metrics/sink/package-info.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/aggregators/metrics/sink/package-info.java new file mode 100644 index 0000000000000..2e6dd0d3a9dbc --- /dev/null +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/aggregators/metrics/sink/package-info.java @@ -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. + */ + +/** + * Spark sinks that support + * the {@link org.apache.beam.runners.spark.aggregators.metrics.AggregatorMetric}. + */ +package org.apache.beam.runners.spark.aggregators.metrics.sink; 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 eefea77de5d2f..b1c567c757d75 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 @@ -47,7 +47,7 @@ public static Unbound from(int num) { /** * {@link PTransform} writing {@link PCollection} on the console. - * @param + * @param the type of the elements in the {@link PCollection} */ public static class Unbound extends PTransform, PDone> { 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 7b106105ee190..70bec78e51f41 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 @@ -55,8 +55,8 @@ public static Bound from(String filepattern, /** * A {@link PTransform} reading bounded collection of data from HDFS. - * @param - * @param + * @param the type of the keys + * @param the type of the values */ public static class Bound extends PTransform>> { 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 2634c65b24484..4e4cd1a170ce3 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,14 +20,19 @@ 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.AggAccumParam; 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; @@ -41,7 +46,9 @@ 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; /** @@ -63,8 +70,9 @@ public class SparkRuntimeContext implements Serializable { private transient CoderRegistry coderRegistry; SparkRuntimeContext(JavaSparkContext jsc, Pipeline pipeline) { - this.accum = jsc.accumulator(new NamedAggregators(), new AggAccumParam()); - this.serializedPipelineOptions = serializePipelineOptions(pipeline.getOptions()); + final SparkPipelineOptions opts = pipeline.getOptions().as(SparkPipelineOptions.class); + accum = registerMetrics(jsc, opts); + serializedPipelineOptions = serializePipelineOptions(opts); } private static String serializePipelineOptions(PipelineOptions pipelineOptions) { @@ -83,6 +91,23 @@ private static PipelineOptions deserializePipelineOptions(String serializedPipel } } + private Accumulator registerMetrics(final JavaSparkContext jsc, + final SparkPipelineOptions opts) { + final NamedAggregators initialValue = new NamedAggregators(); + final Accumulator accum = jsc.accumulator(initialValue, new AggAccumParam()); + + if (opts.getEnableSparkSinks()) { + final MetricsSystem metricsSystem = SparkEnv$.MODULE$.get().metricsSystem(); + final AggregatorMetricSource aggregatorMetricSource = + new AggregatorMetricSource(initialValue); + // in case the context was not cleared + metricsSystem.removeSource(aggregatorMetricSource); + metricsSystem.registerSource(aggregatorMetricSource); + } + + return accum; + } + /** * Retrieves corresponding value of an aggregator. * diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/util/BroadcastHelper.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/util/BroadcastHelper.java index 5f0c79576e595..5c13b80058943 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/util/BroadcastHelper.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/util/BroadcastHelper.java @@ -58,7 +58,7 @@ public static BroadcastHelper create(T value, Coder coder) { * A {@link BroadcastHelper} that relies on the underlying * Spark serialization (Kryo) to broadcast values. This is appropriate when * broadcasting very large values, since no copy of the object is made. - * @param + * @param the type of the value stored in the broadcast variable */ static class DirectBroadcastHelper extends BroadcastHelper { private Broadcast bcast; @@ -86,7 +86,7 @@ public void broadcast(JavaSparkContext jsc) { * A {@link BroadcastHelper} that uses a * {@link Coder} to encode values as byte arrays * before broadcasting. - * @param + * @param the type of the value stored in the broadcast variable */ static class CodedBroadcastHelper extends BroadcastHelper { private Broadcast bcast; diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/InMemoryMetricsSinkRule.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/InMemoryMetricsSinkRule.java new file mode 100644 index 0000000000000..506dbbdeed7d0 --- /dev/null +++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/InMemoryMetricsSinkRule.java @@ -0,0 +1,32 @@ +/* + * 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.runners.spark.aggregators.metrics.sink.InMemoryMetrics; +import org.junit.rules.ExternalResource; + +/** + * A rule that cleans the {@link InMemoryMetrics} after the tests has finished. + */ +class InMemoryMetricsSinkRule extends ExternalResource { + @Override + protected void before() throws Throwable { + InMemoryMetrics.clearAll(); + } +} diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/SimpleWordCountTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/SimpleWordCountTest.java index f644765682f18..8b7762fd73364 100644 --- a/runners/spark/src/test/java/org/apache/beam/runners/spark/SimpleWordCountTest.java +++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/SimpleWordCountTest.java @@ -19,6 +19,8 @@ package org.apache.beam.runners.spark; import static org.hamcrest.Matchers.containsInAnyOrder; +import static org.hamcrest.Matchers.is; +import static org.hamcrest.Matchers.nullValue; import static org.junit.Assert.assertThat; import com.google.common.collect.ImmutableSet; @@ -27,6 +29,7 @@ import java.util.Arrays; import java.util.List; import java.util.Set; +import org.apache.beam.runners.spark.aggregators.metrics.sink.InMemoryMetrics; import org.apache.beam.runners.spark.examples.WordCount; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.coders.StringUtf8Coder; @@ -39,12 +42,17 @@ import org.apache.commons.io.FileUtils; import org.junit.Rule; import org.junit.Test; +import org.junit.rules.ExternalResource; import org.junit.rules.TemporaryFolder; /** * Simple word count test. */ public class SimpleWordCountTest { + + @Rule + public ExternalResource inMemoryMetricsSink = new InMemoryMetricsSinkRule(); + private static final String[] WORDS_ARRAY = { "hi there", "hi", "hi sue bob", "hi sue", "", "bob hi"}; @@ -54,6 +62,8 @@ public class SimpleWordCountTest { @Test public void testInMem() throws Exception { + assertThat(InMemoryMetrics.valueOf("emptyLines"), is(nullValue())); + SparkPipelineOptions options = PipelineOptionsFactory.as(SparkPipelineOptions.class); options.setRunner(SparkRunner.class); Pipeline p = Pipeline.create(options); @@ -66,6 +76,8 @@ public void testInMem() throws Exception { EvaluationResult res = (EvaluationResult) p.run(); res.close(); + + assertThat(InMemoryMetrics.valueOf("emptyLines"), is(1d)); } @Rule diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/aggregators/metrics/sink/InMemoryMetrics.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/aggregators/metrics/sink/InMemoryMetrics.java new file mode 100644 index 0000000000000..35e67173cba8a --- /dev/null +++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/aggregators/metrics/sink/InMemoryMetrics.java @@ -0,0 +1,79 @@ +/* + * 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.metrics.sink; + +import com.codahale.metrics.MetricFilter; +import com.codahale.metrics.MetricRegistry; + +import java.util.Properties; + +import org.apache.beam.runners.spark.aggregators.metrics.WithNamedAggregatorsSupport; +import org.apache.spark.metrics.sink.Sink; + +/** + * An in-memory {@link Sink} implementation for tests. + */ +public class InMemoryMetrics implements Sink { + + private static WithNamedAggregatorsSupport extendedMetricsRegistry; + private static MetricRegistry internalMetricRegistry; + + public InMemoryMetrics(final Properties properties, + final MetricRegistry metricRegistry, + final org.apache.spark.SecurityManager securityMgr) { + extendedMetricsRegistry = WithNamedAggregatorsSupport.forRegistry(metricRegistry); + internalMetricRegistry = metricRegistry; + } + + @SuppressWarnings("unchecked") + public static T valueOf(final String name) { + final T retVal; + + if (extendedMetricsRegistry != null + && extendedMetricsRegistry.getGauges().containsKey(name)) { + retVal = (T) extendedMetricsRegistry.getGauges().get(name).getValue(); + } else { + retVal = null; + } + + return retVal; + } + + public static void clearAll() { + if (internalMetricRegistry != null) { + internalMetricRegistry.removeMatching(MetricFilter.ALL); + } + } + + @Override + public void start() { + + } + + @Override + public void stop() { + + } + + @Override + public void report() { + + } + +} diff --git a/runners/spark/src/test/resources/metrics.properties b/runners/spark/src/test/resources/metrics.properties new file mode 100644 index 0000000000000..4aa01d2dfd4fe --- /dev/null +++ b/runners/spark/src/test/resources/metrics.properties @@ -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. + +*.sink.memory.class=org.apache.beam.runners.spark.aggregators.metrics.sink.InMemoryMetrics + +#*.sink.csv.class=org.apache.beam.runners.spark.aggregators.metrics.sink.CsvSink +#*.sink.csv.directory=/tmp/spark-metrics +#*.sink.csv.period=1 +#*.sink.graphite.unit=SECONDS + +#*.sink.graphite.class=org.apache.beam.runners.spark.aggregators.metrics.sink.GraphiteSink +#*.sink.graphite.host=YOUR_HOST +#*.sink.graphite.port=2003 +#*.sink.graphite.prefix=spark +#*.sink.graphite.period=1 +#*.sink.graphite.unit=SECONDS From 0fbd9c8c41b770b176886540a7083a166b2485b6 Mon Sep 17 00:00:00 2001 From: Dan Halperin Date: Tue, 23 Aug 2016 20:21:30 -0700 Subject: [PATCH 077/346] travis.yml: disable updating snapshots Will still update releases with the --update-snapshots version (I know, bad name) --- .travis.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.travis.yml b/.travis.yml index 656aba0fdf4be..233128266f1c3 100644 --- a/.travis.yml +++ b/.travis.yml @@ -52,7 +52,7 @@ install: - rm -rf "$HOME/.m2/repository/org/apache/beam" script: - - travis_retry mvn --settings .travis/settings.xml --batch-mode --update-snapshots $MAVEN_OVERRIDE verify + - travis_retry mvn --settings .travis/settings.xml --batch-mode --update-snapshots --no-snapshot-updates $MAVEN_OVERRIDE verify - travis_retry .travis/test_wordcount.sh cache: From 79491ebe04e54043ebdf5afa6be78718eae0fe15 Mon Sep 17 00:00:00 2001 From: Vikas Kedigehalli Date: Tue, 23 Aug 2016 16:44:08 -0700 Subject: [PATCH 078/346] Query latest timestamp --- .../sdk/io/gcp/datastore/DatastoreV1.java | 41 +++++++++++-- .../sdk/io/gcp/datastore/DatastoreV1Test.java | 60 ++++++++++++++++--- 2 files changed, 88 insertions(+), 13 deletions(-) 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 c7433d37d1607..8456e0287def3 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 @@ -239,6 +239,7 @@ static int getEstimatedNumSplits(Datastore datastore, Query query, @Nullable Str int numSplits; try { long estimatedSizeBytes = getEstimatedSizeBytes(datastore, query, namespace); + LOG.info("Estimated size bytes for the query is: {}", estimatedSizeBytes); numSplits = (int) Math.min(NUM_QUERY_SPLITS_MAX, Math.round(((double) estimatedSizeBytes) / DEFAULT_BUNDLE_SIZE_BYTES)); } catch (Exception e) { @@ -249,6 +250,33 @@ static int getEstimatedNumSplits(Datastore datastore, Query query, @Nullable Str return Math.max(numSplits, NUM_QUERY_SPLITS_MIN); } + /** + * Datastore system tables with statistics are periodically updated. This method fetches + * the latest timestamp (in microseconds) of statistics update using the {@code __Stat_Total__} + * table. + */ + private static long queryLatestStatisticsTimestamp(Datastore datastore, + @Nullable String namespace) throws DatastoreException { + Query.Builder query = Query.newBuilder(); + if (namespace == null) { + query.addKindBuilder().setName("__Stat_Total__"); + } else { + query.addKindBuilder().setName("__Stat_Ns_Total__"); + } + query.addOrder(makeOrder("timestamp", DESCENDING)); + query.setLimit(Int32Value.newBuilder().setValue(1)); + RunQueryRequest request = makeRequest(query.build(), namespace); + + RunQueryResponse response = datastore.runQuery(request); + QueryResultBatch batch = response.getBatch(); + if (batch.getEntityResultsCount() == 0) { + throw new NoSuchElementException( + "Datastore total statistics unavailable"); + } + Entity entity = batch.getEntityResults(0).getEntity(); + return entity.getProperties().get("timestamp").getTimestampValue().getSeconds() * 1000000; + } + /** * Get the estimated size of the data returned by the given query. * @@ -261,17 +289,17 @@ static int getEstimatedNumSplits(Datastore datastore, Query query, @Nullable Str static long getEstimatedSizeBytes(Datastore datastore, Query query, @Nullable String namespace) throws DatastoreException { String ourKind = query.getKind(0).getName(); + long latestTimestamp = queryLatestStatisticsTimestamp(datastore, namespace); + LOG.info("Latest stats timestamp : {}", latestTimestamp); + Query.Builder queryBuilder = Query.newBuilder(); if (namespace == null) { queryBuilder.addKindBuilder().setName("__Stat_Kind__"); } else { - queryBuilder.addKindBuilder().setName("__Ns_Stat_Kind__"); + queryBuilder.addKindBuilder().setName("__Stat_Ns_Kind__"); } queryBuilder.setFilter(makeFilter("kind_name", EQUAL, makeValue(ourKind).build())); - - // Get the latest statistics - queryBuilder.addOrder(makeOrder("timestamp", DESCENDING)); - queryBuilder.setLimit(Int32Value.newBuilder().setValue(1)); + queryBuilder.setFilter(makeFilter("timestamp", EQUAL, makeValue(latestTimestamp).build())); RunQueryRequest request = makeRequest(queryBuilder.build(), namespace); @@ -547,6 +575,7 @@ public void processElement(ProcessContext c) throws Exception { estimatedNumSplits = numSplits; } + LOG.info("Splitting the query into {} splits", estimatedNumSplits); List querySplits; try { querySplits = splitQuery(query, options.getNamespace(), datastore, querySplitter, @@ -866,7 +895,7 @@ public void processElement(ProcessContext c) throws Exception { @FinishBundle public void finishBundle(Context c) throws Exception { - if (mutations.size() > 0) { + if (!mutations.isEmpty()) { flushBatch(); } } diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/DatastoreV1Test.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/DatastoreV1Test.java index ab1df2f3e9f5d..138671d4a0078 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/DatastoreV1Test.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/DatastoreV1Test.java @@ -61,6 +61,7 @@ import com.google.datastore.v1.client.QuerySplitter; import com.google.protobuf.Int32Value; import java.util.ArrayList; +import java.util.Date; import java.util.HashSet; import java.util.LinkedList; import java.util.List; @@ -561,14 +562,23 @@ private void datastoreWriterFnTest(int numMutations) throws Exception { @Test public void testEstimatedSizeBytes() throws Exception { long entityBytes = 100L; + // In seconds + long timestamp = 1234L; + + RunQueryRequest latestTimestampRequest = makeRequest(makeLatestTimestampQuery(NAMESPACE), + NAMESPACE); + RunQueryResponse latestTimestampResponse = makeLatestTimestampResponse(timestamp); // Per Kind statistics request and response - RunQueryRequest statRequest = makeRequest(makeStatKindQuery(NAMESPACE), NAMESPACE); + RunQueryRequest statRequest = makeRequest(makeStatKindQuery(NAMESPACE, timestamp), NAMESPACE); RunQueryResponse statResponse = makeStatKindResponse(entityBytes); + when(mockDatastore.runQuery(latestTimestampRequest)) + .thenReturn(latestTimestampResponse); when(mockDatastore.runQuery(statRequest)) .thenReturn(statResponse); assertEquals(entityBytes, getEstimatedSizeBytes(mockDatastore, QUERY, NAMESPACE)); + verify(mockDatastore, times(1)).runQuery(latestTimestampRequest); verify(mockDatastore, times(1)).runQuery(statRequest); } @@ -609,11 +619,19 @@ public void testSplitQueryFnWithoutNumSplits() throws Exception { int numSplits = 0; int expectedNumSplits = 20; long entityBytes = expectedNumSplits * DEFAULT_BUNDLE_SIZE_BYTES; + // In seconds + long timestamp = 1234L; + + RunQueryRequest latestTimestampRequest = makeRequest(makeLatestTimestampQuery(NAMESPACE), + NAMESPACE); + RunQueryResponse latestTimestampResponse = makeLatestTimestampResponse(timestamp); // Per Kind statistics request and response - RunQueryRequest statRequest = makeRequest(makeStatKindQuery(NAMESPACE), NAMESPACE); + RunQueryRequest statRequest = makeRequest(makeStatKindQuery(NAMESPACE, timestamp), NAMESPACE); RunQueryResponse statResponse = makeStatKindResponse(entityBytes); + when(mockDatastore.runQuery(latestTimestampRequest)) + .thenReturn(latestTimestampResponse); when(mockDatastore.runQuery(statRequest)) .thenReturn(statResponse); when(mockQuerySplitter.getSplits( @@ -629,6 +647,7 @@ public void testSplitQueryFnWithoutNumSplits() throws Exception { verifyUniqueKeys(queries); verify(mockQuerySplitter, times(1)).getSplits( eq(QUERY), any(PartitionId.class), eq(expectedNumSplits), any(Datastore.class)); + verify(mockDatastore, times(1)).runQuery(latestTimestampRequest); verify(mockDatastore, times(1)).runQuery(statRequest); } @@ -752,7 +771,7 @@ public RunQueryResponse answer(InvocationOnMock invocationOnMock) throws Throwab /** Builds a per-kind statistics response with the given entity size. */ private static RunQueryResponse makeStatKindResponse(long entitySizeInBytes) { - RunQueryResponse.Builder timestampResponse = RunQueryResponse.newBuilder(); + RunQueryResponse.Builder statKindResponse = RunQueryResponse.newBuilder(); Entity.Builder entity = Entity.newBuilder(); entity.setKey(makeKey("dummyKind", "dummyId")); entity.getMutableProperties().put("entity_bytes", makeValue(entitySizeInBytes).build()); @@ -760,24 +779,51 @@ private static RunQueryResponse makeStatKindResponse(long entitySizeInBytes) { entityResult.setEntity(entity); QueryResultBatch.Builder batch = QueryResultBatch.newBuilder(); batch.addEntityResults(entityResult); + statKindResponse.setBatch(batch); + return statKindResponse.build(); + } + + /** Builds a response of the given timestamp. */ + private static RunQueryResponse makeLatestTimestampResponse(long timestamp) { + RunQueryResponse.Builder timestampResponse = RunQueryResponse.newBuilder(); + Entity.Builder entity = Entity.newBuilder(); + entity.setKey(makeKey("dummyKind", "dummyId")); + entity.getMutableProperties().put("timestamp", makeValue(new Date(timestamp * 1000)).build()); + EntityResult.Builder entityResult = EntityResult.newBuilder(); + entityResult.setEntity(entity); + QueryResultBatch.Builder batch = QueryResultBatch.newBuilder(); + batch.addEntityResults(entityResult); timestampResponse.setBatch(batch); return timestampResponse.build(); } /** Builds a per-kind statistics query for the given timestamp and namespace. */ - private static Query makeStatKindQuery(String namespace) { + private static Query makeStatKindQuery(String namespace, long timestamp) { Query.Builder statQuery = Query.newBuilder(); if (namespace == null) { statQuery.addKindBuilder().setName("__Stat_Kind__"); } else { - statQuery.addKindBuilder().setName("__Ns_Stat_Kind__"); + statQuery.addKindBuilder().setName("__Stat_Ns_Kind__"); } statQuery.setFilter(makeFilter("kind_name", EQUAL, makeValue(KIND)).build()); - statQuery.addOrder(makeOrder("timestamp", DESCENDING)); - statQuery.setLimit(Int32Value.newBuilder().setValue(1)); + statQuery.setFilter(makeFilter("timestamp", EQUAL, makeValue(timestamp * 1000000L)).build()); return statQuery.build(); } + /** Builds a latest timestamp statistics query. */ + private static Query makeLatestTimestampQuery(String namespace) { + Query.Builder timestampQuery = Query.newBuilder(); + if (namespace == null) { + timestampQuery.addKindBuilder().setName("__Stat_Total__"); + } else { + timestampQuery.addKindBuilder().setName("__Stat_Ns_Total__"); + } + timestampQuery.addOrder(makeOrder("timestamp", DESCENDING)); + timestampQuery.setLimit(Int32Value.newBuilder().setValue(1)); + return timestampQuery.build(); + } + + /** Generate dummy query splits. */ private List splitQuery(Query query, int numSplits) { List queries = new LinkedList<>(); From 4023167b30f7a2b282f45073d4f9dd9cc532236d Mon Sep 17 00:00:00 2001 From: gaurav gupta Date: Thu, 25 Aug 2016 14:00:06 -0700 Subject: [PATCH 079/346] [BEAM-589] Fixing IO.Read transformation --- .../org/apache/beam/examples/complete/TfIdf.java | 6 +++--- .../runners/core/UnboundedReadFromBoundedSource.java | 6 +++--- .../apache/beam/runners/flink/examples/TFIDF.java | 6 +++--- .../apache/beam/runners/dataflow/DataflowRunner.java | 6 +++--- .../DataflowUnboundedReadFromBoundedSource.java | 6 +++--- .../beam/runners/dataflow/DataflowRunnerTest.java | 4 ++-- .../apache/beam/runners/spark/io/CreateStream.java | 7 +++---- .../org/apache/beam/runners/spark/io/KafkaIO.java | 6 +++--- .../beam/runners/spark/io/hadoop/HadoopIO.java | 6 +++--- .../src/main/java/org/apache/beam/sdk/io/AvroIO.java | 6 +++--- .../beam/sdk/io/BoundedReadFromUnboundedSource.java | 6 +++--- .../main/java/org/apache/beam/sdk/io/PubsubIO.java | 6 +++--- .../src/main/java/org/apache/beam/sdk/io/Read.java | 10 +++++----- .../src/main/java/org/apache/beam/sdk/io/TextIO.java | 6 +++--- .../java/org/apache/beam/sdk/transforms/Create.java | 12 ++++++------ .../test/java/org/apache/beam/sdk/io/AvroIOTest.java | 2 +- .../java/org/apache/beam/sdk/io/PubsubIOTest.java | 2 +- .../test/java/org/apache/beam/sdk/io/TextIOTest.java | 2 +- .../apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java | 8 ++++---- .../beam/sdk/io/gcp/bigquery/BigQueryIOTest.java | 4 ++-- .../main/java/org/apache/beam/sdk/io/jms/JmsIO.java | 3 +-- .../java/org/apache/beam/sdk/io/kafka/KafkaIO.java | 3 +-- 22 files changed, 60 insertions(+), 63 deletions(-) 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 87023edac29ac..6684553c41eb0 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 @@ -51,11 +51,11 @@ import org.apache.beam.sdk.util.GcsUtil; import org.apache.beam.sdk.util.gcsfs.GcsPath; import org.apache.beam.sdk.values.KV; +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.TupleTag; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -152,7 +152,7 @@ public static Set listInputDocuments(Options options) * from the documents tagged with which document they are from. */ public static class ReadDocuments - extends PTransform>> { + extends PTransform>> { private Iterable uris; public ReadDocuments(Iterable uris) { @@ -165,7 +165,7 @@ public Coder getDefaultOutputCoder() { } @Override - public PCollection> apply(PInput input) { + public PCollection> apply(PBegin input) { Pipeline pipeline = input.getPipeline(); // Create one TextIO.Read transform for each document 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 73688d45592e9..91a17158fa0c0 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 @@ -51,8 +51,8 @@ import org.apache.beam.sdk.transforms.display.DisplayData; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.util.PropertyNames; +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.TimestampedValue; import org.joda.time.Instant; import org.slf4j.Logger; @@ -74,7 +74,7 @@ *

              This transform is intended to be used by a runner during pipeline translation to convert * a Read.Bounded into a Read.Unbounded. */ -public class UnboundedReadFromBoundedSource extends PTransform> { +public class UnboundedReadFromBoundedSource extends PTransform> { private static final Logger LOG = LoggerFactory.getLogger(UnboundedReadFromBoundedSource.class); @@ -88,7 +88,7 @@ public UnboundedReadFromBoundedSource(BoundedSource source) { } @Override - public PCollection apply(PInput input) { + public PCollection apply(PBegin input) { return input.getPipeline().apply( Read.from(new BoundedToUnboundedSourceAdapter<>(source))); } 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 0ca94a13463bd..a92d3397cdfd4 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 @@ -53,11 +53,11 @@ import org.apache.beam.sdk.util.GcsUtil; import org.apache.beam.sdk.util.gcsfs.GcsPath; import org.apache.beam.sdk.values.KV; +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.TupleTag; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -154,7 +154,7 @@ public static Set listInputDocuments(Options options) * from the documents tagged with which document they are from. */ public static class ReadDocuments - extends PTransform>> { + extends PTransform>> { private static final long serialVersionUID = 0; private Iterable uris; @@ -169,7 +169,7 @@ public Coder getDefaultOutputCoder() { } @Override - public PCollection> apply(PInput input) { + public PCollection> apply(PBegin input) { Pipeline pipeline = input.getPipeline(); // Create one TextIO.Read transform for each document 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 a0e24b14a5755..0ce4b58aa6acd 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 @@ -400,7 +400,7 @@ public OutputT apply( return windowed; } else if (Flatten.FlattenPCollectionList.class.equals(transform.getClass()) && ((PCollectionList) input).size() == 0) { - return (OutputT) Pipeline.applyTransform(input, Create.of()); + return (OutputT) Pipeline.applyTransform((PBegin) input, Create.of()); } else if (overrides.containsKey(transform.getClass())) { // It is the responsibility of whoever constructs overrides to ensure this is type safe. @SuppressWarnings("unchecked") @@ -2318,7 +2318,7 @@ public void processElement(ProcessContext c) { * Specialized implementation for {@link org.apache.beam.sdk.io.Read.Bounded Read.Bounded} for the * Dataflow runner in streaming mode. */ - private static class StreamingBoundedRead extends PTransform> { + private static class StreamingBoundedRead extends PTransform> { private final BoundedSource source; /** Builds an instance of this class from the overridden transform. */ @@ -2333,7 +2333,7 @@ protected Coder getDefaultOutputCoder() { } @Override - public final PCollection apply(PInput input) { + public final PCollection apply(PBegin input) { source.validate(); return Pipeline.applyTransform(input, new DataflowUnboundedReadFromBoundedSource<>(source)) 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 85f5e734a2537..866da13ccab6e 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 @@ -51,8 +51,8 @@ import org.apache.beam.sdk.transforms.display.DisplayData; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.util.PropertyNames; +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.TimestampedValue; import org.joda.time.Instant; import org.slf4j.Logger; @@ -78,7 +78,7 @@ * time dependency. It should be replaced in the dataflow worker as an execution time dependency. */ @Deprecated -public class DataflowUnboundedReadFromBoundedSource extends PTransform> { +public class DataflowUnboundedReadFromBoundedSource extends PTransform> { private static final Logger LOG = LoggerFactory.getLogger(DataflowUnboundedReadFromBoundedSource.class); @@ -93,7 +93,7 @@ public DataflowUnboundedReadFromBoundedSource(BoundedSource source) { } @Override - public PCollection apply(PInput input) { + public PCollection apply(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/DataflowRunnerTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/DataflowRunnerTest.java index 208e84c966789..58a01aa475f06 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 @@ -101,8 +101,8 @@ import org.apache.beam.sdk.util.WindowingStrategy; import org.apache.beam.sdk.util.gcsfs.GcsPath; import org.apache.beam.sdk.values.KV; +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.TimestampedValue; import org.apache.beam.sdk.values.TupleTag; import org.hamcrest.Description; @@ -970,7 +970,7 @@ private static PipelineOptions makeOptions(boolean streaming) { return options; } - private void testUnsupportedSource(PTransform source, String name, boolean streaming) + private void testUnsupportedSource(PTransform source, String name, boolean streaming) throws Exception { String mode = streaming ? "streaming" : "batch"; thrown.expect(UnsupportedOperationException.class); 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 b3beae6c66315..a08c54ed133e8 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 @@ -21,9 +21,8 @@ import org.apache.beam.sdk.transforms.PTransform; 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.PInput; - /** * Create an input stream from Queue. @@ -49,7 +48,7 @@ public static QueuedValues fromQueue(Iterable> queuedValues) /** * {@link PTransform} for queueing values. */ - public static final class QueuedValues extends PTransform> { + public static final class QueuedValues extends PTransform> { private final Iterable> queuedValues; @@ -64,7 +63,7 @@ public Iterable> getQueuedValues() { } @Override - public PCollection apply(PInput input) { + public PCollection apply(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/KafkaIO.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/io/KafkaIO.java index f57c114e69fd1..8cf20830bc756 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/io/KafkaIO.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/io/KafkaIO.java @@ -25,8 +25,8 @@ import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.util.WindowingStrategy; import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.PBegin; import org.apache.beam.sdk.values.PCollection; -import org.apache.beam.sdk.values.PInput; /** * Read stream from Kafka. @@ -68,7 +68,7 @@ public static Unbound from(Class> keyDecoder, /** * A {@link PTransform} reading from Kafka topics and providing {@link PCollection}. */ - public static class Unbound extends PTransform>> { + public static class Unbound extends PTransform>> { private final Class> keyDecoderClass; private final Class> valueDecoderClass; @@ -120,7 +120,7 @@ public Map getKafkaParams() { } @Override - public PCollection> apply(PInput input) { + public PCollection> apply(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 70bec78e51f41..042c316c19192 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 @@ -26,9 +26,9 @@ import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.util.WindowingStrategy; import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.PBegin; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PDone; -import org.apache.beam.sdk.values.PInput; import org.apache.hadoop.mapreduce.lib.input.FileInputFormat; import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat; @@ -58,7 +58,7 @@ public static Bound from(String filepattern, * @param the type of the keys * @param the type of the values */ - public static class Bound extends PTransform>> { + public static class Bound extends PTransform>> { private final String filepattern; private final Class> formatClass; @@ -94,7 +94,7 @@ public Class getKeyClass() { } @Override - public PCollection> apply(PInput input) { + public PCollection> apply(PBegin input) { return PCollection.createPrimitiveOutputInternal(input.getPipeline(), WindowingStrategy.globalDefault(), PCollection.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 e7c302bed1d70..267265db1daeb 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 @@ -40,9 +40,9 @@ import org.apache.beam.sdk.transforms.display.DisplayData; import org.apache.beam.sdk.util.IOChannelUtils; import org.apache.beam.sdk.util.MimeTypes; +import org.apache.beam.sdk.values.PBegin; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PDone; -import org.apache.beam.sdk.values.PInput; /** * {@link PTransform}s for reading and writing Avro files. @@ -184,7 +184,7 @@ public static Bound withoutValidation() { * @param the type of each of the elements of the resulting * PCollection */ - public static class Bound extends PTransform> { + public static class Bound extends PTransform> { /** The filepattern to read from. */ @Nullable final String filepattern; @@ -270,7 +270,7 @@ public Bound withoutValidation() { } @Override - public PCollection apply(PInput input) { + public PCollection apply(PBegin input) { if (filepattern == null) { throw new IllegalStateException( "need to set the filepattern of an AvroIO.Read 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 ede65a96cc998..28d77468bbc0e 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 @@ -36,8 +36,8 @@ import org.apache.beam.sdk.transforms.display.DisplayData; import org.apache.beam.sdk.util.IntervalBoundedExponentialBackOff; 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.joda.time.Duration; import org.joda.time.Instant; @@ -48,7 +48,7 @@ * *

              Created by {@link Read}. */ -class BoundedReadFromUnboundedSource extends PTransform> { +class BoundedReadFromUnboundedSource extends PTransform> { private final UnboundedSource source; private final long maxNumRecords; private final Duration maxReadTime; @@ -82,7 +82,7 @@ public BoundedReadFromUnboundedSource withMaxReadTime(Duration maxReadTime) { } @Override - public PCollection apply(PInput input) { + public PCollection apply(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/PubsubIO.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/PubsubIO.java index b137f1551010a..d11345780295e 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 @@ -46,9 +46,9 @@ import org.apache.beam.sdk.util.PubsubClient.SubscriptionPath; import org.apache.beam.sdk.util.PubsubClient.TopicPath; import org.apache.beam.sdk.util.PubsubJsonClient; +import org.apache.beam.sdk.values.PBegin; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PDone; -import org.apache.beam.sdk.values.PInput; import org.joda.time.Duration; import org.joda.time.Instant; import org.slf4j.Logger; @@ -481,7 +481,7 @@ public static Bound maxReadTime(Duration maxReadTime) { * A {@link PTransform} that reads from a Cloud Pub/Sub source and returns * a unbounded {@link PCollection} containing the items from the stream. */ - public static class Bound extends PTransform> { + public static class Bound extends PTransform> { /** The Cloud Pub/Sub topic to read from. */ @Nullable private final PubsubTopic topic; @@ -610,7 +610,7 @@ public Bound maxReadTime(Duration maxReadTime) { } @Override - public PCollection apply(PInput input) { + public PCollection apply(PBegin input) { if (topic == null && subscription == null) { throw new IllegalStateException("Need to set either the topic or the subscription for " + "a PubsubIO.Read transform"); 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 f99877d84d29a..29c4e47e64ec7 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 @@ -25,9 +25,9 @@ import org.apache.beam.sdk.transforms.display.DisplayData; import org.apache.beam.sdk.util.SerializableUtils; 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.PInput; import org.joda.time.Duration; /** @@ -87,7 +87,7 @@ public Unbounded from(UnboundedSource source) { /** * {@link PTransform} that reads from a {@link BoundedSource}. */ - public static class Bounded extends PTransform> { + public static class Bounded extends PTransform> { private final BoundedSource source; private Bounded(@Nullable String name, BoundedSource source) { @@ -101,7 +101,7 @@ protected Coder getDefaultOutputCoder() { } @Override - public final PCollection apply(PInput input) { + public final PCollection apply(PBegin input) { source.validate(); return PCollection.createPrimitiveOutputInternal(input.getPipeline(), @@ -134,7 +134,7 @@ public void populateDisplayData(DisplayData.Builder builder) { /** * {@link PTransform} that reads from a {@link UnboundedSource}. */ - public static class Unbounded extends PTransform> { + public static class Unbounded extends PTransform> { private final UnboundedSource source; private Unbounded(@Nullable String name, UnboundedSource source) { @@ -169,7 +169,7 @@ protected Coder getDefaultOutputCoder() { } @Override - public final PCollection apply(PInput input) { + public final PCollection apply(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 ed9a62790afe7..242470b587926 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 @@ -43,9 +43,9 @@ import org.apache.beam.sdk.transforms.display.DisplayData; import org.apache.beam.sdk.util.IOChannelUtils; import org.apache.beam.sdk.util.MimeTypes; +import org.apache.beam.sdk.values.PBegin; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PDone; -import org.apache.beam.sdk.values.PInput; /** * {@link PTransform}s for reading and writing text files. @@ -189,7 +189,7 @@ public static Bound withCompressionType(TextIO.CompressionType compressi * may use {@link #withCoder(Coder)} to supply a {@code Coder} to produce a * {@code PCollection} instead. */ - public static class Bound extends PTransform> { + public static class Bound extends PTransform> { /** The filepattern to read from. */ @Nullable private final String filepattern; @@ -269,7 +269,7 @@ public Bound withCompressionType(TextIO.CompressionType compressionType) { } @Override - public PCollection apply(PInput input) { + public PCollection apply(PBegin input) { if (filepattern == null) { throw new IllegalStateException("need to set the filepattern of a TextIO.Read transform"); } 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 e261db2b30f91..7cd47117565db 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 @@ -46,8 +46,8 @@ import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.util.CoderUtils; import org.apache.beam.sdk.values.KV; +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.TimestampedValue; import org.apache.beam.sdk.values.TimestampedValue.TimestampedValueCoder; import org.apache.beam.sdk.values.TypeDescriptor; @@ -218,7 +218,7 @@ public static TimestampedValues timestamped( /** * A {@code PTransform} that creates a {@code PCollection} from a set of in-memory objects. */ - public static class Values extends PTransform> { + public static class Values extends PTransform> { /** * Returns a {@link Create.Values} PTransform like this one that uses the given * {@code Coder} to decode each of the objects into a @@ -240,7 +240,7 @@ public Iterable getElements() { } @Override - public PCollection apply(PInput input) { + public PCollection apply(PBegin input) { try { Coder coder = getDefaultOutputCoder(input); try { @@ -257,7 +257,7 @@ public PCollection apply(PInput input) { } @Override - public Coder getDefaultOutputCoder(PInput input) throws CannotProvideCoderException { + public Coder getDefaultOutputCoder(PBegin input) throws CannotProvideCoderException { if (coder.isPresent()) { return coder.get(); } else { @@ -421,7 +421,7 @@ protected boolean advanceImpl() throws IOException { * A {@code PTransform} that creates a {@code PCollection} whose elements have * associated timestamps. */ - public static class TimestampedValues extends PTransform>{ + public static class TimestampedValues extends PTransform>{ /** * Returns a {@link Create.TimestampedValues} PTransform like this one that uses the given * {@code Coder} to decode each of the objects into a @@ -440,7 +440,7 @@ public TimestampedValues withCoder(Coder coder) { } @Override - public PCollection apply(PInput input) { + public PCollection apply(PBegin input) { try { Iterable rawElements = Iterables.transform( 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 a8a7746f214bb..81f05d7cfbc9b 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 @@ -291,7 +291,7 @@ public void testPrimitiveReadDisplayData() { .withSchema(Schema.create(Schema.Type.STRING)) .withoutValidation(); - Set displayData = evaluator.displayDataForPrimitiveTransforms(read); + Set displayData = evaluator.displayDataForPrimitiveSourceTransforms(read); assertThat("AvroIO.Read should include the file pattern in its primitive transform", displayData, hasItem(hasDisplayItem("filePattern"))); } 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 4067055b25c75..086b726445817 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 @@ -111,7 +111,7 @@ public void testPrimitiveReadDisplayData() { PubsubIO.Read.subscription("projects/project/subscriptions/subscription") .maxNumRecords(1); - Set displayData = evaluator.displayDataForPrimitiveTransforms(read); + Set displayData = evaluator.displayDataForPrimitiveSourceTransforms(read); assertThat("PubsubIO.Read should include the subscription in its primitive display data", displayData, hasItem(hasDisplayItem("subscription"))); } 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 358a30f3d2b81..8f94766063022 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 @@ -210,7 +210,7 @@ public void testPrimitiveReadDisplayData() { .from("foobar") .withoutValidation(); - Set displayData = evaluator.displayDataForPrimitiveTransforms(read); + Set displayData = evaluator.displayDataForPrimitiveSourceTransforms(read); assertThat("TextIO.Read should include the file prefix in its primitive display data", displayData, hasItem(hasDisplayItem(hasValue(startsWith("foobar"))))); } 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 01a8a1c5cd4e8..304dc820ac9da 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 @@ -122,11 +122,11 @@ import org.apache.beam.sdk.util.Transport; import org.apache.beam.sdk.util.gcsfs.GcsPath; import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.PBegin; 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.PDone; -import org.apache.beam.sdk.values.PInput; import org.apache.beam.sdk.values.TupleTag; import org.apache.beam.sdk.values.TupleTagList; import org.joda.time.Instant; @@ -377,7 +377,7 @@ public static Bound withoutValidation() { * A {@link PTransform} that reads from a BigQuery table and returns a bounded * {@link PCollection} of {@link TableRow TableRows}. */ - public static class Bound extends PTransform> { + public static class Bound extends PTransform> { @Nullable final String jsonTableRef; @Nullable final String query; @@ -480,7 +480,7 @@ Bound withTestServices(BigQueryServices testServices) { } @Override - public void validate(PInput input) { + public void validate(PBegin input) { // Even if existence validation is disabled, we need to make sure that the BigQueryIO // read is properly specified. BigQueryOptions bqOptions = input.getPipeline().getOptions().as(BigQueryOptions.class); @@ -524,7 +524,7 @@ public void validate(PInput input) { } @Override - public PCollection apply(PInput input) { + public PCollection apply(PBegin input) { String uuid = randomUUIDString(); final String jobIdToken = "beam_job_" + uuid; 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 7a7575bfdd24c..57eb4ffb2394e 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 @@ -671,7 +671,7 @@ public void testTableSourcePrimitiveDisplayData() throws IOException, Interrupte .withJobService(mockJobService)) .withoutValidation(); - Set displayData = evaluator.displayDataForPrimitiveTransforms(read); + Set displayData = evaluator.displayDataForPrimitiveSourceTransforms(read); assertThat("BigQueryIO.Read should include the table spec in its primitive display data", displayData, hasItem(hasDisplayItem("table"))); } @@ -688,7 +688,7 @@ public void testQuerySourcePrimitiveDisplayData() throws IOException, Interrupte .withJobService(mockJobService)) .withoutValidation(); - Set displayData = evaluator.displayDataForPrimitiveTransforms(read); + Set displayData = evaluator.displayDataForPrimitiveSourceTransforms(read); assertThat("BigQueryIO.Read should include the query in its primitive display data", displayData, hasItem(hasDisplayItem("query"))); } 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 f92dbd456b79f..29d0c5fac1d83 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 @@ -51,7 +51,6 @@ import org.apache.beam.sdk.values.PBegin; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PDone; -import org.apache.beam.sdk.values.PInput; import org.joda.time.Duration; import org.joda.time.Instant; import org.slf4j.Logger; @@ -140,7 +139,7 @@ public PCollection apply(PBegin input) { // handles unbounded source to bounded conversion if maxNumRecords is set. Unbounded unbounded = org.apache.beam.sdk.io.Read.from(createSource()); - PTransform> transform = unbounded; + PTransform> transform = unbounded; if (maxNumRecords != Long.MAX_VALUE) { transform = unbounded.withMaxNumRecords(maxNumRecords); 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 885d5d10b4b7b..f6394220ceba7 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 @@ -73,7 +73,6 @@ import org.apache.beam.sdk.values.PBegin; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PDone; -import org.apache.beam.sdk.values.PInput; import org.apache.kafka.clients.consumer.Consumer; import org.apache.kafka.clients.consumer.ConsumerConfig; import org.apache.kafka.clients.consumer.ConsumerRecord; @@ -450,7 +449,7 @@ public PCollection> apply(PBegin input) { Unbounded> unbounded = org.apache.beam.sdk.io.Read.from(makeSource()); - PTransform>> transform = unbounded; + PTransform>> transform = unbounded; if (maxNumRecords < Long.MAX_VALUE) { transform = unbounded.withMaxNumRecords(maxNumRecords); From bce9aefe55226fc795d4920a849a0402b45d5235 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Pastuszka=20Przemys=C5=82aw?= Date: Mon, 18 Jul 2016 20:03:37 +0200 Subject: [PATCH 080/346] kinesis: a connector for Amazon Kinesis --- sdks/java/io/kinesis/pom.xml | 179 +++++++++ .../sdk/io/kinesis/CheckpointGenerator.java | 30 ++ .../beam/sdk/io/kinesis/CustomOptional.java | 85 ++++ .../kinesis/DynamicCheckpointGenerator.java | 58 +++ .../io/kinesis/GetKinesisRecordsResult.java | 54 +++ .../sdk/io/kinesis/KinesisClientProvider.java | 31 ++ .../apache/beam/sdk/io/kinesis/KinesisIO.java | 190 +++++++++ .../beam/sdk/io/kinesis/KinesisReader.java | 145 +++++++ .../io/kinesis/KinesisReaderCheckpoint.java | 98 +++++ .../beam/sdk/io/kinesis/KinesisRecord.java | 121 ++++++ .../sdk/io/kinesis/KinesisRecordCoder.java | 75 ++++ .../beam/sdk/io/kinesis/KinesisSource.java | 114 ++++++ .../beam/sdk/io/kinesis/RecordFilter.java | 41 ++ .../beam/sdk/io/kinesis/RoundRobin.java | 54 +++ .../beam/sdk/io/kinesis/ShardCheckpoint.java | 175 ++++++++ .../sdk/io/kinesis/ShardRecordsIterator.java | 98 +++++ .../io/kinesis/SimplifiedKinesisClient.java | 158 ++++++++ .../beam/sdk/io/kinesis/StartingPoint.java | 85 ++++ .../io/kinesis/StaticCheckpointGenerator.java | 42 ++ .../io/kinesis/TransientKinesisException.java | 29 ++ .../beam/sdk/io/kinesis/package-info.java | 22 + .../sdk/io/kinesis/AmazonKinesisMock.java | 375 ++++++++++++++++++ .../sdk/io/kinesis/CustomOptionalTest.java | 31 ++ .../DynamicCheckpointGeneratorTest.java | 56 +++ .../sdk/io/kinesis/KinesisMockReadTest.java | 92 +++++ .../kinesis/KinesisReaderCheckpointTest.java | 67 ++++ .../beam/sdk/io/kinesis/KinesisReaderIT.java | 119 ++++++ .../sdk/io/kinesis/KinesisReaderTest.java | 119 ++++++ .../io/kinesis/KinesisRecordCoderTest.java | 46 +++ .../sdk/io/kinesis/KinesisTestOptions.java | 47 +++ .../beam/sdk/io/kinesis/KinesisUploader.java | 84 ++++ .../beam/sdk/io/kinesis/RecordFilterTest.java | 66 +++ .../beam/sdk/io/kinesis/RoundRobinTest.java | 57 +++ .../sdk/io/kinesis/ShardCheckpointTest.java | 148 +++++++ .../io/kinesis/ShardRecordsIteratorTest.java | 150 +++++++ .../kinesis/SimplifiedKinesisClientTest.java | 223 +++++++++++ .../beam/sdk/io/kinesis/package-info.java | 22 + sdks/java/io/pom.xml | 1 + 38 files changed, 3587 insertions(+) create mode 100644 sdks/java/io/kinesis/pom.xml create mode 100644 sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/CheckpointGenerator.java create mode 100644 sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/CustomOptional.java create mode 100644 sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/DynamicCheckpointGenerator.java create mode 100644 sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/GetKinesisRecordsResult.java create mode 100644 sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/KinesisClientProvider.java create mode 100644 sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/KinesisIO.java create mode 100644 sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/KinesisReader.java create mode 100644 sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/KinesisReaderCheckpoint.java create mode 100644 sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/KinesisRecord.java create mode 100644 sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/KinesisRecordCoder.java create mode 100644 sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/KinesisSource.java create mode 100644 sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/RecordFilter.java create mode 100644 sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/RoundRobin.java create mode 100644 sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/ShardCheckpoint.java create mode 100644 sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/ShardRecordsIterator.java create mode 100644 sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/SimplifiedKinesisClient.java create mode 100644 sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/StartingPoint.java create mode 100644 sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/StaticCheckpointGenerator.java create mode 100644 sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/TransientKinesisException.java create mode 100644 sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/package-info.java create mode 100644 sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/AmazonKinesisMock.java create mode 100644 sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/CustomOptionalTest.java create mode 100644 sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/DynamicCheckpointGeneratorTest.java create mode 100644 sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/KinesisMockReadTest.java create mode 100644 sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/KinesisReaderCheckpointTest.java create mode 100644 sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/KinesisReaderIT.java create mode 100644 sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/KinesisReaderTest.java create mode 100644 sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/KinesisRecordCoderTest.java create mode 100644 sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/KinesisTestOptions.java create mode 100644 sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/KinesisUploader.java create mode 100644 sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/RecordFilterTest.java create mode 100644 sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/RoundRobinTest.java create mode 100644 sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/ShardCheckpointTest.java create mode 100644 sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/ShardRecordsIteratorTest.java create mode 100644 sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/SimplifiedKinesisClientTest.java create mode 100644 sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/package-info.java diff --git a/sdks/java/io/kinesis/pom.xml b/sdks/java/io/kinesis/pom.xml new file mode 100644 index 0000000000000..aec1786eee914 --- /dev/null +++ b/sdks/java/io/kinesis/pom.xml @@ -0,0 +1,179 @@ + + + + 4.0.0 + + + org.apache.beam + beam-sdks-java-io-parent + 0.3.0-incubating-SNAPSHOT + ../pom.xml + + + beam-sdks-java-io-kinesis + Apache Beam :: SDKs :: Java :: IO :: Kinesis + Library to read Kinesis streams. + + + + + org.apache.maven.plugins + maven-compiler-plugin + + + org.apache.maven.plugins + maven-source-plugin + + + org.apache.maven.plugins + maven-surefire-plugin + + + false + + + + + org.apache.maven.plugins + maven-jar-plugin + + + org.apache.maven.plugins + maven-checkstyle-plugin + + + org.apache.maven.plugins + maven-javadoc-plugin + + -Xdoclint:missing + + + + + org.apache.maven.plugins + maven-failsafe-plugin + + false + true + + + + + integration-test + verify + + + + ${integrationTestPipelineOptions} + + + + + + + + + + 1.11.18 + + + + + org.apache.beam + beam-sdks-java-core + + + + com.amazonaws + aws-java-sdk-kinesis + ${aws.version} + + + + com.amazonaws + amazon-kinesis-client + 1.6.1 + + + + org.slf4j + slf4j-api + + + + joda-time + joda-time + + + + com.google.guava + guava + + + + commons-lang + commons-lang + 2.6 + + + + com.amazonaws + aws-java-sdk-core + ${aws.version} + + + + com.google.code.findbugs + annotations + + + + + junit + junit + test + + + + org.mockito + mockito-all + test + + + + org.assertj + assertj-core + 2.5.0 + test + + + + org.hamcrest + hamcrest-all + test + + + + org.apache.beam + beam-runners-direct-java + ${project.version} + test + + + + diff --git a/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/CheckpointGenerator.java b/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/CheckpointGenerator.java new file mode 100644 index 0000000000000..919d85aacb4c7 --- /dev/null +++ b/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/CheckpointGenerator.java @@ -0,0 +1,30 @@ +/* + * 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.kinesis; + + +import java.io.Serializable; + +/** + * Used to generate checkpoint object on demand. + * How exactly the checkpoint is generated is up to implementing class. + */ +interface CheckpointGenerator extends Serializable { + KinesisReaderCheckpoint generate(SimplifiedKinesisClient client) + throws TransientKinesisException; +} diff --git a/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/CustomOptional.java b/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/CustomOptional.java new file mode 100644 index 0000000000000..804d6ccb97a05 --- /dev/null +++ b/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/CustomOptional.java @@ -0,0 +1,85 @@ +/* + * 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.kinesis; + +import java.util.NoSuchElementException; + +/*** + * Similar to Guava {@code Optional}, but throws {@link NoSuchElementException} for missing element. + */ +abstract class CustomOptional { + public static CustomOptional absent() { + return Absent.INSTANCE; + } + + public static CustomOptional of(T v) { + return new Present<>(v); + } + + public abstract boolean isPresent(); + + public abstract T get(); + + private static class Present extends CustomOptional { + private final T value; + + private Present(T value) { + this.value = value; + } + + @Override + public boolean isPresent() { + return true; + } + + @Override + public T get() { + return value; + } + + + @Override + public boolean equals(Object o) { + Present present = (Present) o; + + return value != null ? value.equals(present.value) : present.value == null; + } + + @Override + public int hashCode() { + return value != null ? value.hashCode() : 0; + } + } + + private static class Absent extends CustomOptional { + public static final Absent INSTANCE = new Absent(); + + private Absent() { + } + + @Override + public boolean isPresent() { + return false; + } + + @Override + public T get() { + throw new NoSuchElementException(); + } + } +} diff --git a/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/DynamicCheckpointGenerator.java b/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/DynamicCheckpointGenerator.java new file mode 100644 index 0000000000000..d86960f0d030e --- /dev/null +++ b/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/DynamicCheckpointGenerator.java @@ -0,0 +1,58 @@ +/* + * 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.kinesis; + +import static com.google.common.base.Preconditions.checkNotNull; +import static com.google.common.collect.Lists.transform; + + +import com.google.common.base.Function; + +import com.amazonaws.services.kinesis.model.Shard; + +/** + * Creates {@link KinesisReaderCheckpoint}, which spans over all shards in given stream. + * List of shards is obtained dynamically on call to {@link #generate(SimplifiedKinesisClient)}. + */ +class DynamicCheckpointGenerator implements CheckpointGenerator { + private final String streamName; + private final StartingPoint startingPoint; + + public DynamicCheckpointGenerator(String streamName, StartingPoint startingPoint) { + this.streamName = checkNotNull(streamName, "streamName"); + this.startingPoint = checkNotNull(startingPoint, "startingPoint"); + } + + @Override + public KinesisReaderCheckpoint generate(SimplifiedKinesisClient kinesis) + throws TransientKinesisException { + return new KinesisReaderCheckpoint( + transform(kinesis.listShards(streamName), new Function() { + @Override + public ShardCheckpoint apply(Shard shard) { + return new ShardCheckpoint(streamName, shard.getShardId(), startingPoint); + } + }) + ); + } + + @Override + public String toString() { + return String.format("Checkpoint generator for %s: %s", streamName, startingPoint); + } +} diff --git a/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/GetKinesisRecordsResult.java b/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/GetKinesisRecordsResult.java new file mode 100644 index 0000000000000..f48b9d53ade85 --- /dev/null +++ b/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/GetKinesisRecordsResult.java @@ -0,0 +1,54 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.sdk.io.kinesis; + +import static com.google.common.collect.Lists.transform; +import com.google.common.base.Function; + +import com.amazonaws.services.kinesis.clientlibrary.types.UserRecord; +import java.util.List; +import javax.annotation.Nullable; + +/*** + * Represents the output of 'get' operation on Kinesis stream. + */ +class GetKinesisRecordsResult { + private final List records; + private final String nextShardIterator; + + public GetKinesisRecordsResult(List records, String nextShardIterator, + final String streamName, final String shardId) { + this.records = transform(records, new Function() { + @Nullable + @Override + public KinesisRecord apply(@Nullable UserRecord input) { + assert input != null; // to make FindBugs happy + return new KinesisRecord(input, streamName, shardId); + } + }); + this.nextShardIterator = nextShardIterator; + } + + public List getRecords() { + return records; + } + + public String getNextShardIterator() { + return nextShardIterator; + } +} diff --git a/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/KinesisClientProvider.java b/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/KinesisClientProvider.java new file mode 100644 index 0000000000000..36c8953f601b2 --- /dev/null +++ b/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/KinesisClientProvider.java @@ -0,0 +1,31 @@ +/* + * 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.kinesis; + +import com.amazonaws.services.kinesis.AmazonKinesis; +import java.io.Serializable; + +/** + * Provides instances of {@link AmazonKinesis} interface. + * + * Please note, that any instance of {@link KinesisClientProvider} must be + * {@link Serializable} to ensure it can be sent to worker machines. + */ +interface KinesisClientProvider extends Serializable { + AmazonKinesis get(); +} diff --git a/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/KinesisIO.java b/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/KinesisIO.java new file mode 100644 index 0000000000000..b3cb464e1cfbf --- /dev/null +++ b/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/KinesisIO.java @@ -0,0 +1,190 @@ +/* + * 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.kinesis; + + +import org.apache.beam.sdk.transforms.PTransform; +import static com.google.common.base.Preconditions.checkNotNull; + +import com.amazonaws.auth.AWSCredentialsProvider; +import com.amazonaws.auth.BasicAWSCredentials; +import com.amazonaws.internal.StaticCredentialsProvider; +import com.amazonaws.regions.Regions; +import com.amazonaws.services.kinesis.AmazonKinesis; +import com.amazonaws.services.kinesis.AmazonKinesisClient; +import com.amazonaws.services.kinesis.clientlibrary.lib.worker.InitialPositionInStream; +import org.joda.time.Instant; + +/** + * {@link PTransform}s for reading from + * Kinesis streams. + * + *

              Usage

              + * + *

              Main class you're going to operate is called {@link KinesisIO}. + * It follows the usage conventions laid out by other *IO classes like + * BigQueryIO or PubsubIOLet's see how you can set up a simple Pipeline, which reads from Kinesis: + * + *

              {@code}
              + * p.
              + *   apply(KinesisIO.Read.
              + *     from("streamName", InitialPositionInStream.LATEST).
              + *     using("AWS_KEY", _"AWS_SECRET", STREAM_REGION).
              + *     apply( ... ) // other transformations
              + *
              + *

              + * + *

              + * As you can see you need to provide 3 things: + *

                + *
              • name of the stream you're going to read
              • + *
              • position in the stream where reading should start. There are two options:
              • + *
                  + *
                • {@link InitialPositionInStream#LATEST} - reading will begin from end of the stream
                • + *
                • {@link InitialPositionInStream#TRIM_HORIZON} - reading will begin at + * the very beginning of the stream
                • + *
                + *
              • data used to initialize {@link AmazonKinesis} client
              • + *
                  + *
                • credentials (aws key, aws secret)
                • + *
                • region where the stream is located
                • + *
                + *
              + *

              + * + *

              In case when you want to set up {@link AmazonKinesis} client by your own + * (for example if you're using more sophisticated authorization methods like Amazon STS, etc.) + * you can do it by implementing {@link KinesisClientProvider} class: + * + *

              {@code}
              + * public class MyCustomKinesisClientProvider implements KinesisClientProvider {
              + *   @Override
              + *   public AmazonKinesis get() {
              + *     // set up your client here
              + *   }
              + * }
              + * 
              + * + * Usage is pretty straightforward: + * + *
              {@code}
              + * p.
              + *   apply(KinesisIO.Read.
              + *    from("streamName", InitialPositionInStream.LATEST).
              + *    using(MyCustomKinesisClientProvider()).
              + *    apply( ... ) // other transformations
              + * 
              + *

              + * + *

              There’s also possibility to start reading using arbitrary point in time - + * in this case you need to provide {@link Instant} object: + * + *

              {@code}
              + * p.
              + *   apply(KinesisIO.Read.
              + *     from("streamName", instant).
              + *     using(MyCustomKinesisClientProvider()).
              + *     apply( ... ) // other transformations
              + * 
              + *

              + * + */ +public final class KinesisIO { + /*** + * A {@link PTransform} that reads from a Kinesis stream. + */ + public static final class Read { + + private final String streamName; + private final StartingPoint initialPosition; + + private Read(String streamName, StartingPoint initialPosition) { + this.streamName = checkNotNull(streamName, "streamName"); + this.initialPosition = checkNotNull(initialPosition, "initialPosition"); + } + + /*** + * Specify reading from streamName at some initial position. + */ + public static Read from(String streamName, InitialPositionInStream initialPosition) { + return new Read(streamName, new StartingPoint( + checkNotNull(initialPosition, "initialPosition"))); + } + + /*** + * Specify reading from streamName beginning at given {@link Instant}. + * This {@link Instant} must be in the past, i.e. before {@link Instant#now()}. + */ + public static Read from(String streamName, Instant initialTimestamp) { + return new Read(streamName, new StartingPoint( + checkNotNull(initialTimestamp, "initialTimestamp"))); + } + + /*** + * Allows to specify custom {@link KinesisClientProvider}. + * {@link KinesisClientProvider} provides {@link AmazonKinesis} instances which are later + * used for communication with Kinesis. + * You should use this method if {@link Read#using(String, String, Regions)} does not + * suite your needs. + */ + public org.apache.beam.sdk.io.Read.Unbounded using + (KinesisClientProvider kinesisClientProvider) { + return org.apache.beam.sdk.io.Read.from( + new KinesisSource(kinesisClientProvider, streamName, + initialPosition)); + } + + /*** + * Specify credential details and region to be used to read from Kinesis. + * If you need more sophisticated credential protocol, then you should look at + * {@link Read#using(KinesisClientProvider)}. + */ + public org.apache.beam.sdk.io.Read.Unbounded using(String awsAccessKey, + String awsSecretKey, + Regions region) { + return using(new BasicKinesisProvider(awsAccessKey, awsSecretKey, region)); + } + + private static final class BasicKinesisProvider implements KinesisClientProvider { + + private final String accessKey; + private final String secretKey; + private final Regions region; + + private BasicKinesisProvider(String accessKey, String secretKey, Regions region) { + this.accessKey = checkNotNull(accessKey, "accessKey"); + this.secretKey = checkNotNull(secretKey, "secretKey"); + this.region = checkNotNull(region, "region"); + } + + + private AWSCredentialsProvider getCredentialsProvider() { + return new StaticCredentialsProvider(new BasicAWSCredentials( + accessKey, + secretKey + )); + + } + + @Override + public AmazonKinesis get() { + return new AmazonKinesisClient(getCredentialsProvider()).withRegion(region); + } + } + } +} diff --git a/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/KinesisReader.java b/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/KinesisReader.java new file mode 100644 index 0000000000000..38a0050fa72aa --- /dev/null +++ b/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/KinesisReader.java @@ -0,0 +1,145 @@ +/* + * 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.kinesis; + + +import org.apache.beam.sdk.io.UnboundedSource; +import static com.google.common.base.Preconditions.checkNotNull; +import static com.google.common.collect.Lists.newArrayList; + +import org.joda.time.Instant; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import java.io.IOException; +import java.util.List; +import java.util.NoSuchElementException; + + +/*** + * Reads data from multiple kinesis shards in a single thread. + * It uses simple round robin algorithm when fetching data from shards. + */ +class KinesisReader extends UnboundedSource.UnboundedReader { + private static final Logger LOG = LoggerFactory.getLogger(KinesisReader.class); + + private final SimplifiedKinesisClient kinesis; + private final UnboundedSource source; + private final CheckpointGenerator initialCheckpointGenerator; + private RoundRobin shardIterators; + private CustomOptional currentRecord = CustomOptional.absent(); + + public KinesisReader(SimplifiedKinesisClient kinesis, + CheckpointGenerator initialCheckpointGenerator, + UnboundedSource source) { + this.kinesis = checkNotNull(kinesis, "kinesis"); + this.initialCheckpointGenerator = + checkNotNull(initialCheckpointGenerator, "initialCheckpointGenerator"); + this.source = source; + } + + /*** + * Generates initial checkpoint and instantiates iterators for shards. + */ + @Override + public boolean start() throws IOException { + LOG.info("Starting reader using {}", initialCheckpointGenerator); + + try { + KinesisReaderCheckpoint initialCheckpoint = + initialCheckpointGenerator.generate(kinesis); + List iterators = newArrayList(); + for (ShardCheckpoint checkpoint : initialCheckpoint) { + iterators.add(checkpoint.getShardRecordsIterator(kinesis)); + } + shardIterators = new RoundRobin<>(iterators); + } catch (TransientKinesisException e) { + throw new IOException(e); + } + + return advance(); + } + + /*** + * Moves to the next record in one of the shards. + * If current shard iterator can be move forward (i.e. there's a record present) then we do it. + * If not, we iterate over shards in a round-robin manner. + */ + @Override + public boolean advance() throws IOException { + try { + for (int i = 0; i < shardIterators.size(); ++i) { + currentRecord = shardIterators.getCurrent().next(); + if (currentRecord.isPresent()) { + return true; + } else { + shardIterators.moveForward(); + } + } + } catch (TransientKinesisException e) { + LOG.warn("Transient exception occurred", e); + } + return false; + } + + @Override + public byte[] getCurrentRecordId() throws NoSuchElementException { + return currentRecord.get().getUniqueId(); + } + + @Override + public KinesisRecord getCurrent() throws NoSuchElementException { + return currentRecord.get(); + } + + /*** + * When {@link KinesisReader} was advanced to the current record. + * We cannot use approximate arrival timestamp given for each record by Kinesis as it + * is not guaranteed to be accurate - this could lead to mark some records as "late" + * even if they were not. + */ + @Override + public Instant getCurrentTimestamp() throws NoSuchElementException { + return currentRecord.get().getReadTime(); + } + + @Override + public void close() throws IOException { + } + + /*** + * Current time. + * We cannot give better approximation of the watermark with current semantics of + * {@link KinesisReader#getCurrentTimestamp()}, because we don't know when the next + * {@link KinesisReader#advance()} will be called. + */ + @Override + public Instant getWatermark() { + return Instant.now(); + } + + @Override + public UnboundedSource.CheckpointMark getCheckpointMark() { + return KinesisReaderCheckpoint.asCurrentStateOf(shardIterators); + } + + @Override + public UnboundedSource getCurrentSource() { + return source; + } + +} diff --git a/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/KinesisReaderCheckpoint.java b/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/KinesisReaderCheckpoint.java new file mode 100644 index 0000000000000..6ceb74295d863 --- /dev/null +++ b/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/KinesisReaderCheckpoint.java @@ -0,0 +1,98 @@ +/* + * 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.kinesis; + +import org.apache.beam.sdk.io.UnboundedSource; +import static com.google.common.collect.Iterables.transform; +import static com.google.common.collect.Lists.newArrayList; +import static com.google.common.collect.Lists.partition; + + +import com.google.common.base.Function; +import com.google.common.collect.ImmutableList; + +import java.io.IOException; +import java.io.Serializable; +import java.util.Iterator; +import java.util.List; +import javax.annotation.Nullable; + +/*** + * Checkpoint representing a total progress in a set of shards in single stream. + * The set of shards covered by {@link KinesisReaderCheckpoint} may or may not be equal to set of + * all shards present in the stream. + * This class is immutable. + */ +class KinesisReaderCheckpoint implements Iterable, UnboundedSource + .CheckpointMark, Serializable { + private final List shardCheckpoints; + + public KinesisReaderCheckpoint(Iterable shardCheckpoints) { + this.shardCheckpoints = ImmutableList.copyOf(shardCheckpoints); + } + + public static KinesisReaderCheckpoint asCurrentStateOf(Iterable + iterators) { + return new KinesisReaderCheckpoint(transform(iterators, + new Function() { + + @Nullable + @Override + public ShardCheckpoint apply(@Nullable + ShardRecordsIterator shardRecordsIterator) { + assert shardRecordsIterator != null; + return shardRecordsIterator.getCheckpoint(); + } + })); + } + + /*** + * Splits given multi-shard checkpoint into partitions of approximately equal size. + * + * @param desiredNumSplits - upper limit for number of partitions to generate. + * @return list of checkpoints covering consecutive partitions of current checkpoint. + */ + public List splitInto(int desiredNumSplits) { + int partitionSize = divideAndRoundUp(shardCheckpoints.size(), desiredNumSplits); + + List checkpoints = newArrayList(); + for (List shardPartition : partition(shardCheckpoints, partitionSize)) { + checkpoints.add(new KinesisReaderCheckpoint(shardPartition)); + } + return checkpoints; + } + + private int divideAndRoundUp(int nominator, int denominator) { + return (nominator + denominator - 1) / denominator; + } + + @Override + public void finalizeCheckpoint() throws IOException { + + } + + @Override + public String toString() { + return shardCheckpoints.toString(); + } + + @Override + public Iterator iterator() { + return shardCheckpoints.iterator(); + } +} diff --git a/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/KinesisRecord.java b/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/KinesisRecord.java new file mode 100644 index 0000000000000..cdb495ce00783 --- /dev/null +++ b/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/KinesisRecord.java @@ -0,0 +1,121 @@ +/* + * 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.kinesis; + +import com.google.common.base.Charsets; + +import com.amazonaws.services.kinesis.clientlibrary.types.ExtendedSequenceNumber; +import com.amazonaws.services.kinesis.clientlibrary.types.UserRecord; +import static org.apache.commons.lang.builder.HashCodeBuilder.reflectionHashCode; +import org.apache.commons.lang.builder.EqualsBuilder; +import org.joda.time.Instant; +import java.io.Serializable; +import java.nio.ByteBuffer; + +/** + * {@link UserRecord} enhanced with utility methods. + */ +public class KinesisRecord implements Serializable { + private Instant readTime; + private String streamName; + private String shardId; + private long subSequenceNumber; + private String sequenceNumber; + private Instant approximateArrivalTimestamp; + private ByteBuffer data; + private String partitionKey; + + public KinesisRecord(UserRecord record, String streamName, String shardId) { + this(record.getData(), record.getSequenceNumber(), record.getSubSequenceNumber(), + record.getPartitionKey(), + new Instant(record.getApproximateArrivalTimestamp()), + Instant.now(), + streamName, shardId); + } + + public KinesisRecord(ByteBuffer data, String sequenceNumber, long subSequenceNumber, + String partitionKey, Instant approximateArrivalTimestamp, + Instant readTime, + String streamName, String shardId) { + this.data = data; + this.sequenceNumber = sequenceNumber; + this.subSequenceNumber = subSequenceNumber; + this.partitionKey = partitionKey; + this.approximateArrivalTimestamp = approximateArrivalTimestamp; + this.readTime = readTime; + this.streamName = streamName; + this.shardId = shardId; + } + + public ExtendedSequenceNumber getExtendedSequenceNumber() { + return new ExtendedSequenceNumber(getSequenceNumber(), getSubSequenceNumber()); + } + + /*** + * @return unique id of the record based on its position in the stream + */ + public byte[] getUniqueId() { + return getExtendedSequenceNumber().toString().getBytes(Charsets.UTF_8); + } + + public Instant getReadTime() { + return readTime; + } + + public String getStreamName() { + return streamName; + } + + public String getShardId() { + return shardId; + } + + public byte[] getDataAsBytes() { + return getData().array(); + } + + @Override + public boolean equals(Object obj) { + return EqualsBuilder.reflectionEquals(this, obj); + } + + @Override + public int hashCode() { + return reflectionHashCode(this); + } + + public long getSubSequenceNumber() { + return subSequenceNumber; + } + + public String getSequenceNumber() { + return sequenceNumber; + } + + public Instant getApproximateArrivalTimestamp() { + return approximateArrivalTimestamp; + } + + public ByteBuffer getData() { + return data; + } + + public String getPartitionKey() { + return partitionKey; + } +} diff --git a/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/KinesisRecordCoder.java b/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/KinesisRecordCoder.java new file mode 100644 index 0000000000000..c383a4ff8a91a --- /dev/null +++ b/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/KinesisRecordCoder.java @@ -0,0 +1,75 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.sdk.io.kinesis; + +import org.apache.beam.sdk.coders.AtomicCoder; +import org.apache.beam.sdk.coders.ByteArrayCoder; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.coders.InstantCoder; +import org.apache.beam.sdk.coders.StringUtf8Coder; +import org.apache.beam.sdk.coders.VarLongCoder; + +import org.joda.time.Instant; +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.nio.ByteBuffer; + +/*** + * A {@link Coder} for {@link KinesisRecord}. + */ +class KinesisRecordCoder extends AtomicCoder { + private static final StringUtf8Coder STRING_CODER = StringUtf8Coder.of(); + private static final ByteArrayCoder BYTE_ARRAY_CODER = ByteArrayCoder.of(); + private static final InstantCoder INSTANT_CODER = InstantCoder.of(); + private static final VarLongCoder VAR_LONG_CODER = VarLongCoder.of(); + + public static KinesisRecordCoder of() { + return new KinesisRecordCoder(); + } + + @Override + public void encode(KinesisRecord value, OutputStream outStream, Context context) throws + IOException { + Context nested = context.nested(); + BYTE_ARRAY_CODER.encode(value.getData().array(), outStream, nested); + STRING_CODER.encode(value.getSequenceNumber(), outStream, nested); + STRING_CODER.encode(value.getPartitionKey(), outStream, nested); + INSTANT_CODER.encode(value.getApproximateArrivalTimestamp(), outStream, nested); + VAR_LONG_CODER.encode(value.getSubSequenceNumber(), outStream, nested); + INSTANT_CODER.encode(value.getReadTime(), outStream, nested); + STRING_CODER.encode(value.getStreamName(), outStream, nested); + STRING_CODER.encode(value.getShardId(), outStream, nested); + } + + @Override + public KinesisRecord decode(InputStream inStream, Context context) throws IOException { + Context nested = context.nested(); + ByteBuffer data = ByteBuffer.wrap(BYTE_ARRAY_CODER.decode(inStream, nested)); + String sequenceNumber = STRING_CODER.decode(inStream, nested); + String partitionKey = STRING_CODER.decode(inStream, nested); + Instant approximateArrivalTimestamp = INSTANT_CODER.decode(inStream, nested); + long subSequenceNumber = VAR_LONG_CODER.decode(inStream, nested); + Instant readTimestamp = INSTANT_CODER.decode(inStream, nested); + String streamName = STRING_CODER.decode(inStream, nested); + String shardId = STRING_CODER.decode(inStream, nested); + return new KinesisRecord(data, sequenceNumber, subSequenceNumber, partitionKey, + approximateArrivalTimestamp, readTimestamp, streamName, shardId + ); + } +} diff --git a/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/KinesisSource.java b/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/KinesisSource.java new file mode 100644 index 0000000000000..38c9fa4072ef3 --- /dev/null +++ b/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/KinesisSource.java @@ -0,0 +1,114 @@ +/* + * 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.kinesis; + +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.coders.SerializableCoder; +import org.apache.beam.sdk.io.UnboundedSource; +import org.apache.beam.sdk.options.PipelineOptions; + + +import static com.google.common.base.Preconditions.checkNotNull; +import static com.google.common.collect.Lists.newArrayList; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import java.util.List; + + +/*** + * Represents source for single stream in Kinesis. + */ +class KinesisSource extends UnboundedSource { + private static final Logger LOG = LoggerFactory.getLogger(KinesisSource.class); + + private final KinesisClientProvider kinesis; + private CheckpointGenerator initialCheckpointGenerator; + + public KinesisSource(KinesisClientProvider kinesis, String streamName, + StartingPoint startingPoint) { + this(kinesis, new DynamicCheckpointGenerator(streamName, startingPoint)); + } + + private KinesisSource(KinesisClientProvider kinesisClientProvider, + CheckpointGenerator initialCheckpoint) { + this.kinesis = kinesisClientProvider; + this.initialCheckpointGenerator = initialCheckpoint; + validate(); + } + + /*** + * Generate splits for reading from the stream. + * Basically, it'll try to evenly split set of shards in the stream into + * {@code desiredNumSplits} partitions. Each partition is then a split. + */ + @Override + public List generateInitialSplits(int desiredNumSplits, + PipelineOptions options) throws Exception { + KinesisReaderCheckpoint checkpoint = + initialCheckpointGenerator.generate(SimplifiedKinesisClient.from(kinesis)); + + List sources = newArrayList(); + + for (KinesisReaderCheckpoint partition : checkpoint.splitInto(desiredNumSplits)) { + sources.add(new KinesisSource( + kinesis, + new StaticCheckpointGenerator(partition))); + } + return sources; + } + + /*** + * Creates reader based on given {@link KinesisReaderCheckpoint}. + * If {@link KinesisReaderCheckpoint} is not given, then we use + * {@code initialCheckpointGenerator} to generate new checkpoint. + */ + @Override + public UnboundedReader createReader(PipelineOptions options, + KinesisReaderCheckpoint checkpointMark) { + + CheckpointGenerator checkpointGenerator = initialCheckpointGenerator; + + if (checkpointMark != null) { + checkpointGenerator = new StaticCheckpointGenerator(checkpointMark); + } + + LOG.info("Creating new reader using {}", checkpointGenerator); + + return new KinesisReader( + SimplifiedKinesisClient.from(kinesis), + checkpointGenerator, + this); + } + + @Override + public Coder getCheckpointMarkCoder() { + return SerializableCoder.of(KinesisReaderCheckpoint.class); + } + + @Override + public void validate() { + checkNotNull(kinesis); + checkNotNull(initialCheckpointGenerator); + } + + @Override + public Coder getDefaultOutputCoder() { + return KinesisRecordCoder.of(); + } +} diff --git a/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/RecordFilter.java b/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/RecordFilter.java new file mode 100644 index 0000000000000..4c7f39ae8c61b --- /dev/null +++ b/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/RecordFilter.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.kinesis; + +import static com.google.common.collect.Lists.newArrayList; + +import java.util.List; + + +/** + * Filters out records, which were already processed and checkpointed. + *

              + * We need this step, because we can get iterators from Kinesis only with "sequenceNumber" accuracy, + * not with "subSequenceNumber" accuracy. + */ +class RecordFilter { + public List apply(List records, ShardCheckpoint checkpoint) { + List filteredRecords = newArrayList(); + for (KinesisRecord record : records) { + if (checkpoint.isBeforeOrAt(record)) { + filteredRecords.add(record); + } + } + return filteredRecords; + } +} diff --git a/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/RoundRobin.java b/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/RoundRobin.java new file mode 100644 index 0000000000000..7257aa1492bbb --- /dev/null +++ b/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/RoundRobin.java @@ -0,0 +1,54 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.sdk.io.kinesis; + +import static com.google.common.base.Preconditions.checkArgument; +import static com.google.common.collect.Queues.newArrayDeque; + + +import java.util.Deque; +import java.util.Iterator; + +/*** + * Very simple implementation of round robin algorithm. + */ +class RoundRobin implements Iterable { + private final Deque deque; + + public RoundRobin(Iterable collection) { + this.deque = newArrayDeque(collection); + checkArgument(!deque.isEmpty(), "Tried to initialize RoundRobin with empty collection"); + } + + public T getCurrent() { + return deque.getFirst(); + } + + public void moveForward() { + deque.addLast(deque.removeFirst()); + } + + public int size() { + return deque.size(); + } + + @Override + public Iterator iterator() { + return deque.iterator(); + } +} diff --git a/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/ShardCheckpoint.java b/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/ShardCheckpoint.java new file mode 100644 index 0000000000000..1d8628bc8851e --- /dev/null +++ b/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/ShardCheckpoint.java @@ -0,0 +1,175 @@ +/* + * 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.kinesis; + + +import static com.google.common.base.Preconditions.checkArgument; +import static com.google.common.base.Preconditions.checkNotNull; + +import static com.amazonaws.services.kinesis.model.ShardIteratorType.AFTER_SEQUENCE_NUMBER; +import static com.amazonaws.services.kinesis.model.ShardIteratorType.AT_SEQUENCE_NUMBER; +import static com.amazonaws.services.kinesis.model.ShardIteratorType.AT_TIMESTAMP; +import com.amazonaws.services.kinesis.clientlibrary.types.ExtendedSequenceNumber; +import com.amazonaws.services.kinesis.model.Record; +import com.amazonaws.services.kinesis.model.ShardIteratorType; +import org.joda.time.Instant; +import java.io.Serializable; + + +/*** + * Checkpoint mark for single shard in the stream. + * Current position in the shard is determined by either: + *

                + *
              • {@link #shardIteratorType} if it is equal to {@link ShardIteratorType#LATEST} or + * {@link ShardIteratorType#TRIM_HORIZON}
              • + *
              • combination of + * {@link #sequenceNumber} and {@link #subSequenceNumber} if + * {@link ShardIteratorType#AFTER_SEQUENCE_NUMBER} or + * {@link ShardIteratorType#AT_SEQUENCE_NUMBER}
              • + *
              + * This class is immutable. + */ +class ShardCheckpoint implements Serializable { + private final String streamName; + private final String shardId; + private final String sequenceNumber; + private final ShardIteratorType shardIteratorType; + private final Long subSequenceNumber; + private final Instant timestamp; + + public ShardCheckpoint(String streamName, String shardId, StartingPoint + startingPoint) { + this(streamName, shardId, + ShardIteratorType.fromValue(startingPoint.getPositionName()), + startingPoint.getTimestamp()); + } + + public ShardCheckpoint(String streamName, String shardId, ShardIteratorType + shardIteratorType, Instant timestamp) { + this(streamName, shardId, shardIteratorType, null, null, timestamp); + } + + public ShardCheckpoint(String streamName, String shardId, ShardIteratorType + shardIteratorType, String sequenceNumber, Long subSequenceNumber) { + this(streamName, shardId, shardIteratorType, sequenceNumber, subSequenceNumber, null); + } + + private ShardCheckpoint(String streamName, String shardId, ShardIteratorType shardIteratorType, + String sequenceNumber, Long subSequenceNumber, Instant timestamp) { + this.shardIteratorType = checkNotNull(shardIteratorType, "shardIteratorType"); + this.streamName = checkNotNull(streamName, "streamName"); + this.shardId = checkNotNull(shardId, "shardId"); + if (shardIteratorType == AT_SEQUENCE_NUMBER || shardIteratorType == AFTER_SEQUENCE_NUMBER) { + checkNotNull(sequenceNumber, + "You must provide sequence number for AT_SEQUENCE_NUMBER" + + " or AFTER_SEQUENCE_NUMBER"); + } else { + checkArgument(sequenceNumber == null, + "Sequence number must be null for LATEST, TRIM_HORIZON or AT_TIMESTAMP"); + } + if (shardIteratorType == AT_TIMESTAMP) { + checkNotNull(timestamp, + "You must provide timestamp for AT_SEQUENCE_NUMBER" + + " or AFTER_SEQUENCE_NUMBER"); + } else { + checkArgument(timestamp == null, + "Timestamp must be null for an iterator type other than AT_TIMESTAMP"); + } + + this.subSequenceNumber = subSequenceNumber; + this.sequenceNumber = sequenceNumber; + this.timestamp = timestamp; + } + + /*** + * Used to compare {@link ShardCheckpoint} object to {@link KinesisRecord}. Depending + * on the the underlying shardIteratorType, it will either compare the timestamp or the + * {@link ExtendedSequenceNumber}. + * + * @param other + * @return if current checkpoint mark points before or at given {@link ExtendedSequenceNumber} + */ + public boolean isBeforeOrAt(KinesisRecord other) { + if (shardIteratorType == AT_TIMESTAMP) { + return timestamp.compareTo(other.getApproximateArrivalTimestamp()) <= 0; + } + int result = extendedSequenceNumber().compareTo(other.getExtendedSequenceNumber()); + if (result == 0) { + return shardIteratorType == AT_SEQUENCE_NUMBER; + } + return result < 0; + } + + private ExtendedSequenceNumber extendedSequenceNumber() { + String fullSequenceNumber = sequenceNumber; + if (fullSequenceNumber == null) { + fullSequenceNumber = shardIteratorType.toString(); + } + return new ExtendedSequenceNumber(fullSequenceNumber, subSequenceNumber); + } + + @Override + public String toString() { + return String.format("Checkpoint %s for stream %s, shard %s: %s", shardIteratorType, + streamName, shardId, + sequenceNumber); + } + + public ShardRecordsIterator getShardRecordsIterator(SimplifiedKinesisClient kinesis) + throws TransientKinesisException { + return new ShardRecordsIterator(this, kinesis); + } + + public String getShardIterator(SimplifiedKinesisClient kinesisClient) + throws TransientKinesisException { + if (checkpointIsInTheMiddleOfAUserRecord()) { + return kinesisClient.getShardIterator(streamName, + shardId, AT_SEQUENCE_NUMBER, + sequenceNumber, null); + } + return kinesisClient.getShardIterator(streamName, + shardId, shardIteratorType, + sequenceNumber, timestamp); + } + + private boolean checkpointIsInTheMiddleOfAUserRecord() { + return shardIteratorType == AFTER_SEQUENCE_NUMBER && subSequenceNumber != null; + } + + /*** + * Used to advance checkpoint mark to position after given {@link Record}. + * + * @param record + * @return new checkpoint object pointing directly after given {@link Record} + */ + public ShardCheckpoint moveAfter(KinesisRecord record) { + return new ShardCheckpoint( + streamName, shardId, + AFTER_SEQUENCE_NUMBER, + record.getSequenceNumber(), + record.getSubSequenceNumber()); + } + + public String getStreamName() { + return streamName; + } + + public String getShardId() { + return shardId; + } +} diff --git a/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/ShardRecordsIterator.java b/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/ShardRecordsIterator.java new file mode 100644 index 0000000000000..7dfe158ca41db --- /dev/null +++ b/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/ShardRecordsIterator.java @@ -0,0 +1,98 @@ +/* + * 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.kinesis; + +import static com.google.common.base.Preconditions.checkNotNull; +import static com.google.common.collect.Queues.newArrayDeque; + +import com.amazonaws.services.kinesis.model.ExpiredIteratorException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import java.util.Deque; + +/*** + * Iterates over records in a single shard. + * Under the hood records are retrieved from Kinesis in batches and stored in the in-memory queue. + * Then the caller of {@link ShardRecordsIterator#next()} can read from queue one by one. + */ +class ShardRecordsIterator { + private static final Logger LOG = LoggerFactory.getLogger(ShardRecordsIterator.class); + + private final SimplifiedKinesisClient kinesis; + private final RecordFilter filter; + private ShardCheckpoint checkpoint; + private String shardIterator; + private Deque data = newArrayDeque(); + + public ShardRecordsIterator(final ShardCheckpoint initialCheckpoint, + SimplifiedKinesisClient simplifiedKinesisClient) throws + TransientKinesisException { + this(initialCheckpoint, simplifiedKinesisClient, new RecordFilter()); + } + + public ShardRecordsIterator(final ShardCheckpoint initialCheckpoint, + SimplifiedKinesisClient simplifiedKinesisClient, + RecordFilter filter) throws + TransientKinesisException { + + this.checkpoint = checkNotNull(initialCheckpoint, "initialCheckpoint"); + this.filter = checkNotNull(filter, "filter"); + this.kinesis = checkNotNull(simplifiedKinesisClient, "simplifiedKinesisClient"); + shardIterator = checkpoint.getShardIterator(kinesis); + } + + /*** + * Returns record if there's any present. + * Returns absent() if there are no new records at this time in the shard. + */ + public CustomOptional next() throws TransientKinesisException { + readMoreIfNecessary(); + + if (data.isEmpty()) { + return CustomOptional.absent(); + } else { + KinesisRecord record = data.removeFirst(); + checkpoint = checkpoint.moveAfter(record); + return CustomOptional.of(record); + } + } + + private void readMoreIfNecessary() throws TransientKinesisException { + if (data.isEmpty()) { + GetKinesisRecordsResult response; + try { + response = kinesis.getRecords(shardIterator, checkpoint.getStreamName(), + checkpoint.getShardId()); + } catch (ExpiredIteratorException e) { + LOG.info("Refreshing expired iterator", e); + shardIterator = checkpoint.getShardIterator(kinesis); + response = kinesis.getRecords(shardIterator, checkpoint.getStreamName(), + checkpoint.getShardId()); + } + LOG.debug("Fetched {} new records", response.getRecords().size()); + shardIterator = response.getNextShardIterator(); + data.addAll(filter.apply(response.getRecords(), checkpoint)); + } + } + + public ShardCheckpoint getCheckpoint() { + return checkpoint; + } + + +} diff --git a/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/SimplifiedKinesisClient.java b/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/SimplifiedKinesisClient.java new file mode 100644 index 0000000000000..f9a1ea2af8109 --- /dev/null +++ b/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/SimplifiedKinesisClient.java @@ -0,0 +1,158 @@ +/* + * 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.kinesis; + + +import com.google.common.collect.Lists; + +import com.amazonaws.AmazonServiceException; +import com.amazonaws.services.kinesis.AmazonKinesis; +import com.amazonaws.services.kinesis.clientlibrary.types.UserRecord; +import com.amazonaws.services.kinesis.model.ExpiredIteratorException; +import com.amazonaws.services.kinesis.model.GetRecordsRequest; +import com.amazonaws.services.kinesis.model.GetRecordsResult; +import com.amazonaws.services.kinesis.model.GetShardIteratorRequest; +import com.amazonaws.services.kinesis.model.LimitExceededException; +import com.amazonaws.services.kinesis.model.ProvisionedThroughputExceededException; +import com.amazonaws.services.kinesis.model.Shard; +import com.amazonaws.services.kinesis.model.ShardIteratorType; +import com.amazonaws.services.kinesis.model.StreamDescription; +import org.joda.time.Instant; +import java.util.Date; +import java.util.List; +import java.util.concurrent.Callable; + +/*** + * Wraps {@link AmazonKinesis} class providing much simpler interface and + * proper error handling. + */ +class SimplifiedKinesisClient { + private final AmazonKinesis kinesis; + + public SimplifiedKinesisClient(AmazonKinesis kinesis) { + this.kinesis = kinesis; + } + + public static SimplifiedKinesisClient from(KinesisClientProvider provider) { + return new SimplifiedKinesisClient(provider.get()); + } + + public String getShardIterator(final String streamName, final String shardId, + final ShardIteratorType shardIteratorType, + final String startingSequenceNumber, final Instant timestamp) + throws TransientKinesisException { + final Date date = timestamp != null ? timestamp.toDate() : null; + return wrapExceptions(new Callable() { + @Override + public String call() throws Exception { + return kinesis.getShardIterator(new GetShardIteratorRequest() + .withStreamName(streamName) + .withShardId(shardId) + .withShardIteratorType(shardIteratorType) + .withStartingSequenceNumber(startingSequenceNumber) + .withTimestamp(date) + ).getShardIterator(); + } + }); + } + + public List listShards(final String streamName) throws TransientKinesisException { + return wrapExceptions(new Callable>() { + @Override + public List call() throws Exception { + List shards = Lists.newArrayList(); + String lastShardId = null; + + StreamDescription description; + do { + description = kinesis.describeStream(streamName, lastShardId) + .getStreamDescription(); + + shards.addAll(description.getShards()); + lastShardId = shards.get(shards.size() - 1).getShardId(); + } while (description.getHasMoreShards()); + + return shards; + } + }); + } + + /*** + * Gets records from Kinesis and deaggregates them if needed. + * + * @return list of deaggregated records + * @throws TransientKinesisException - in case of recoverable situation + */ + public GetKinesisRecordsResult getRecords(String shardIterator, String streamName, + String shardId) throws TransientKinesisException { + return getRecords(shardIterator, streamName, shardId, null); + } + + /*** + * Gets records from Kinesis and deaggregates them if needed. + * + * @return list of deaggregated records + * @throws TransientKinesisException - in case of recoverable situation + */ + public GetKinesisRecordsResult getRecords(final String shardIterator, final String streamName, + final String shardId, final Integer limit) + throws + TransientKinesisException { + return wrapExceptions(new Callable() { + @Override + public GetKinesisRecordsResult call() throws Exception { + GetRecordsResult response = kinesis.getRecords(new GetRecordsRequest() + .withShardIterator(shardIterator) + .withLimit(limit)); + return new GetKinesisRecordsResult( + UserRecord.deaggregate(response.getRecords()), + response.getNextShardIterator(), + streamName, shardId); + } + }); + } + + /*** + * Wraps Amazon specific exceptions into more friendly format. + * + * @throws TransientKinesisException - in case of recoverable situation, i.e. + * the request rate is too high, Kinesis remote service + * failed, network issue, etc. + * @throws ExpiredIteratorException - if iterator needs to be refreshed + * @throws RuntimeException - in all other cases + */ + private T wrapExceptions(Callable callable) throws TransientKinesisException { + try { + return callable.call(); + } catch (ExpiredIteratorException e) { + throw e; + } catch (LimitExceededException | ProvisionedThroughputExceededException e) { + throw new TransientKinesisException( + "Too many requests to Kinesis. Wait some time and retry.", e); + } catch (AmazonServiceException e) { + if (e.getErrorType() == AmazonServiceException.ErrorType.Service) { + throw new TransientKinesisException( + "Kinesis backend failed. Wait some time and retry.", e); + } + throw new RuntimeException("Kinesis client side failure", e); + } catch (Exception e) { + throw new RuntimeException("Unknown kinesis failure, when trying to reach kinesis", e); + } + } + +} diff --git a/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/StartingPoint.java b/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/StartingPoint.java new file mode 100644 index 0000000000000..8140269c170c2 --- /dev/null +++ b/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/StartingPoint.java @@ -0,0 +1,85 @@ +/* + * 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.kinesis; + + +import static com.google.common.base.Preconditions.checkNotNull; + +import com.amazonaws.services.kinesis.clientlibrary.lib.worker.InitialPositionInStream; +import com.amazonaws.services.kinesis.model.ShardIteratorType; +import org.joda.time.Instant; +import java.io.Serializable; +import java.util.Objects; + +/*** + * Denotes a point at which the reader should start reading from a Kinesis stream. + * It can be expressed either as an {@link InitialPositionInStream} enum constant or a timestamp, + * in which case the reader will start reading at the specified point in time. + */ +class StartingPoint implements Serializable { + private final InitialPositionInStream position; + private final Instant timestamp; + + public StartingPoint(InitialPositionInStream position) { + this.position = checkNotNull(position, "position"); + this.timestamp = null; + } + + public StartingPoint(Instant timestamp) { + this.timestamp = checkNotNull(timestamp, "timestamp"); + this.position = null; + } + + public InitialPositionInStream getPosition() { + return position; + } + + public String getPositionName() { + return position != null ? position.name() : ShardIteratorType.AT_TIMESTAMP.name(); + } + + public Instant getTimestamp() { + return timestamp != null ? timestamp : null; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + StartingPoint that = (StartingPoint) o; + return position == that.position && Objects.equals(timestamp, that.timestamp); + } + + @Override + public int hashCode() { + return Objects.hash(position, timestamp); + } + + @Override + public String toString() { + if (timestamp == null) { + return position.toString(); + } else { + return "Starting at timestamp " + timestamp; + } + } +} diff --git a/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/StaticCheckpointGenerator.java b/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/StaticCheckpointGenerator.java new file mode 100644 index 0000000000000..22dc9734f441e --- /dev/null +++ b/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/StaticCheckpointGenerator.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.kinesis; + +import static com.google.common.base.Preconditions.checkNotNull; + +/** + * Always returns the same instance of checkpoint. + */ +class StaticCheckpointGenerator implements CheckpointGenerator { + private final KinesisReaderCheckpoint checkpoint; + + public StaticCheckpointGenerator(KinesisReaderCheckpoint checkpoint) { + checkNotNull(checkpoint, "checkpoint"); + this.checkpoint = checkpoint; + } + + @Override + public KinesisReaderCheckpoint generate(SimplifiedKinesisClient client) { + return checkpoint; + } + + @Override + public String toString() { + return checkpoint.toString(); + } +} diff --git a/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/TransientKinesisException.java b/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/TransientKinesisException.java new file mode 100644 index 0000000000000..a1a974bf3464b --- /dev/null +++ b/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/TransientKinesisException.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.kinesis; + +import com.amazonaws.AmazonServiceException; + +/** + * Created by p.pastuszka on 21.06.2016. + */ +class TransientKinesisException extends Exception { + public TransientKinesisException(String s, AmazonServiceException e) { + super(s, e); + } +} diff --git a/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/package-info.java b/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/package-info.java new file mode 100644 index 0000000000000..5e37ef1249888 --- /dev/null +++ b/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/package-info.java @@ -0,0 +1,22 @@ +/* + * 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. + */ + +/** + * Tests for KinesisIO. + */ +package org.apache.beam.sdk.io.kinesis; diff --git a/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/AmazonKinesisMock.java b/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/AmazonKinesisMock.java new file mode 100644 index 0000000000000..7ca8e0b4bd8dc --- /dev/null +++ b/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/AmazonKinesisMock.java @@ -0,0 +1,375 @@ +/* + * 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.kinesis; + +import static com.google.common.collect.Lists.newArrayList; +import static com.google.common.collect.Lists.transform; +import com.google.common.base.Function; + +import com.amazonaws.AmazonWebServiceRequest; +import com.amazonaws.ResponseMetadata; +import com.amazonaws.regions.Region; +import com.amazonaws.services.kinesis.AmazonKinesis; +import com.amazonaws.services.kinesis.model.AddTagsToStreamRequest; +import com.amazonaws.services.kinesis.model.AddTagsToStreamResult; +import com.amazonaws.services.kinesis.model.CreateStreamRequest; +import com.amazonaws.services.kinesis.model.CreateStreamResult; +import com.amazonaws.services.kinesis.model.DecreaseStreamRetentionPeriodRequest; +import com.amazonaws.services.kinesis.model.DecreaseStreamRetentionPeriodResult; +import com.amazonaws.services.kinesis.model.DeleteStreamRequest; +import com.amazonaws.services.kinesis.model.DeleteStreamResult; +import com.amazonaws.services.kinesis.model.DescribeStreamRequest; +import com.amazonaws.services.kinesis.model.DescribeStreamResult; +import com.amazonaws.services.kinesis.model.DisableEnhancedMonitoringRequest; +import com.amazonaws.services.kinesis.model.DisableEnhancedMonitoringResult; +import com.amazonaws.services.kinesis.model.EnableEnhancedMonitoringRequest; +import com.amazonaws.services.kinesis.model.EnableEnhancedMonitoringResult; +import com.amazonaws.services.kinesis.model.GetRecordsRequest; +import com.amazonaws.services.kinesis.model.GetRecordsResult; +import com.amazonaws.services.kinesis.model.GetShardIteratorRequest; +import com.amazonaws.services.kinesis.model.GetShardIteratorResult; +import com.amazonaws.services.kinesis.model.IncreaseStreamRetentionPeriodRequest; +import com.amazonaws.services.kinesis.model.IncreaseStreamRetentionPeriodResult; +import com.amazonaws.services.kinesis.model.ListStreamsRequest; +import com.amazonaws.services.kinesis.model.ListStreamsResult; +import com.amazonaws.services.kinesis.model.ListTagsForStreamRequest; +import com.amazonaws.services.kinesis.model.ListTagsForStreamResult; +import com.amazonaws.services.kinesis.model.MergeShardsRequest; +import com.amazonaws.services.kinesis.model.MergeShardsResult; +import com.amazonaws.services.kinesis.model.PutRecordRequest; +import com.amazonaws.services.kinesis.model.PutRecordResult; +import com.amazonaws.services.kinesis.model.PutRecordsRequest; +import com.amazonaws.services.kinesis.model.PutRecordsResult; +import com.amazonaws.services.kinesis.model.Record; +import com.amazonaws.services.kinesis.model.RemoveTagsFromStreamRequest; +import com.amazonaws.services.kinesis.model.RemoveTagsFromStreamResult; +import com.amazonaws.services.kinesis.model.Shard; +import com.amazonaws.services.kinesis.model.ShardIteratorType; +import com.amazonaws.services.kinesis.model.SplitShardRequest; +import com.amazonaws.services.kinesis.model.SplitShardResult; +import com.amazonaws.services.kinesis.model.StreamDescription; +import static org.apache.commons.lang.builder.HashCodeBuilder.reflectionHashCode; +import org.apache.commons.lang.builder.EqualsBuilder; +import org.joda.time.Instant; +import static java.lang.Integer.parseInt; +import static java.lang.Math.min; +import java.io.Serializable; +import java.nio.ByteBuffer; +import java.util.List; +import javax.annotation.Nullable; + +/** + * Created by p.pastuszka on 21.07.2016. + */ +class AmazonKinesisMock implements AmazonKinesis { + + static class TestData implements Serializable { + private final String data; + private final Instant arrivalTimestamp; + private final String sequenceNumber; + + public TestData(KinesisRecord record) { + this(new String(record.getData().array()), + record.getApproximateArrivalTimestamp(), + record.getSequenceNumber()); + } + + public TestData(String data, Instant arrivalTimestamp, String sequenceNumber) { + this.data = data; + this.arrivalTimestamp = arrivalTimestamp; + this.sequenceNumber = sequenceNumber; + } + + public Record convertToRecord() { + return new Record(). + withApproximateArrivalTimestamp(arrivalTimestamp.toDate()). + withData(ByteBuffer.wrap(data.getBytes())). + withSequenceNumber(sequenceNumber). + withPartitionKey(""); + } + + @Override + public boolean equals(Object obj) { + return EqualsBuilder.reflectionEquals(this, obj); + } + + @Override + public int hashCode() { + return reflectionHashCode(this); + } + } + + static class Provider implements KinesisClientProvider { + + private final List> shardedData; + private final int numberOfRecordsPerGet; + + public Provider(List> shardedData, int numberOfRecordsPerGet) { + this.shardedData = shardedData; + this.numberOfRecordsPerGet = numberOfRecordsPerGet; + } + + @Override + public AmazonKinesis get() { + return new AmazonKinesisMock(transform(shardedData, + new Function, List>() { + @Override + public List apply(@Nullable List testDatas) { + return transform(testDatas, new Function() { + @Override + public Record apply(@Nullable TestData testData) { + return testData.convertToRecord(); + } + }); + } + }), numberOfRecordsPerGet); + } + } + + private final List> shardedData; + private final int numberOfRecordsPerGet; + + public AmazonKinesisMock(List> shardedData, int numberOfRecordsPerGet) { + this.shardedData = shardedData; + this.numberOfRecordsPerGet = numberOfRecordsPerGet; + } + + @Override + public GetRecordsResult getRecords(GetRecordsRequest getRecordsRequest) { + String[] shardIteratorParts = getRecordsRequest.getShardIterator().split(":"); + int shardId = parseInt(shardIteratorParts[0]); + int startingRecord = parseInt(shardIteratorParts[1]); + List shardData = shardedData.get(shardId); + + int toIndex = min(startingRecord + numberOfRecordsPerGet, shardData.size()); + int fromIndex = min(startingRecord, toIndex); + return new GetRecordsResult(). + withRecords(shardData.subList(fromIndex, toIndex)). + withNextShardIterator(String.format("%s:%s", shardId, toIndex)); + } + + @Override + public GetShardIteratorResult getShardIterator( + GetShardIteratorRequest getShardIteratorRequest) { + ShardIteratorType shardIteratorType = ShardIteratorType.fromValue( + getShardIteratorRequest.getShardIteratorType()); + + String shardIterator; + if (shardIteratorType == ShardIteratorType.TRIM_HORIZON) { + shardIterator = String.format("%s:%s", getShardIteratorRequest.getShardId(), 0); + } else { + throw new RuntimeException("Not implemented"); + } + + return new GetShardIteratorResult().withShardIterator(shardIterator); + } + + @Override + public DescribeStreamResult describeStream(String streamName, String exclusiveStartShardId) { + int nextShardId = 0; + if (exclusiveStartShardId != null) { + nextShardId = parseInt(exclusiveStartShardId) + 1; + } + boolean hasMoreShards = nextShardId + 1 < shardedData.size(); + + List shards = newArrayList(); + if (nextShardId < shardedData.size()) { + shards.add(new Shard().withShardId(Integer.toString(nextShardId))); + } + + return new DescribeStreamResult().withStreamDescription( + new StreamDescription().withHasMoreShards(hasMoreShards).withShards(shards) + ); + } + + @Override + public void setEndpoint(String endpoint) { + + } + + @Override + public void setRegion(Region region) { + + } + + @Override + public AddTagsToStreamResult addTagsToStream(AddTagsToStreamRequest addTagsToStreamRequest) { + throw new RuntimeException("Not implemented"); + } + + @Override + public CreateStreamResult createStream(CreateStreamRequest createStreamRequest) { + throw new RuntimeException("Not implemented"); + } + + @Override + public CreateStreamResult createStream(String streamName, Integer shardCount) { + throw new RuntimeException("Not implemented"); + } + + @Override + public DecreaseStreamRetentionPeriodResult decreaseStreamRetentionPeriod( + DecreaseStreamRetentionPeriodRequest decreaseStreamRetentionPeriodRequest) { + throw new RuntimeException("Not implemented"); + } + + @Override + public DeleteStreamResult deleteStream(DeleteStreamRequest deleteStreamRequest) { + throw new RuntimeException("Not implemented"); + } + + @Override + public DeleteStreamResult deleteStream(String streamName) { + throw new RuntimeException("Not implemented"); + } + + @Override + public DescribeStreamResult describeStream(DescribeStreamRequest describeStreamRequest) { + throw new RuntimeException("Not implemented"); + } + + @Override + public DescribeStreamResult describeStream(String streamName) { + + throw new RuntimeException("Not implemented"); + } + + @Override + public DescribeStreamResult describeStream(String streamName, + Integer limit, String exclusiveStartShardId) { + throw new RuntimeException("Not implemented"); + } + + @Override + public DisableEnhancedMonitoringResult disableEnhancedMonitoring( + DisableEnhancedMonitoringRequest disableEnhancedMonitoringRequest) { + throw new RuntimeException("Not implemented"); + } + + @Override + public EnableEnhancedMonitoringResult enableEnhancedMonitoring( + EnableEnhancedMonitoringRequest enableEnhancedMonitoringRequest) { + throw new RuntimeException("Not implemented"); + } + + @Override + public GetShardIteratorResult getShardIterator(String streamName, + String shardId, + String shardIteratorType) { + throw new RuntimeException("Not implemented"); + } + + @Override + public GetShardIteratorResult getShardIterator(String streamName, + String shardId, + String shardIteratorType, + String startingSequenceNumber) { + throw new RuntimeException("Not implemented"); + } + + @Override + public IncreaseStreamRetentionPeriodResult increaseStreamRetentionPeriod( + IncreaseStreamRetentionPeriodRequest increaseStreamRetentionPeriodRequest) { + throw new RuntimeException("Not implemented"); + } + + @Override + public ListStreamsResult listStreams(ListStreamsRequest listStreamsRequest) { + throw new RuntimeException("Not implemented"); + } + + @Override + public ListStreamsResult listStreams() { + throw new RuntimeException("Not implemented"); + } + + @Override + public ListStreamsResult listStreams(String exclusiveStartStreamName) { + throw new RuntimeException("Not implemented"); + } + + @Override + public ListStreamsResult listStreams(Integer limit, String exclusiveStartStreamName) { + throw new RuntimeException("Not implemented"); + } + + @Override + public ListTagsForStreamResult listTagsForStream( + ListTagsForStreamRequest listTagsForStreamRequest) { + throw new RuntimeException("Not implemented"); + } + + @Override + public MergeShardsResult mergeShards(MergeShardsRequest mergeShardsRequest) { + throw new RuntimeException("Not implemented"); + } + + @Override + public MergeShardsResult mergeShards(String streamName, + String shardToMerge, String adjacentShardToMerge) { + throw new RuntimeException("Not implemented"); + } + + @Override + public PutRecordResult putRecord(PutRecordRequest putRecordRequest) { + throw new RuntimeException("Not implemented"); + } + + @Override + public PutRecordResult putRecord(String streamName, ByteBuffer data, String partitionKey) { + throw new RuntimeException("Not implemented"); + } + + @Override + public PutRecordResult putRecord(String streamName, ByteBuffer data, + String partitionKey, String sequenceNumberForOrdering) { + throw new RuntimeException("Not implemented"); + } + + @Override + public PutRecordsResult putRecords(PutRecordsRequest putRecordsRequest) { + throw new RuntimeException("Not implemented"); + } + + @Override + public RemoveTagsFromStreamResult removeTagsFromStream( + RemoveTagsFromStreamRequest removeTagsFromStreamRequest) { + throw new RuntimeException("Not implemented"); + } + + @Override + public SplitShardResult splitShard(SplitShardRequest splitShardRequest) { + throw new RuntimeException("Not implemented"); + } + + @Override + public SplitShardResult splitShard(String streamName, + String shardToSplit, String newStartingHashKey) { + throw new RuntimeException("Not implemented"); + } + + @Override + public void shutdown() { + + } + + @Override + public ResponseMetadata getCachedResponseMetadata(AmazonWebServiceRequest request) { + throw new RuntimeException("Not implemented"); + } +} diff --git a/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/CustomOptionalTest.java b/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/CustomOptionalTest.java new file mode 100644 index 0000000000000..152fd6dfde434 --- /dev/null +++ b/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/CustomOptionalTest.java @@ -0,0 +1,31 @@ +/* + * 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.kinesis; + +import org.junit.Test; +import java.util.NoSuchElementException; + +/** + * Created by ppastuszka on 12.12.15. + */ +public class CustomOptionalTest { + @Test(expected = NoSuchElementException.class) + public void absentThrowsNoSuchElementExceptionOnGet() { + CustomOptional.absent().get(); + } +} diff --git a/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/DynamicCheckpointGeneratorTest.java b/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/DynamicCheckpointGeneratorTest.java new file mode 100644 index 0000000000000..a9e5a69a2d513 --- /dev/null +++ b/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/DynamicCheckpointGeneratorTest.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.sdk.io.kinesis; + +import com.amazonaws.services.kinesis.clientlibrary.lib.worker.InitialPositionInStream; +import com.amazonaws.services.kinesis.model.Shard; +import static org.assertj.core.api.Assertions.assertThat; +import static org.mockito.BDDMockito.given; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.mockito.Mock; +import org.mockito.runners.MockitoJUnitRunner; +import static java.util.Arrays.asList; + + +/*** + */ +@RunWith(MockitoJUnitRunner.class) +public class DynamicCheckpointGeneratorTest { + + @Mock + private SimplifiedKinesisClient kinesisClient; + @Mock + private Shard shard1, shard2, shard3; + + @Test + public void shouldMapAllShardsToCheckpoints() throws Exception { + given(shard1.getShardId()).willReturn("shard-01"); + given(shard2.getShardId()).willReturn("shard-02"); + given(shard3.getShardId()).willReturn("shard-03"); + given(kinesisClient.listShards("stream")).willReturn(asList(shard1, shard2, shard3)); + + StartingPoint startingPoint = new StartingPoint(InitialPositionInStream.LATEST); + DynamicCheckpointGenerator underTest = new DynamicCheckpointGenerator("stream", + startingPoint); + + KinesisReaderCheckpoint checkpoint = underTest.generate(kinesisClient); + + assertThat(checkpoint).hasSize(3); + } +} 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 new file mode 100644 index 0000000000000..61a858f8dea53 --- /dev/null +++ b/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/KinesisMockReadTest.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.sdk.io.kinesis; + +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 static com.google.common.collect.Lists.newArrayList; + +import com.google.common.collect.Iterables; + +import com.amazonaws.services.kinesis.clientlibrary.lib.worker.InitialPositionInStream; +import org.joda.time.DateTime; +import org.junit.Test; +import java.util.List; + +/** + * Created by p.pastuszka on 22.07.2016. + */ +public class KinesisMockReadTest { + @Test + public void readsDataFromMockKinesis() { + int noOfShards = 3; + int noOfEventsPerShard = 100; + List> testData = + provideTestData(noOfShards, noOfEventsPerShard); + + final Pipeline p = TestPipeline.create(); + PCollection result = p. + apply( + KinesisIO.Read. + from("stream", InitialPositionInStream.TRIM_HORIZON). + using(new AmazonKinesisMock.Provider(testData, 10)). + withMaxNumRecords(noOfShards * noOfEventsPerShard)). + apply(ParDo.of(new KinesisRecordToTestData())); + PAssert.that(result).containsInAnyOrder(Iterables.concat(testData)); + p.run(); + } + + private static class KinesisRecordToTestData extends + DoFn { + @ProcessElement + public void processElement(ProcessContext c) throws Exception { + c.output(new AmazonKinesisMock.TestData(c.element())); + } + } + + private List> provideTestData( + int noOfShards, + int noOfEventsPerShard) { + + int seqNumber = 0; + + List> shardedData = newArrayList(); + for (int i = 0; i < noOfShards; ++i) { + List shardData = newArrayList(); + shardedData.add(shardData); + + DateTime arrival = DateTime.now(); + for (int j = 0; j < noOfEventsPerShard; ++j) { + arrival = arrival.plusSeconds(1); + + seqNumber++; + shardData.add(new AmazonKinesisMock.TestData( + Integer.toString(seqNumber), + arrival.toInstant(), + Integer.toString(seqNumber)) + ); + } + } + + return shardedData; + } +} diff --git a/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/KinesisReaderCheckpointTest.java b/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/KinesisReaderCheckpointTest.java new file mode 100644 index 0000000000000..205f050bb8ac2 --- /dev/null +++ b/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/KinesisReaderCheckpointTest.java @@ -0,0 +1,67 @@ +/* + * 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.kinesis; + + +import com.google.common.collect.Iterables; + +import static org.assertj.core.api.Assertions.assertThat; +import org.junit.Before; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.mockito.Mock; +import org.mockito.runners.MockitoJUnitRunner; +import static java.util.Arrays.asList; +import java.util.Iterator; +import java.util.List; + +/*** + * + */ +@RunWith(MockitoJUnitRunner.class) +public class KinesisReaderCheckpointTest { + @Mock + private ShardCheckpoint a, b, c; + + private KinesisReaderCheckpoint checkpoint; + + @Before + public void setUp() { + checkpoint = new KinesisReaderCheckpoint(asList(a, b, c)); + } + + @Test + public void splitsCheckpointAccordingly() { + verifySplitInto(1); + verifySplitInto(2); + verifySplitInto(3); + verifySplitInto(4); + } + + @Test(expected = UnsupportedOperationException.class) + public void isImmutable() { + Iterator iterator = checkpoint.iterator(); + iterator.remove(); + } + + private void verifySplitInto(int size) { + List split = checkpoint.splitInto(size); + assertThat(Iterables.concat(split)).containsOnly(a, b, c); + assertThat(split).hasSize(Math.min(size, 3)); + } +} 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 new file mode 100644 index 0000000000000..fbc7c66aa3b93 --- /dev/null +++ b/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/KinesisReaderIT.java @@ -0,0 +1,119 @@ +/* + * 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.kinesis; + +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; +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 static com.google.common.base.Preconditions.checkNotNull; +import static com.google.common.collect.Lists.newArrayList; + +import com.amazonaws.regions.Regions; +import static org.assertj.core.api.Assertions.assertThat; +import org.apache.commons.lang.RandomStringUtils; +import org.joda.time.Duration; +import org.joda.time.Instant; +import org.junit.Ignore; +import org.junit.Test; +import static java.util.concurrent.Executors.newSingleThreadExecutor; +import java.io.IOException; +import java.nio.charset.StandardCharsets; +import java.util.List; +import java.util.concurrent.Callable; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Future; +import java.util.concurrent.TimeUnit; + +/** + * Integration test, that reads from the real Kinesis. + * You need to provide all {@link KinesisTestOptions} in order to run this. + */ +public class KinesisReaderIT { + private static final long PIPELINE_STARTUP_TIME = TimeUnit.SECONDS.toMillis(10); + private ExecutorService singleThreadExecutor = newSingleThreadExecutor(); + + + @Ignore + @Test + public void readsDataFromRealKinesisStream() + throws IOException, InterruptedException, ExecutionException { + KinesisTestOptions options = readKinesisOptions(); + List testData = prepareTestData(1000); + + Future future = startTestPipeline(testData, options); + KinesisUploader.uploadAll(testData, options); + future.get(); + } + + private List prepareTestData(int count) { + List data = newArrayList(); + for (int i = 0; i < count; ++i) { + data.add(RandomStringUtils.randomAlphabetic(32)); + } + return data; + } + + 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()). + using(options.getAwsAccessKey(), options.getAwsSecretKey(), + Regions.fromName(options.getAwsKinesisRegion())). + withMaxReadTime(Duration.standardMinutes(3)) + ). + apply(ParDo.of(new RecordDataToString())); + PAssert.that(result).containsInAnyOrder(testData); + + Future future = singleThreadExecutor.submit(new Callable() { + @Override + public Void call() throws Exception { + PipelineResult result = p.run(); + PipelineResult.State state = result.getState(); + while (state != PipelineResult.State.DONE && state != PipelineResult.State.FAILED) { + Thread.sleep(1000); + state = result.getState(); + } + assertThat(state).isEqualTo(PipelineResult.State.DONE); + return null; + } + }); + Thread.sleep(PIPELINE_STARTUP_TIME); + return future; + } + + private KinesisTestOptions readKinesisOptions() { + PipelineOptionsFactory.register(KinesisTestOptions.class); + return TestPipeline.testingPipelineOptions().as(KinesisTestOptions.class); + } + + private static class RecordDataToString extends DoFn { + @ProcessElement + public void processElement(ProcessContext c) throws Exception { + checkNotNull(c.element(), "Null record given"); + c.output(new String(c.element().getData().array(), StandardCharsets.UTF_8)); + } + } +} diff --git a/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/KinesisReaderTest.java b/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/KinesisReaderTest.java new file mode 100644 index 0000000000000..793fb579c26b5 --- /dev/null +++ b/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/KinesisReaderTest.java @@ -0,0 +1,119 @@ +/* + * 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.kinesis; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.mockito.Mockito.when; +import org.junit.Before; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.mockito.Mock; +import org.mockito.runners.MockitoJUnitRunner; +import static java.util.Arrays.asList; +import java.io.IOException; +import java.util.NoSuchElementException; + +/** + * Created by ppastuszka on 12.12.15. + */ +@RunWith(MockitoJUnitRunner.class) +public class KinesisReaderTest { + @Mock + private SimplifiedKinesisClient kinesis; + @Mock + private CheckpointGenerator generator; + @Mock + private ShardCheckpoint firstCheckpoint, secondCheckpoint; + @Mock + private ShardRecordsIterator firstIterator, secondIterator; + @Mock + private KinesisRecord a, b, c, d; + + private KinesisReader reader; + + @Before + public void setUp() throws IOException, TransientKinesisException { + when(generator.generate(kinesis)).thenReturn(new KinesisReaderCheckpoint( + asList(firstCheckpoint, secondCheckpoint) + )); + when(firstCheckpoint.getShardRecordsIterator(kinesis)).thenReturn(firstIterator); + when(secondCheckpoint.getShardRecordsIterator(kinesis)).thenReturn(secondIterator); + when(firstIterator.next()).thenReturn(CustomOptional.absent()); + when(secondIterator.next()).thenReturn(CustomOptional.absent()); + + reader = new KinesisReader(kinesis, generator, null); + } + + @Test + public void startReturnsFalseIfNoDataAtTheBeginning() throws IOException { + assertThat(reader.start()).isFalse(); + } + + @Test(expected = NoSuchElementException.class) + public void throwsNoSuchElementExceptionIfNoData() throws IOException { + reader.start(); + reader.getCurrent(); + } + + @Test + public void startReturnsTrueIfSomeDataAvailable() throws IOException, + TransientKinesisException { + when(firstIterator.next()). + thenReturn(CustomOptional.of(a)). + thenReturn(CustomOptional.absent()); + + assertThat(reader.start()).isTrue(); + } + + @Test + public void advanceReturnsFalseIfThereIsTransientExceptionInKinesis() + throws IOException, TransientKinesisException { + reader.start(); + + when(firstIterator.next()).thenThrow(TransientKinesisException.class); + + assertThat(reader.advance()).isFalse(); + } + + @Test + public void readsThroughAllDataAvailable() throws IOException, TransientKinesisException { + when(firstIterator.next()). + thenReturn(CustomOptional.absent()). + thenReturn(CustomOptional.of(a)). + thenReturn(CustomOptional.absent()). + thenReturn(CustomOptional.of(b)). + thenReturn(CustomOptional.absent()); + + when(secondIterator.next()). + thenReturn(CustomOptional.of(c)). + thenReturn(CustomOptional.absent()). + thenReturn(CustomOptional.of(d)). + thenReturn(CustomOptional.absent()); + + assertThat(reader.start()).isTrue(); + assertThat(reader.getCurrent()).isEqualTo(c); + assertThat(reader.advance()).isTrue(); + assertThat(reader.getCurrent()).isEqualTo(a); + assertThat(reader.advance()).isTrue(); + assertThat(reader.getCurrent()).isEqualTo(d); + assertThat(reader.advance()).isTrue(); + assertThat(reader.getCurrent()).isEqualTo(b); + assertThat(reader.advance()).isFalse(); + } + +} diff --git a/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/KinesisRecordCoderTest.java b/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/KinesisRecordCoderTest.java new file mode 100644 index 0000000000000..b09b7eb9b378d --- /dev/null +++ b/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/KinesisRecordCoderTest.java @@ -0,0 +1,46 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.sdk.io.kinesis; + +import org.apache.beam.sdk.testing.CoderProperties; + +import org.joda.time.Instant; +import org.junit.Test; +import java.nio.ByteBuffer; + +/** + * Created by p.pastuszka on 20.07.2016. + */ +public class KinesisRecordCoderTest { + @Test + public void encodingAndDecodingWorks() throws Exception { + KinesisRecord record = new KinesisRecord( + ByteBuffer.wrap("data".getBytes()), + "sequence", + 128L, + "partition", + Instant.now(), + Instant.now(), + "stream", + "shard" + ); + CoderProperties.coderDecodeEncodeEqual( + new KinesisRecordCoder(), record + ); + } +} diff --git a/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/KinesisTestOptions.java b/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/KinesisTestOptions.java new file mode 100644 index 0000000000000..65a7605ea46a9 --- /dev/null +++ b/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/KinesisTestOptions.java @@ -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. + */ +package org.apache.beam.sdk.io.kinesis; + +import org.apache.beam.sdk.options.Default; +import org.apache.beam.sdk.options.Description; +import org.apache.beam.sdk.testing.TestPipelineOptions; + +/*** + * Options for Kinesis integration tests. + */ +public interface KinesisTestOptions extends TestPipelineOptions { + @Description("AWS region where Kinesis stream resided") + @Default.String("aws-kinesis-region") + String getAwsKinesisRegion(); + void setAwsKinesisRegion(String value); + + @Description("Kinesis stream name") + @Default.String("aws-kinesis-stream") + String getAwsKinesisStream(); + void setAwsKinesisStream(String value); + + @Description("AWS secret key") + @Default.String("aws-secret-key") + String getAwsSecretKey(); + void setAwsSecretKey(String value); + + @Description("AWS access key") + @Default.String("aws-access-key") + String getAwsAccessKey(); + void setAwsAccessKey(String value); +} diff --git a/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/KinesisUploader.java b/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/KinesisUploader.java new file mode 100644 index 0000000000000..0dcede9856713 --- /dev/null +++ b/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/KinesisUploader.java @@ -0,0 +1,84 @@ +/* + * 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.kinesis; + +import static com.google.common.collect.Lists.newArrayList; +import com.google.common.base.Charsets; +import com.google.common.collect.Lists; + +import com.amazonaws.auth.BasicAWSCredentials; +import com.amazonaws.internal.StaticCredentialsProvider; +import com.amazonaws.regions.Regions; +import com.amazonaws.services.kinesis.AmazonKinesis; +import com.amazonaws.services.kinesis.AmazonKinesisClient; +import com.amazonaws.services.kinesis.model.PutRecordsRequest; +import com.amazonaws.services.kinesis.model.PutRecordsRequestEntry; +import com.amazonaws.services.kinesis.model.PutRecordsResult; +import com.amazonaws.services.kinesis.model.PutRecordsResultEntry; +import java.nio.ByteBuffer; +import java.util.List; + +/*** + * Sends records to Kinesis in reliable way. + */ +public class KinesisUploader { + + public static final int MAX_NUMBER_OF_RECORDS_IN_BATCH = 499; + + public static void uploadAll(List data, KinesisTestOptions options) { + AmazonKinesis client = new AmazonKinesisClient( + new StaticCredentialsProvider( + new BasicAWSCredentials( + options.getAwsAccessKey(), options.getAwsSecretKey())) + ).withRegion(Regions.fromName(options.getAwsKinesisRegion())); + + List> partitions = Lists.partition(data, MAX_NUMBER_OF_RECORDS_IN_BATCH); + + + for (List partition : partitions) { + List allRecords = newArrayList(); + for (String row : partition) { + allRecords.add(new PutRecordsRequestEntry(). + withData(ByteBuffer.wrap(row.getBytes(Charsets.UTF_8))). + withPartitionKey(Integer.toString(row.hashCode())) + + ); + } + + PutRecordsResult result; + do { + result = client.putRecords( + new PutRecordsRequest(). + withStreamName(options.getAwsKinesisStream()). + withRecords(allRecords)); + List failedRecords = newArrayList(); + int i = 0; + for (PutRecordsResultEntry row : result.getRecords()) { + if (row.getErrorCode() != null) { + failedRecords.add(allRecords.get(i)); + } + ++i; + } + allRecords = failedRecords; + } + + while (result.getFailedRecordCount() > 0); + } + } + +} diff --git a/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/RecordFilterTest.java b/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/RecordFilterTest.java new file mode 100644 index 0000000000000..360106d00c4b3 --- /dev/null +++ b/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/RecordFilterTest.java @@ -0,0 +1,66 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.sdk.io.kinesis; + +import com.google.common.collect.Lists; + +import static org.mockito.BDDMockito.given; +import org.assertj.core.api.Assertions; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.mockito.Mock; +import org.mockito.runners.MockitoJUnitRunner; +import java.util.Collections; +import java.util.List; + + +/*** + */ +@RunWith(MockitoJUnitRunner.class) +public class RecordFilterTest { + @Mock + private ShardCheckpoint checkpoint; + @Mock + private KinesisRecord record1, record2, record3, record4, record5; + + @Test + public void shouldFilterOutRecordsBeforeOrAtCheckpoint() { + given(checkpoint.isBeforeOrAt(record1)).willReturn(false); + given(checkpoint.isBeforeOrAt(record2)).willReturn(true); + given(checkpoint.isBeforeOrAt(record3)).willReturn(true); + given(checkpoint.isBeforeOrAt(record4)).willReturn(false); + given(checkpoint.isBeforeOrAt(record5)).willReturn(true); + List records = Lists.newArrayList(record1, record2, + record3, record4, record5); + RecordFilter underTest = new RecordFilter(); + + List retainedRecords = underTest.apply(records, checkpoint); + + Assertions.assertThat(retainedRecords).containsOnly(record2, record3, record5); + } + + @Test + public void shouldNotFailOnEmptyList() { + List records = Collections.emptyList(); + RecordFilter underTest = new RecordFilter(); + + List retainedRecords = underTest.apply(records, checkpoint); + + Assertions.assertThat(retainedRecords).isEmpty(); + } +} diff --git a/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/RoundRobinTest.java b/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/RoundRobinTest.java new file mode 100644 index 0000000000000..a508ddf5ce0fb --- /dev/null +++ b/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/RoundRobinTest.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.sdk.io.kinesis; + +import static com.google.common.collect.Lists.newArrayList; + +import static org.assertj.core.api.Assertions.assertThat; +import org.junit.Test; +import java.util.Collections; +import java.util.List; + +/** + * Created by ppastuszka on 12.12.15. + */ +public class RoundRobinTest { + @Test(expected = IllegalArgumentException.class) + public void doesNotAllowCreationWithEmptyCollection() { + new RoundRobin<>(Collections.emptyList()); + } + + @Test + public void goesThroughElementsInCycle() { + List input = newArrayList("a", "b", "c"); + + RoundRobin roundRobin = new RoundRobin<>(newArrayList(input)); + + input.addAll(input); // duplicate the input + for (String element : input) { + assertThat(roundRobin.getCurrent()).isEqualTo(element); + assertThat(roundRobin.getCurrent()).isEqualTo(element); + roundRobin.moveForward(); + } + } + + @Test + public void usualIteratorGoesThroughElementsOnce() { + List input = newArrayList("a", "b", "c"); + + RoundRobin roundRobin = new RoundRobin<>(input); + assertThat(roundRobin).hasSize(3).containsOnly(input.toArray(new String[0])); + } +} diff --git a/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/ShardCheckpointTest.java b/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/ShardCheckpointTest.java new file mode 100644 index 0000000000000..2227cef638285 --- /dev/null +++ b/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/ShardCheckpointTest.java @@ -0,0 +1,148 @@ +/* + * 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.kinesis; + +import static com.amazonaws.services.kinesis.clientlibrary.lib.worker.InitialPositionInStream.LATEST; +import static com.amazonaws.services.kinesis.clientlibrary.lib.worker.InitialPositionInStream.TRIM_HORIZON; +import static com.amazonaws.services.kinesis.model.ShardIteratorType.AFTER_SEQUENCE_NUMBER; +import static com.amazonaws.services.kinesis.model.ShardIteratorType.AT_SEQUENCE_NUMBER; +import static com.amazonaws.services.kinesis.model.ShardIteratorType.AT_TIMESTAMP; +import com.amazonaws.services.kinesis.clientlibrary.types.ExtendedSequenceNumber; +import com.amazonaws.services.kinesis.model.ShardIteratorType; +import static org.assertj.core.api.Assertions.assertThat; +import static org.mockito.BDDMockito.given; +import static org.mockito.Matchers.anyString; +import static org.mockito.Matchers.eq; +import static org.mockito.Matchers.isNull; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; +import org.joda.time.DateTime; +import org.joda.time.Instant; +import org.junit.Before; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.mockito.Mock; +import org.mockito.runners.MockitoJUnitRunner; +import java.io.IOException; + +/** + * + */ +@RunWith(MockitoJUnitRunner.class) +public class ShardCheckpointTest { + private static final String AT_SEQUENCE_SHARD_IT = "AT_SEQUENCE_SHARD_IT"; + private static final String AFTER_SEQUENCE_SHARD_IT = "AFTER_SEQUENCE_SHARD_IT"; + private static final String STREAM_NAME = "STREAM"; + private static final String SHARD_ID = "SHARD_ID"; + @Mock + private SimplifiedKinesisClient client; + + @Before + public void setUp() throws IOException, TransientKinesisException { + when(client.getShardIterator( + eq(STREAM_NAME), eq(SHARD_ID), eq(AT_SEQUENCE_NUMBER), + anyString(), isNull(Instant.class))). + thenReturn(AT_SEQUENCE_SHARD_IT); + when(client.getShardIterator( + eq(STREAM_NAME), eq(SHARD_ID), eq(AFTER_SEQUENCE_NUMBER), + anyString(), isNull(Instant.class))). + thenReturn(AFTER_SEQUENCE_SHARD_IT); + } + + @Test + public void testProvidingShardIterator() throws IOException, TransientKinesisException { + assertThat(checkpoint(AT_SEQUENCE_NUMBER, "100", null).getShardIterator(client)) + .isEqualTo(AT_SEQUENCE_SHARD_IT); + assertThat(checkpoint(AFTER_SEQUENCE_NUMBER, "100", null).getShardIterator(client)) + .isEqualTo(AFTER_SEQUENCE_SHARD_IT); + assertThat(checkpoint(AT_SEQUENCE_NUMBER, "100", 10L).getShardIterator(client)).isEqualTo + (AT_SEQUENCE_SHARD_IT); + assertThat(checkpoint(AFTER_SEQUENCE_NUMBER, "100", 10L).getShardIterator(client)) + .isEqualTo(AT_SEQUENCE_SHARD_IT); + } + + @Test + public void testComparisonWithExtendedSequenceNumber() { + assertThat(new ShardCheckpoint("", "", new StartingPoint(LATEST)).isBeforeOrAt( + recordWith(new ExtendedSequenceNumber("100", 0L)) + )).isTrue(); + + assertThat(new ShardCheckpoint("", "", new StartingPoint(TRIM_HORIZON)).isBeforeOrAt( + recordWith(new ExtendedSequenceNumber("100", 0L)) + )).isTrue(); + + assertThat(checkpoint(AFTER_SEQUENCE_NUMBER, "10", 1L).isBeforeOrAt( + recordWith(new ExtendedSequenceNumber("100", 0L)) + )).isTrue(); + + assertThat(checkpoint(AT_SEQUENCE_NUMBER, "100", 0L).isBeforeOrAt( + recordWith(new ExtendedSequenceNumber("100", 0L)) + )).isTrue(); + + assertThat(checkpoint(AFTER_SEQUENCE_NUMBER, "100", 0L).isBeforeOrAt( + recordWith(new ExtendedSequenceNumber("100", 0L)) + )).isFalse(); + + assertThat(checkpoint(AT_SEQUENCE_NUMBER, "100", 1L).isBeforeOrAt( + recordWith(new ExtendedSequenceNumber("100", 0L)) + )).isFalse(); + + assertThat(checkpoint(AFTER_SEQUENCE_NUMBER, "100", 0L).isBeforeOrAt( + recordWith(new ExtendedSequenceNumber("99", 1L)) + )).isFalse(); + } + + @Test + public void testComparisonWithTimestamp() { + DateTime referenceTimestamp = DateTime.now(); + + assertThat(checkpoint(AT_TIMESTAMP, referenceTimestamp.toInstant()) + .isBeforeOrAt(recordWith(referenceTimestamp.minusMillis(10).toInstant())) + ).isFalse(); + + assertThat(checkpoint(AT_TIMESTAMP, referenceTimestamp.toInstant()) + .isBeforeOrAt(recordWith(referenceTimestamp.toInstant())) + ).isTrue(); + + assertThat(checkpoint(AT_TIMESTAMP, referenceTimestamp.toInstant()) + .isBeforeOrAt(recordWith(referenceTimestamp.plusMillis(10).toInstant())) + ).isTrue(); + } + + private KinesisRecord recordWith(ExtendedSequenceNumber extendedSequenceNumber) { + KinesisRecord record = mock(KinesisRecord.class); + given(record.getExtendedSequenceNumber()).willReturn(extendedSequenceNumber); + return record; + } + + private ShardCheckpoint checkpoint(ShardIteratorType iteratorType, String sequenceNumber, + Long subSequenceNumber) { + return new ShardCheckpoint(STREAM_NAME, SHARD_ID, iteratorType, sequenceNumber, + subSequenceNumber); + } + + private KinesisRecord recordWith(Instant approximateArrivalTimestamp) { + KinesisRecord record = mock(KinesisRecord.class); + given(record.getApproximateArrivalTimestamp()).willReturn(approximateArrivalTimestamp); + return record; + } + + private ShardCheckpoint checkpoint(ShardIteratorType iteratorType, Instant timestamp) { + return new ShardCheckpoint(STREAM_NAME, SHARD_ID, iteratorType, timestamp); + } +} diff --git a/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/ShardRecordsIteratorTest.java b/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/ShardRecordsIteratorTest.java new file mode 100644 index 0000000000000..e2a3ccc0057e3 --- /dev/null +++ b/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/ShardRecordsIteratorTest.java @@ -0,0 +1,150 @@ +/* + * 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.kinesis; + +import com.amazonaws.services.kinesis.model.ExpiredIteratorException; +import static org.assertj.core.api.Assertions.assertThat; +import static org.mockito.Matchers.any; +import static org.mockito.Matchers.anyListOf; +import static org.mockito.Mockito.when; +import org.junit.Before; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.mockito.Mock; +import org.mockito.invocation.InvocationOnMock; +import org.mockito.runners.MockitoJUnitRunner; +import org.mockito.stubbing.Answer; +import static java.util.Arrays.asList; +import static java.util.Collections.singletonList; +import java.io.IOException; +import java.util.Collections; + +/** + * Created by ppastuszka on 12.12.15. + */ +@RunWith(MockitoJUnitRunner.class) +public class ShardRecordsIteratorTest { + private static final String INITIAL_ITERATOR = "INITIAL_ITERATOR"; + private static final String SECOND_ITERATOR = "SECOND_ITERATOR"; + private static final String SECOND_REFRESHED_ITERATOR = "SECOND_REFRESHED_ITERATOR"; + private static final String THIRD_ITERATOR = "THIRD_ITERATOR"; + private static final String STREAM_NAME = "STREAM_NAME"; + private static final String SHARD_ID = "SHARD_ID"; + + @Mock + private SimplifiedKinesisClient kinesisClient; + @Mock + private ShardCheckpoint firstCheckpoint, aCheckpoint, bCheckpoint, cCheckpoint, dCheckpoint; + @Mock + private GetKinesisRecordsResult firstResult, secondResult, thirdResult; + @Mock + private KinesisRecord a, b, c, d; + @Mock + private RecordFilter recordFilter; + + private ShardRecordsIterator iterator; + + @Before + public void setUp() throws IOException, TransientKinesisException { + when(firstCheckpoint.getShardIterator(kinesisClient)).thenReturn(INITIAL_ITERATOR); + when(firstCheckpoint.getStreamName()).thenReturn(STREAM_NAME); + when(firstCheckpoint.getShardId()).thenReturn(SHARD_ID); + + when(firstCheckpoint.moveAfter(a)).thenReturn(aCheckpoint); + when(aCheckpoint.moveAfter(b)).thenReturn(bCheckpoint); + when(aCheckpoint.getStreamName()).thenReturn(STREAM_NAME); + when(aCheckpoint.getShardId()).thenReturn(SHARD_ID); + when(bCheckpoint.moveAfter(c)).thenReturn(cCheckpoint); + when(bCheckpoint.getStreamName()).thenReturn(STREAM_NAME); + when(bCheckpoint.getShardId()).thenReturn(SHARD_ID); + when(cCheckpoint.moveAfter(d)).thenReturn(dCheckpoint); + when(cCheckpoint.getStreamName()).thenReturn(STREAM_NAME); + when(cCheckpoint.getShardId()).thenReturn(SHARD_ID); + when(dCheckpoint.getStreamName()).thenReturn(STREAM_NAME); + when(dCheckpoint.getShardId()).thenReturn(SHARD_ID); + + when(kinesisClient.getRecords(INITIAL_ITERATOR, STREAM_NAME, SHARD_ID)) + .thenReturn(firstResult); + when(kinesisClient.getRecords(SECOND_ITERATOR, STREAM_NAME, SHARD_ID)) + .thenReturn(secondResult); + when(kinesisClient.getRecords(THIRD_ITERATOR, STREAM_NAME, SHARD_ID)) + .thenReturn(thirdResult); + + when(firstResult.getNextShardIterator()).thenReturn(SECOND_ITERATOR); + when(secondResult.getNextShardIterator()).thenReturn(THIRD_ITERATOR); + when(thirdResult.getNextShardIterator()).thenReturn(THIRD_ITERATOR); + + when(firstResult.getRecords()).thenReturn(Collections.emptyList()); + when(secondResult.getRecords()).thenReturn(Collections.emptyList()); + when(thirdResult.getRecords()).thenReturn(Collections.emptyList()); + + when(recordFilter.apply(anyListOf(KinesisRecord.class), any(ShardCheckpoint + .class))).thenAnswer(new IdentityAnswer()); + + iterator = new ShardRecordsIterator(firstCheckpoint, kinesisClient, recordFilter); + } + + @Test + public void returnsAbsentIfNoRecordsPresent() throws IOException, TransientKinesisException { + assertThat(iterator.next()).isEqualTo(CustomOptional.absent()); + assertThat(iterator.next()).isEqualTo(CustomOptional.absent()); + assertThat(iterator.next()).isEqualTo(CustomOptional.absent()); + } + + @Test + public void goesThroughAvailableRecords() throws IOException, TransientKinesisException { + when(firstResult.getRecords()).thenReturn(asList(a, b, c)); + when(secondResult.getRecords()).thenReturn(singletonList(d)); + + assertThat(iterator.getCheckpoint()).isEqualTo(firstCheckpoint); + assertThat(iterator.next()).isEqualTo(CustomOptional.of(a)); + assertThat(iterator.getCheckpoint()).isEqualTo(aCheckpoint); + assertThat(iterator.next()).isEqualTo(CustomOptional.of(b)); + assertThat(iterator.getCheckpoint()).isEqualTo(bCheckpoint); + assertThat(iterator.next()).isEqualTo(CustomOptional.of(c)); + assertThat(iterator.getCheckpoint()).isEqualTo(cCheckpoint); + assertThat(iterator.next()).isEqualTo(CustomOptional.of(d)); + assertThat(iterator.getCheckpoint()).isEqualTo(dCheckpoint); + assertThat(iterator.next()).isEqualTo(CustomOptional.absent()); + assertThat(iterator.getCheckpoint()).isEqualTo(dCheckpoint); + } + + @Test + public void refreshesExpiredIterator() throws IOException, TransientKinesisException { + when(firstResult.getRecords()).thenReturn(singletonList(a)); + when(secondResult.getRecords()).thenReturn(singletonList(b)); + + when(kinesisClient.getRecords(SECOND_ITERATOR, STREAM_NAME, SHARD_ID)) + .thenThrow(ExpiredIteratorException.class); + when(aCheckpoint.getShardIterator(kinesisClient)) + .thenReturn(SECOND_REFRESHED_ITERATOR); + when(kinesisClient.getRecords(SECOND_REFRESHED_ITERATOR, STREAM_NAME, SHARD_ID)) + .thenReturn(secondResult); + + assertThat(iterator.next()).isEqualTo(CustomOptional.of(a)); + assertThat(iterator.next()).isEqualTo(CustomOptional.of(b)); + assertThat(iterator.next()).isEqualTo(CustomOptional.absent()); + } + + private static class IdentityAnswer implements Answer { + @Override + public Object answer(InvocationOnMock invocation) throws Throwable { + return invocation.getArguments()[0]; + } + } +} diff --git a/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/SimplifiedKinesisClientTest.java b/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/SimplifiedKinesisClientTest.java new file mode 100644 index 0000000000000..44d29d6cf8fd7 --- /dev/null +++ b/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/SimplifiedKinesisClientTest.java @@ -0,0 +1,223 @@ +/* + * 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.kinesis; + +import com.amazonaws.AmazonServiceException; +import com.amazonaws.AmazonServiceException.ErrorType; +import com.amazonaws.services.kinesis.AmazonKinesis; +import com.amazonaws.services.kinesis.model.DescribeStreamResult; +import com.amazonaws.services.kinesis.model.ExpiredIteratorException; +import com.amazonaws.services.kinesis.model.GetShardIteratorRequest; +import com.amazonaws.services.kinesis.model.GetShardIteratorResult; +import com.amazonaws.services.kinesis.model.LimitExceededException; +import com.amazonaws.services.kinesis.model.ProvisionedThroughputExceededException; +import com.amazonaws.services.kinesis.model.Shard; +import com.amazonaws.services.kinesis.model.ShardIteratorType; +import com.amazonaws.services.kinesis.model.StreamDescription; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.failBecauseExceptionWasNotThrown; +import static org.mockito.BDDMockito.given; +import static org.mockito.Mockito.reset; +import org.joda.time.Instant; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.mockito.InjectMocks; +import org.mockito.Mock; +import org.mockito.runners.MockitoJUnitRunner; +import java.util.List; + +/*** + */ +@RunWith(MockitoJUnitRunner.class) +public class SimplifiedKinesisClientTest { + private static final String STREAM = "stream"; + private static final String SHARD_1 = "shard-01"; + private static final String SHARD_2 = "shard-02"; + private static final String SHARD_3 = "shard-03"; + private static final String SHARD_ITERATOR = "iterator"; + private static final String SEQUENCE_NUMBER = "abc123"; + + @Mock + private AmazonKinesis kinesis; + @InjectMocks + private SimplifiedKinesisClient underTest; + + @Test + public void shouldReturnIteratorStartingWithSequenceNumber() throws Exception { + given(kinesis.getShardIterator(new GetShardIteratorRequest() + .withStreamName(STREAM) + .withShardId(SHARD_1) + .withShardIteratorType(ShardIteratorType.AT_SEQUENCE_NUMBER) + .withStartingSequenceNumber(SEQUENCE_NUMBER) + )).willReturn(new GetShardIteratorResult() + .withShardIterator(SHARD_ITERATOR)); + + String stream = underTest.getShardIterator(STREAM, SHARD_1, + ShardIteratorType.AT_SEQUENCE_NUMBER, SEQUENCE_NUMBER, null); + + assertThat(stream).isEqualTo(SHARD_ITERATOR); + } + + @Test + public void shouldReturnIteratorStartingWithTimestamp() throws Exception { + Instant timestamp = Instant.now(); + given(kinesis.getShardIterator(new GetShardIteratorRequest() + .withStreamName(STREAM) + .withShardId(SHARD_1) + .withShardIteratorType(ShardIteratorType.AT_SEQUENCE_NUMBER) + .withTimestamp(timestamp.toDate()) + )).willReturn(new GetShardIteratorResult() + .withShardIterator(SHARD_ITERATOR)); + + String stream = underTest.getShardIterator(STREAM, SHARD_1, + ShardIteratorType.AT_SEQUENCE_NUMBER, null, timestamp); + + assertThat(stream).isEqualTo(SHARD_ITERATOR); + } + + @Test + public void shouldHandleExpiredIterationExceptionForGetShardIterator() { + shouldHandleGetShardIteratorError(new ExpiredIteratorException(""), + ExpiredIteratorException.class); + } + + @Test + public void shouldHandleLimitExceededExceptionForGetShardIterator() { + shouldHandleGetShardIteratorError(new LimitExceededException(""), + TransientKinesisException.class); + } + + @Test + public void shouldHandleProvisionedThroughputExceededExceptionForGetShardIterator() { + shouldHandleGetShardIteratorError(new ProvisionedThroughputExceededException(""), + TransientKinesisException.class); + } + + @Test + public void shouldHandleServiceErrorForGetShardIterator() { + shouldHandleGetShardIteratorError(newAmazonServiceException(ErrorType.Service), + TransientKinesisException.class); + } + + @Test + public void shouldHandleClientErrorForGetShardIterator() { + shouldHandleGetShardIteratorError(newAmazonServiceException(ErrorType.Client), + RuntimeException.class); + } + + @Test + public void shouldHandleUnexpectedExceptionForGetShardIterator() { + shouldHandleGetShardIteratorError(new NullPointerException(), + RuntimeException.class); + } + + private void shouldHandleGetShardIteratorError( + Exception thrownException, + Class expectedExceptionClass) { + GetShardIteratorRequest request = new GetShardIteratorRequest() + .withStreamName(STREAM) + .withShardId(SHARD_1) + .withShardIteratorType(ShardIteratorType.LATEST); + + given(kinesis.getShardIterator(request)).willThrow(thrownException); + + try { + underTest.getShardIterator(STREAM, SHARD_1, ShardIteratorType.LATEST, null, null); + failBecauseExceptionWasNotThrown(expectedExceptionClass); + } catch (Exception e) { + assertThat(e).isExactlyInstanceOf(expectedExceptionClass); + } finally { + reset(kinesis); + } + } + + @Test + public void shouldListAllShards() throws Exception { + Shard shard1 = new Shard().withShardId(SHARD_1); + Shard shard2 = new Shard().withShardId(SHARD_2); + Shard shard3 = new Shard().withShardId(SHARD_3); + given(kinesis.describeStream(STREAM, null)).willReturn(new DescribeStreamResult() + .withStreamDescription(new StreamDescription() + .withShards(shard1, shard2) + .withHasMoreShards(true))); + given(kinesis.describeStream(STREAM, SHARD_2)).willReturn(new DescribeStreamResult() + .withStreamDescription(new StreamDescription() + .withShards(shard3) + .withHasMoreShards(false))); + + List shards = underTest.listShards(STREAM); + + assertThat(shards).containsOnly(shard1, shard2, shard3); + } + + @Test + public void shouldHandleExpiredIterationExceptionForShardListing() { + shouldHandleShardListingError(new ExpiredIteratorException(""), + ExpiredIteratorException.class); + } + + @Test + public void shouldHandleLimitExceededExceptionForShardListing() { + shouldHandleShardListingError(new LimitExceededException(""), + TransientKinesisException.class); + } + + @Test + public void shouldHandleProvisionedThroughputExceededExceptionForShardListing() { + shouldHandleShardListingError(new ProvisionedThroughputExceededException(""), + TransientKinesisException.class); + } + + @Test + public void shouldHandleServiceErrorForShardListing() { + shouldHandleShardListingError(newAmazonServiceException(ErrorType.Service), + TransientKinesisException.class); + } + + @Test + public void shouldHandleClientErrorForShardListing() { + shouldHandleShardListingError(newAmazonServiceException(ErrorType.Client), + RuntimeException.class); + } + + @Test + public void shouldHandleUnexpectedExceptionForShardListing() { + shouldHandleShardListingError(new NullPointerException(), + RuntimeException.class); + } + + private void shouldHandleShardListingError( + Exception thrownException, + Class expectedExceptionClass) { + given(kinesis.describeStream(STREAM, null)).willThrow(thrownException); + try { + underTest.listShards(STREAM); + failBecauseExceptionWasNotThrown(expectedExceptionClass); + } catch (Exception e) { + assertThat(e).isExactlyInstanceOf(expectedExceptionClass); + } finally { + reset(kinesis); + } + } + + private AmazonServiceException newAmazonServiceException(ErrorType errorType) { + AmazonServiceException exception = new AmazonServiceException(""); + exception.setErrorType(errorType); + return exception; + } +} diff --git a/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/package-info.java b/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/package-info.java new file mode 100644 index 0000000000000..44dbf4a92cd27 --- /dev/null +++ b/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/package-info.java @@ -0,0 +1,22 @@ +/* + * 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. + */ + +/** + * Transforms for reading and writing from Amazon Kinesis. + */ +package org.apache.beam.sdk.io.kinesis; diff --git a/sdks/java/io/pom.xml b/sdks/java/io/pom.xml index 419849956cea6..6cbd615a1d6e9 100644 --- a/sdks/java/io/pom.xml +++ b/sdks/java/io/pom.xml @@ -37,6 +37,7 @@ hdfs jms kafka + kinesis From aee5fbf0f259dee90618319947eb2aad7024078b Mon Sep 17 00:00:00 2001 From: Dan Halperin Date: Fri, 26 Aug 2016 15:17:05 -0700 Subject: [PATCH 081/346] Organize imports in Kinesis --- .../kinesis/DynamicCheckpointGenerator.java | 4 +-- .../io/kinesis/GetKinesisRecordsResult.java | 2 +- .../apache/beam/sdk/io/kinesis/KinesisIO.java | 2 +- .../beam/sdk/io/kinesis/KinesisReader.java | 8 +++--- .../io/kinesis/KinesisReaderCheckpoint.java | 4 +-- .../beam/sdk/io/kinesis/KinesisRecord.java | 8 +++--- .../sdk/io/kinesis/KinesisRecordCoder.java | 9 +++--- .../beam/sdk/io/kinesis/KinesisSource.java | 10 +++---- .../beam/sdk/io/kinesis/RoundRobin.java | 1 - .../beam/sdk/io/kinesis/ShardCheckpoint.java | 8 +++--- .../sdk/io/kinesis/ShardRecordsIterator.java | 2 +- .../io/kinesis/SimplifiedKinesisClient.java | 5 ++-- .../beam/sdk/io/kinesis/StartingPoint.java | 2 +- .../sdk/io/kinesis/AmazonKinesisMock.java | 12 ++++---- .../sdk/io/kinesis/CustomOptionalTest.java | 2 +- .../DynamicCheckpointGeneratorTest.java | 7 +++-- .../sdk/io/kinesis/KinesisMockReadTest.java | 11 ++++---- .../kinesis/KinesisReaderCheckpointTest.java | 10 +++---- .../beam/sdk/io/kinesis/KinesisReaderIT.java | 28 +++++++++---------- .../sdk/io/kinesis/KinesisReaderTest.java | 7 +++-- .../io/kinesis/KinesisRecordCoderTest.java | 3 +- .../beam/sdk/io/kinesis/KinesisUploader.java | 4 +-- .../beam/sdk/io/kinesis/RecordFilterTest.java | 8 +++--- .../beam/sdk/io/kinesis/RoundRobinTest.java | 4 +-- .../sdk/io/kinesis/ShardCheckpointTest.java | 7 +++-- .../io/kinesis/ShardRecordsIteratorTest.java | 11 ++++---- .../kinesis/SimplifiedKinesisClientTest.java | 11 ++++---- 27 files changed, 92 insertions(+), 98 deletions(-) diff --git a/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/DynamicCheckpointGenerator.java b/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/DynamicCheckpointGenerator.java index d86960f0d030e..2ec293cfeac21 100644 --- a/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/DynamicCheckpointGenerator.java +++ b/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/DynamicCheckpointGenerator.java @@ -20,10 +20,8 @@ import static com.google.common.base.Preconditions.checkNotNull; import static com.google.common.collect.Lists.transform; - -import com.google.common.base.Function; - import com.amazonaws.services.kinesis.model.Shard; +import com.google.common.base.Function; /** * Creates {@link KinesisReaderCheckpoint}, which spans over all shards in given stream. diff --git a/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/GetKinesisRecordsResult.java b/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/GetKinesisRecordsResult.java index f48b9d53ade85..c0f00de610c7b 100644 --- a/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/GetKinesisRecordsResult.java +++ b/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/GetKinesisRecordsResult.java @@ -18,9 +18,9 @@ package org.apache.beam.sdk.io.kinesis; import static com.google.common.collect.Lists.transform; -import com.google.common.base.Function; import com.amazonaws.services.kinesis.clientlibrary.types.UserRecord; +import com.google.common.base.Function; import java.util.List; import javax.annotation.Nullable; diff --git a/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/KinesisIO.java b/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/KinesisIO.java index b3cb464e1cfbf..811051c5fb0ab 100644 --- a/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/KinesisIO.java +++ b/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/KinesisIO.java @@ -18,7 +18,6 @@ package org.apache.beam.sdk.io.kinesis; -import org.apache.beam.sdk.transforms.PTransform; import static com.google.common.base.Preconditions.checkNotNull; import com.amazonaws.auth.AWSCredentialsProvider; @@ -28,6 +27,7 @@ import com.amazonaws.services.kinesis.AmazonKinesis; import com.amazonaws.services.kinesis.AmazonKinesisClient; import com.amazonaws.services.kinesis.clientlibrary.lib.worker.InitialPositionInStream; +import org.apache.beam.sdk.transforms.PTransform; import org.joda.time.Instant; /** diff --git a/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/KinesisReader.java b/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/KinesisReader.java index 38a0050fa72aa..219a7059207da 100644 --- a/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/KinesisReader.java +++ b/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/KinesisReader.java @@ -18,16 +18,16 @@ package org.apache.beam.sdk.io.kinesis; -import org.apache.beam.sdk.io.UnboundedSource; import static com.google.common.base.Preconditions.checkNotNull; import static com.google.common.collect.Lists.newArrayList; -import org.joda.time.Instant; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; import java.io.IOException; import java.util.List; import java.util.NoSuchElementException; +import org.apache.beam.sdk.io.UnboundedSource; +import org.joda.time.Instant; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /*** diff --git a/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/KinesisReaderCheckpoint.java b/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/KinesisReaderCheckpoint.java index 6ceb74295d863..663ba44b3340b 100644 --- a/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/KinesisReaderCheckpoint.java +++ b/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/KinesisReaderCheckpoint.java @@ -17,20 +17,18 @@ */ package org.apache.beam.sdk.io.kinesis; -import org.apache.beam.sdk.io.UnboundedSource; import static com.google.common.collect.Iterables.transform; import static com.google.common.collect.Lists.newArrayList; import static com.google.common.collect.Lists.partition; - import com.google.common.base.Function; import com.google.common.collect.ImmutableList; - import java.io.IOException; import java.io.Serializable; import java.util.Iterator; import java.util.List; import javax.annotation.Nullable; +import org.apache.beam.sdk.io.UnboundedSource; /*** * Checkpoint representing a total progress in a set of shards in single stream. diff --git a/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/KinesisRecord.java b/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/KinesisRecord.java index cdb495ce00783..fe2a33dec7988 100644 --- a/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/KinesisRecord.java +++ b/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/KinesisRecord.java @@ -17,15 +17,15 @@ */ package org.apache.beam.sdk.io.kinesis; -import com.google.common.base.Charsets; +import static org.apache.commons.lang.builder.HashCodeBuilder.reflectionHashCode; import com.amazonaws.services.kinesis.clientlibrary.types.ExtendedSequenceNumber; import com.amazonaws.services.kinesis.clientlibrary.types.UserRecord; -import static org.apache.commons.lang.builder.HashCodeBuilder.reflectionHashCode; -import org.apache.commons.lang.builder.EqualsBuilder; -import org.joda.time.Instant; +import com.google.common.base.Charsets; import java.io.Serializable; import java.nio.ByteBuffer; +import org.apache.commons.lang.builder.EqualsBuilder; +import org.joda.time.Instant; /** * {@link UserRecord} enhanced with utility methods. diff --git a/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/KinesisRecordCoder.java b/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/KinesisRecordCoder.java index c383a4ff8a91a..5b13e319691a4 100644 --- a/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/KinesisRecordCoder.java +++ b/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/KinesisRecordCoder.java @@ -17,18 +17,17 @@ */ package org.apache.beam.sdk.io.kinesis; +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.nio.ByteBuffer; import org.apache.beam.sdk.coders.AtomicCoder; import org.apache.beam.sdk.coders.ByteArrayCoder; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.InstantCoder; import org.apache.beam.sdk.coders.StringUtf8Coder; import org.apache.beam.sdk.coders.VarLongCoder; - import org.joda.time.Instant; -import java.io.IOException; -import java.io.InputStream; -import java.io.OutputStream; -import java.nio.ByteBuffer; /*** * A {@link Coder} for {@link KinesisRecord}. diff --git a/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/KinesisSource.java b/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/KinesisSource.java index 38c9fa4072ef3..62cba08498560 100644 --- a/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/KinesisSource.java +++ b/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/KinesisSource.java @@ -17,18 +17,16 @@ */ package org.apache.beam.sdk.io.kinesis; +import static com.google.common.base.Preconditions.checkNotNull; +import static com.google.common.collect.Lists.newArrayList; + +import java.util.List; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.SerializableCoder; import org.apache.beam.sdk.io.UnboundedSource; import org.apache.beam.sdk.options.PipelineOptions; - - -import static com.google.common.base.Preconditions.checkNotNull; -import static com.google.common.collect.Lists.newArrayList; - import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.util.List; /*** diff --git a/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/RoundRobin.java b/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/RoundRobin.java index 7257aa1492bbb..7adae4b6ffd98 100644 --- a/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/RoundRobin.java +++ b/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/RoundRobin.java @@ -20,7 +20,6 @@ import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.collect.Queues.newArrayDeque; - import java.util.Deque; import java.util.Iterator; diff --git a/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/ShardCheckpoint.java b/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/ShardCheckpoint.java index 1d8628bc8851e..9920aca0300ad 100644 --- a/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/ShardCheckpoint.java +++ b/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/ShardCheckpoint.java @@ -18,17 +18,17 @@ package org.apache.beam.sdk.io.kinesis; -import static com.google.common.base.Preconditions.checkArgument; -import static com.google.common.base.Preconditions.checkNotNull; - import static com.amazonaws.services.kinesis.model.ShardIteratorType.AFTER_SEQUENCE_NUMBER; import static com.amazonaws.services.kinesis.model.ShardIteratorType.AT_SEQUENCE_NUMBER; import static com.amazonaws.services.kinesis.model.ShardIteratorType.AT_TIMESTAMP; +import static com.google.common.base.Preconditions.checkArgument; +import static com.google.common.base.Preconditions.checkNotNull; + import com.amazonaws.services.kinesis.clientlibrary.types.ExtendedSequenceNumber; import com.amazonaws.services.kinesis.model.Record; import com.amazonaws.services.kinesis.model.ShardIteratorType; -import org.joda.time.Instant; import java.io.Serializable; +import org.joda.time.Instant; /*** diff --git a/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/ShardRecordsIterator.java b/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/ShardRecordsIterator.java index 7dfe158ca41db..d17996a192183 100644 --- a/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/ShardRecordsIterator.java +++ b/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/ShardRecordsIterator.java @@ -21,9 +21,9 @@ import static com.google.common.collect.Queues.newArrayDeque; import com.amazonaws.services.kinesis.model.ExpiredIteratorException; +import java.util.Deque; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.util.Deque; /*** * Iterates over records in a single shard. diff --git a/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/SimplifiedKinesisClient.java b/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/SimplifiedKinesisClient.java index f9a1ea2af8109..96267d14f0be9 100644 --- a/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/SimplifiedKinesisClient.java +++ b/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/SimplifiedKinesisClient.java @@ -18,8 +18,6 @@ package org.apache.beam.sdk.io.kinesis; -import com.google.common.collect.Lists; - import com.amazonaws.AmazonServiceException; import com.amazonaws.services.kinesis.AmazonKinesis; import com.amazonaws.services.kinesis.clientlibrary.types.UserRecord; @@ -32,10 +30,11 @@ import com.amazonaws.services.kinesis.model.Shard; import com.amazonaws.services.kinesis.model.ShardIteratorType; import com.amazonaws.services.kinesis.model.StreamDescription; -import org.joda.time.Instant; +import com.google.common.collect.Lists; import java.util.Date; import java.util.List; import java.util.concurrent.Callable; +import org.joda.time.Instant; /*** * Wraps {@link AmazonKinesis} class providing much simpler interface and diff --git a/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/StartingPoint.java b/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/StartingPoint.java index 8140269c170c2..b7ee91746fa3e 100644 --- a/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/StartingPoint.java +++ b/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/StartingPoint.java @@ -22,9 +22,9 @@ import com.amazonaws.services.kinesis.clientlibrary.lib.worker.InitialPositionInStream; import com.amazonaws.services.kinesis.model.ShardIteratorType; -import org.joda.time.Instant; import java.io.Serializable; import java.util.Objects; +import org.joda.time.Instant; /*** * Denotes a point at which the reader should start reading from a Kinesis stream. diff --git a/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/AmazonKinesisMock.java b/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/AmazonKinesisMock.java index 7ca8e0b4bd8dc..b007fa411db41 100644 --- a/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/AmazonKinesisMock.java +++ b/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/AmazonKinesisMock.java @@ -19,7 +19,9 @@ import static com.google.common.collect.Lists.newArrayList; import static com.google.common.collect.Lists.transform; -import com.google.common.base.Function; +import static java.lang.Integer.parseInt; +import static java.lang.Math.min; +import static org.apache.commons.lang.builder.HashCodeBuilder.reflectionHashCode; import com.amazonaws.AmazonWebServiceRequest; import com.amazonaws.ResponseMetadata; @@ -63,15 +65,13 @@ import com.amazonaws.services.kinesis.model.SplitShardRequest; import com.amazonaws.services.kinesis.model.SplitShardResult; import com.amazonaws.services.kinesis.model.StreamDescription; -import static org.apache.commons.lang.builder.HashCodeBuilder.reflectionHashCode; -import org.apache.commons.lang.builder.EqualsBuilder; -import org.joda.time.Instant; -import static java.lang.Integer.parseInt; -import static java.lang.Math.min; +import com.google.common.base.Function; import java.io.Serializable; import java.nio.ByteBuffer; import java.util.List; import javax.annotation.Nullable; +import org.apache.commons.lang.builder.EqualsBuilder; +import org.joda.time.Instant; /** * Created by p.pastuszka on 21.07.2016. diff --git a/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/CustomOptionalTest.java b/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/CustomOptionalTest.java index 152fd6dfde434..cb0d0e248f883 100644 --- a/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/CustomOptionalTest.java +++ b/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/CustomOptionalTest.java @@ -17,8 +17,8 @@ */ package org.apache.beam.sdk.io.kinesis; -import org.junit.Test; import java.util.NoSuchElementException; +import org.junit.Test; /** * Created by ppastuszka on 12.12.15. diff --git a/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/DynamicCheckpointGeneratorTest.java b/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/DynamicCheckpointGeneratorTest.java index a9e5a69a2d513..c92ac9a4b2939 100644 --- a/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/DynamicCheckpointGeneratorTest.java +++ b/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/DynamicCheckpointGeneratorTest.java @@ -17,15 +17,16 @@ */ package org.apache.beam.sdk.io.kinesis; -import com.amazonaws.services.kinesis.clientlibrary.lib.worker.InitialPositionInStream; -import com.amazonaws.services.kinesis.model.Shard; +import static java.util.Arrays.asList; import static org.assertj.core.api.Assertions.assertThat; import static org.mockito.BDDMockito.given; + +import com.amazonaws.services.kinesis.clientlibrary.lib.worker.InitialPositionInStream; +import com.amazonaws.services.kinesis.model.Shard; import org.junit.Test; import org.junit.runner.RunWith; import org.mockito.Mock; import org.mockito.runners.MockitoJUnitRunner; -import static java.util.Arrays.asList; /*** 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 61a858f8dea53..304220be46976 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 @@ -17,20 +17,19 @@ */ package org.apache.beam.sdk.io.kinesis; +import static com.google.common.collect.Lists.newArrayList; + +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 static com.google.common.collect.Lists.newArrayList; - -import com.google.common.collect.Iterables; - -import com.amazonaws.services.kinesis.clientlibrary.lib.worker.InitialPositionInStream; import org.joda.time.DateTime; import org.junit.Test; -import java.util.List; /** * Created by p.pastuszka on 22.07.2016. diff --git a/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/KinesisReaderCheckpointTest.java b/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/KinesisReaderCheckpointTest.java index 205f050bb8ac2..8c8da641804f1 100644 --- a/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/KinesisReaderCheckpointTest.java +++ b/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/KinesisReaderCheckpointTest.java @@ -18,17 +18,17 @@ package org.apache.beam.sdk.io.kinesis; -import com.google.common.collect.Iterables; - +import static java.util.Arrays.asList; import static org.assertj.core.api.Assertions.assertThat; + +import com.google.common.collect.Iterables; +import java.util.Iterator; +import java.util.List; import org.junit.Before; import org.junit.Test; import org.junit.runner.RunWith; import org.mockito.Mock; import org.mockito.runners.MockitoJUnitRunner; -import static java.util.Arrays.asList; -import java.util.Iterator; -import java.util.List; /*** * 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 fbc7c66aa3b93..73a24555fa9c9 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 @@ -17,6 +17,20 @@ */ package org.apache.beam.sdk.io.kinesis; +import static com.google.common.base.Preconditions.checkNotNull; +import static com.google.common.collect.Lists.newArrayList; +import static java.util.concurrent.Executors.newSingleThreadExecutor; +import static org.assertj.core.api.Assertions.assertThat; + +import com.amazonaws.regions.Regions; +import java.io.IOException; +import java.nio.charset.StandardCharsets; +import java.util.List; +import java.util.concurrent.Callable; +import java.util.concurrent.ExecutionException; +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; @@ -25,25 +39,11 @@ import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.ParDo; import org.apache.beam.sdk.values.PCollection; -import static com.google.common.base.Preconditions.checkNotNull; -import static com.google.common.collect.Lists.newArrayList; - -import com.amazonaws.regions.Regions; -import static org.assertj.core.api.Assertions.assertThat; import org.apache.commons.lang.RandomStringUtils; import org.joda.time.Duration; import org.joda.time.Instant; import org.junit.Ignore; import org.junit.Test; -import static java.util.concurrent.Executors.newSingleThreadExecutor; -import java.io.IOException; -import java.nio.charset.StandardCharsets; -import java.util.List; -import java.util.concurrent.Callable; -import java.util.concurrent.ExecutionException; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.Future; -import java.util.concurrent.TimeUnit; /** * Integration test, that reads from the real Kinesis. diff --git a/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/KinesisReaderTest.java b/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/KinesisReaderTest.java index 793fb579c26b5..29a24821bdf3b 100644 --- a/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/KinesisReaderTest.java +++ b/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/KinesisReaderTest.java @@ -17,16 +17,17 @@ */ package org.apache.beam.sdk.io.kinesis; +import static java.util.Arrays.asList; import static org.assertj.core.api.Assertions.assertThat; import static org.mockito.Mockito.when; + +import java.io.IOException; +import java.util.NoSuchElementException; import org.junit.Before; import org.junit.Test; import org.junit.runner.RunWith; import org.mockito.Mock; import org.mockito.runners.MockitoJUnitRunner; -import static java.util.Arrays.asList; -import java.io.IOException; -import java.util.NoSuchElementException; /** * Created by ppastuszka on 12.12.15. diff --git a/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/KinesisRecordCoderTest.java b/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/KinesisRecordCoderTest.java index b09b7eb9b378d..d301f256a4839 100644 --- a/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/KinesisRecordCoderTest.java +++ b/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/KinesisRecordCoderTest.java @@ -17,11 +17,10 @@ */ package org.apache.beam.sdk.io.kinesis; +import java.nio.ByteBuffer; import org.apache.beam.sdk.testing.CoderProperties; - import org.joda.time.Instant; import org.junit.Test; -import java.nio.ByteBuffer; /** * Created by p.pastuszka on 20.07.2016. diff --git a/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/KinesisUploader.java b/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/KinesisUploader.java index 0dcede9856713..c98242b4a0a26 100644 --- a/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/KinesisUploader.java +++ b/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/KinesisUploader.java @@ -18,8 +18,6 @@ package org.apache.beam.sdk.io.kinesis; import static com.google.common.collect.Lists.newArrayList; -import com.google.common.base.Charsets; -import com.google.common.collect.Lists; import com.amazonaws.auth.BasicAWSCredentials; import com.amazonaws.internal.StaticCredentialsProvider; @@ -30,6 +28,8 @@ import com.amazonaws.services.kinesis.model.PutRecordsRequestEntry; import com.amazonaws.services.kinesis.model.PutRecordsResult; import com.amazonaws.services.kinesis.model.PutRecordsResultEntry; +import com.google.common.base.Charsets; +import com.google.common.collect.Lists; import java.nio.ByteBuffer; import java.util.List; diff --git a/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/RecordFilterTest.java b/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/RecordFilterTest.java index 360106d00c4b3..f979c0108cdc2 100644 --- a/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/RecordFilterTest.java +++ b/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/RecordFilterTest.java @@ -17,16 +17,16 @@ */ package org.apache.beam.sdk.io.kinesis; -import com.google.common.collect.Lists; - import static org.mockito.BDDMockito.given; + +import com.google.common.collect.Lists; +import java.util.Collections; +import java.util.List; import org.assertj.core.api.Assertions; import org.junit.Test; import org.junit.runner.RunWith; import org.mockito.Mock; import org.mockito.runners.MockitoJUnitRunner; -import java.util.Collections; -import java.util.List; /*** diff --git a/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/RoundRobinTest.java b/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/RoundRobinTest.java index a508ddf5ce0fb..aedc89eac8b8e 100644 --- a/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/RoundRobinTest.java +++ b/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/RoundRobinTest.java @@ -18,11 +18,11 @@ package org.apache.beam.sdk.io.kinesis; import static com.google.common.collect.Lists.newArrayList; - import static org.assertj.core.api.Assertions.assertThat; -import org.junit.Test; + import java.util.Collections; import java.util.List; +import org.junit.Test; /** * Created by ppastuszka on 12.12.15. diff --git a/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/ShardCheckpointTest.java b/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/ShardCheckpointTest.java index 2227cef638285..39ab36f9255a5 100644 --- a/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/ShardCheckpointTest.java +++ b/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/ShardCheckpointTest.java @@ -22,8 +22,6 @@ import static com.amazonaws.services.kinesis.model.ShardIteratorType.AFTER_SEQUENCE_NUMBER; import static com.amazonaws.services.kinesis.model.ShardIteratorType.AT_SEQUENCE_NUMBER; import static com.amazonaws.services.kinesis.model.ShardIteratorType.AT_TIMESTAMP; -import com.amazonaws.services.kinesis.clientlibrary.types.ExtendedSequenceNumber; -import com.amazonaws.services.kinesis.model.ShardIteratorType; import static org.assertj.core.api.Assertions.assertThat; import static org.mockito.BDDMockito.given; import static org.mockito.Matchers.anyString; @@ -31,6 +29,10 @@ import static org.mockito.Matchers.isNull; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; + +import com.amazonaws.services.kinesis.clientlibrary.types.ExtendedSequenceNumber; +import com.amazonaws.services.kinesis.model.ShardIteratorType; +import java.io.IOException; import org.joda.time.DateTime; import org.joda.time.Instant; import org.junit.Before; @@ -38,7 +40,6 @@ import org.junit.runner.RunWith; import org.mockito.Mock; import org.mockito.runners.MockitoJUnitRunner; -import java.io.IOException; /** * diff --git a/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/ShardRecordsIteratorTest.java b/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/ShardRecordsIteratorTest.java index e2a3ccc0057e3..585b8843c489d 100644 --- a/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/ShardRecordsIteratorTest.java +++ b/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/ShardRecordsIteratorTest.java @@ -17,11 +17,16 @@ */ package org.apache.beam.sdk.io.kinesis; -import com.amazonaws.services.kinesis.model.ExpiredIteratorException; +import static java.util.Arrays.asList; +import static java.util.Collections.singletonList; import static org.assertj.core.api.Assertions.assertThat; import static org.mockito.Matchers.any; import static org.mockito.Matchers.anyListOf; import static org.mockito.Mockito.when; + +import com.amazonaws.services.kinesis.model.ExpiredIteratorException; +import java.io.IOException; +import java.util.Collections; import org.junit.Before; import org.junit.Test; import org.junit.runner.RunWith; @@ -29,10 +34,6 @@ import org.mockito.invocation.InvocationOnMock; import org.mockito.runners.MockitoJUnitRunner; import org.mockito.stubbing.Answer; -import static java.util.Arrays.asList; -import static java.util.Collections.singletonList; -import java.io.IOException; -import java.util.Collections; /** * Created by ppastuszka on 12.12.15. diff --git a/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/SimplifiedKinesisClientTest.java b/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/SimplifiedKinesisClientTest.java index 44d29d6cf8fd7..96434fd4c838a 100644 --- a/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/SimplifiedKinesisClientTest.java +++ b/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/SimplifiedKinesisClientTest.java @@ -17,6 +17,11 @@ */ package org.apache.beam.sdk.io.kinesis; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.failBecauseExceptionWasNotThrown; +import static org.mockito.BDDMockito.given; +import static org.mockito.Mockito.reset; + import com.amazonaws.AmazonServiceException; import com.amazonaws.AmazonServiceException.ErrorType; import com.amazonaws.services.kinesis.AmazonKinesis; @@ -29,17 +34,13 @@ import com.amazonaws.services.kinesis.model.Shard; import com.amazonaws.services.kinesis.model.ShardIteratorType; import com.amazonaws.services.kinesis.model.StreamDescription; -import static org.assertj.core.api.Assertions.assertThat; -import static org.assertj.core.api.Assertions.failBecauseExceptionWasNotThrown; -import static org.mockito.BDDMockito.given; -import static org.mockito.Mockito.reset; +import java.util.List; import org.joda.time.Instant; import org.junit.Test; import org.junit.runner.RunWith; import org.mockito.InjectMocks; import org.mockito.Mock; import org.mockito.runners.MockitoJUnitRunner; -import java.util.List; /*** */ From 973081e63378bcc2cfcc8f7ecee519ebe8addb8c Mon Sep 17 00:00:00 2001 From: Dan Halperin Date: Fri, 26 Aug 2016 15:20:44 -0700 Subject: [PATCH 082/346] Fix javadoc in Kinesis --- .../apache/beam/sdk/io/kinesis/TransientKinesisException.java | 2 +- .../java/org/apache/beam/sdk/io/kinesis/AmazonKinesisMock.java | 2 +- .../java/org/apache/beam/sdk/io/kinesis/CustomOptionalTest.java | 2 +- .../org/apache/beam/sdk/io/kinesis/KinesisMockReadTest.java | 2 +- .../java/org/apache/beam/sdk/io/kinesis/KinesisReaderTest.java | 2 +- .../org/apache/beam/sdk/io/kinesis/KinesisRecordCoderTest.java | 2 +- .../java/org/apache/beam/sdk/io/kinesis/RoundRobinTest.java | 2 +- .../apache/beam/sdk/io/kinesis/ShardRecordsIteratorTest.java | 2 +- 8 files changed, 8 insertions(+), 8 deletions(-) diff --git a/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/TransientKinesisException.java b/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/TransientKinesisException.java index a1a974bf3464b..57ad8a89103e9 100644 --- a/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/TransientKinesisException.java +++ b/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/TransientKinesisException.java @@ -20,7 +20,7 @@ import com.amazonaws.AmazonServiceException; /** - * Created by p.pastuszka on 21.06.2016. + * A transient exception thrown by Kinesis. */ class TransientKinesisException extends Exception { public TransientKinesisException(String s, AmazonServiceException e) { diff --git a/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/AmazonKinesisMock.java b/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/AmazonKinesisMock.java index b007fa411db41..046c9d9126d2f 100644 --- a/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/AmazonKinesisMock.java +++ b/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/AmazonKinesisMock.java @@ -74,7 +74,7 @@ import org.joda.time.Instant; /** - * Created by p.pastuszka on 21.07.2016. + * Mock implemenation of {@link AmazonKinesis} for testing. */ class AmazonKinesisMock implements AmazonKinesis { diff --git a/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/CustomOptionalTest.java b/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/CustomOptionalTest.java index cb0d0e248f883..20e8372506141 100644 --- a/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/CustomOptionalTest.java +++ b/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/CustomOptionalTest.java @@ -21,7 +21,7 @@ import org.junit.Test; /** - * Created by ppastuszka on 12.12.15. + * Tests {@link CustomOptional}. */ public class CustomOptionalTest { @Test(expected = NoSuchElementException.class) 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 304220be46976..f0ab46c3ae701 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 @@ -32,7 +32,7 @@ import org.junit.Test; /** - * Created by p.pastuszka on 22.07.2016. + * Tests {@link AmazonKinesisMock}. */ public class KinesisMockReadTest { @Test diff --git a/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/KinesisReaderTest.java b/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/KinesisReaderTest.java index 29a24821bdf3b..3111029d74e51 100644 --- a/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/KinesisReaderTest.java +++ b/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/KinesisReaderTest.java @@ -30,7 +30,7 @@ import org.mockito.runners.MockitoJUnitRunner; /** - * Created by ppastuszka on 12.12.15. + * Tests {@link KinesisReader}. */ @RunWith(MockitoJUnitRunner.class) public class KinesisReaderTest { diff --git a/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/KinesisRecordCoderTest.java b/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/KinesisRecordCoderTest.java index d301f256a4839..8771c86c82a5a 100644 --- a/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/KinesisRecordCoderTest.java +++ b/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/KinesisRecordCoderTest.java @@ -23,7 +23,7 @@ import org.junit.Test; /** - * Created by p.pastuszka on 20.07.2016. + * Tests {@link KinesisRecordCoder}. */ public class KinesisRecordCoderTest { @Test diff --git a/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/RoundRobinTest.java b/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/RoundRobinTest.java index aedc89eac8b8e..f032eeab377d4 100644 --- a/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/RoundRobinTest.java +++ b/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/RoundRobinTest.java @@ -25,7 +25,7 @@ import org.junit.Test; /** - * Created by ppastuszka on 12.12.15. + * Tests {@link RoundRobin}. */ public class RoundRobinTest { @Test(expected = IllegalArgumentException.class) diff --git a/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/ShardRecordsIteratorTest.java b/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/ShardRecordsIteratorTest.java index 585b8843c489d..49e806dc12e85 100644 --- a/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/ShardRecordsIteratorTest.java +++ b/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/ShardRecordsIteratorTest.java @@ -36,7 +36,7 @@ import org.mockito.stubbing.Answer; /** - * Created by ppastuszka on 12.12.15. + * Tests {@link ShardRecordsIterator}. */ @RunWith(MockitoJUnitRunner.class) public class ShardRecordsIteratorTest { From 1c1115eeeb59707a401513e903dfbae15a85a029 Mon Sep 17 00:00:00 2001 From: Mark Liu Date: Fri, 26 Aug 2016 14:23:54 -0700 Subject: [PATCH 083/346] [BEAM-592] Fix SparkRunner Dependency Problem in WordCount --- examples/java/pom.xml | 6 ++++++ examples/java8/pom.xml | 6 ++++++ 2 files changed, 12 insertions(+) diff --git a/examples/java/pom.xml b/examples/java/pom.xml index 096bc4e32fefd..47f036613ac65 100644 --- a/examples/java/pom.xml +++ b/examples/java/pom.xml @@ -97,6 +97,12 @@ ${spark.version} runtime true + + + org.slf4j + jul-to-slf4j + + diff --git a/examples/java8/pom.xml b/examples/java8/pom.xml index f3f2c3c83f17c..44cf918ba2e6a 100644 --- a/examples/java8/pom.xml +++ b/examples/java8/pom.xml @@ -98,6 +98,12 @@ ${spark.version} runtime true + + + org.slf4j + jul-to-slf4j + + From 8454d5ca18c265ab1c5f54f2222c13d9ff17e4ef Mon Sep 17 00:00:00 2001 From: Daniel Halperin Date: Sat, 27 Aug 2016 09:42:08 -0700 Subject: [PATCH 084/346] DataflowRunner: get PBegin from PInput Fixes an invalid cast that breaks some PCollectionList-related tests. --- .../java/org/apache/beam/runners/dataflow/DataflowRunner.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 0ce4b58aa6acd..e5b66147c3820 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 @@ -400,7 +400,7 @@ public OutputT apply( return windowed; } else if (Flatten.FlattenPCollectionList.class.equals(transform.getClass()) && ((PCollectionList) input).size() == 0) { - return (OutputT) Pipeline.applyTransform((PBegin) input, Create.of()); + return (OutputT) Pipeline.applyTransform(input.getPipeline().begin(), Create.of()); } else if (overrides.containsKey(transform.getClass())) { // It is the responsibility of whoever constructs overrides to ensure this is type safe. @SuppressWarnings("unchecked") From 07dd9780dbf0887ae8e7e5c73e391cedded9df3f Mon Sep 17 00:00:00 2001 From: Abbass MAROUNI Date: Mon, 29 Aug 2016 13:28:46 +0200 Subject: [PATCH 085/346] [BEAM-313] Provide a context for SparkRunner --- .../runners/spark/SparkPipelineOptions.java | 13 ++ .../beam/runners/spark/SparkRunner.java | 16 +- .../spark/ProvidedSparkContextTest.java | 138 ++++++++++++++++++ 3 files changed, 164 insertions(+), 3 deletions(-) create mode 100644 runners/spark/src/test/java/org/apache/beam/runners/spark/ProvidedSparkContextTest.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 be4f7f06a9bc0..db6b75cb4a034 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 @@ -18,11 +18,14 @@ package org.apache.beam.runners.spark; +import com.fasterxml.jackson.annotation.JsonIgnore; + import org.apache.beam.sdk.options.ApplicationNameOptions; 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.StreamingOptions; +import org.apache.spark.api.java.JavaSparkContext; /** * Spark runner pipeline options. @@ -49,4 +52,14 @@ public interface SparkPipelineOptions extends PipelineOptions, StreamingOptions, @Default.Boolean(true) Boolean getEnableSparkSinks(); void setEnableSparkSinks(Boolean enableSparkSinks); + + @Description("If the spark runner will be initialized with a provided Spark Context") + @Default.Boolean(false) + boolean getUsesProvidedSparkContext(); + void setUsesProvidedSparkContext(boolean value); + + @Description("Provided Java Spark Context") + @JsonIgnore + JavaSparkContext getProvidedSparkContext(); + void setProvidedSparkContext(JavaSparkContext jsc); } 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 fa85a2e25e262..9f1a83996fb76 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 @@ -143,9 +143,19 @@ private SparkRunner(SparkPipelineOptions options) { public EvaluationResult run(Pipeline pipeline) { try { LOG.info("Executing pipeline using the SparkRunner."); - JavaSparkContext jsc = SparkContextFactory.getSparkContext(mOptions.getSparkMaster(), - mOptions.getAppName()); - + JavaSparkContext jsc; + if (mOptions.getUsesProvidedSparkContext()) { + LOG.info("Using a provided Spark Context"); + jsc = mOptions.getProvidedSparkContext(); + if (jsc == null || jsc.sc().isStopped()){ + LOG.error("The provided Spark context " + + jsc + " was not created or was stopped"); + throw new RuntimeException("The provided Spark context was not created or was stopped"); + } + } else { + LOG.info("Creating a new Spark Context"); + jsc = SparkContextFactory.getSparkContext(mOptions.getSparkMaster(), mOptions.getAppName()); + } if (mOptions.isStreaming()) { SparkPipelineTranslator translator = new StreamingTransformTranslator.Translator(new TransformTranslator.Translator()); 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 new file mode 100644 index 0000000000000..cbc5976b95424 --- /dev/null +++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/ProvidedSparkContextTest.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.spark; + +import static org.junit.Assert.fail; + +import com.google.common.collect.ImmutableSet; +import java.util.Arrays; +import java.util.List; +import java.util.Set; +import org.apache.beam.runners.spark.examples.WordCount; +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; +import org.apache.beam.sdk.transforms.Create; +import org.apache.beam.sdk.transforms.MapElements; +import org.apache.beam.sdk.values.PCollection; +import org.apache.spark.api.java.JavaSparkContext; +import org.junit.Test; + +/** + * Provided Spark Context tests. + */ +public class ProvidedSparkContextTest { + private static final String[] WORDS_ARRAY = { + "hi there", "hi", "hi sue bob", + "hi sue", "", "bob hi"}; + private static final List WORDS = Arrays.asList(WORDS_ARRAY); + private static final Set EXPECTED_COUNT_SET = + ImmutableSet.of("hi: 5", "there: 1", "sue: 2", "bob: 2"); + private static final String PROVIDED_CONTEXT_EXCEPTION = + "The provided Spark context was not created or was stopped"; + + /** + * Provide a context and call pipeline run. + * @throws Exception + */ + @Test + public void testWithProvidedContext() throws Exception { + JavaSparkContext jsc = new JavaSparkContext("local[*]", "Existing_Context"); + + SparkPipelineOptions options = PipelineOptionsFactory.as(SparkPipelineOptions.class); + options.setRunner(SparkRunner.class); + options.setUsesProvidedSparkContext(true); + options.setProvidedSparkContext(jsc); + + Pipeline p = Pipeline.create(options); + PCollection inputWords = p.apply(Create.of(WORDS).withCoder(StringUtf8Coder + .of())); + PCollection output = inputWords.apply(new WordCount.CountWords()) + .apply(MapElements.via(new WordCount.FormatAsTextFn())); + + PAssert.that(output).containsInAnyOrder(EXPECTED_COUNT_SET); + + // Run test from pipeline + p.run(); + + jsc.stop(); + } + + /** + * Provide a context and call pipeline run. + * @throws Exception + */ + @Test + public void testWithNullContext() throws Exception { + JavaSparkContext jsc = null; + + SparkPipelineOptions options = PipelineOptionsFactory.as(SparkPipelineOptions.class); + options.setRunner(SparkRunner.class); + options.setUsesProvidedSparkContext(true); + options.setProvidedSparkContext(jsc); + + Pipeline p = Pipeline.create(options); + PCollection inputWords = p.apply(Create.of(WORDS).withCoder(StringUtf8Coder + .of())); + PCollection output = inputWords.apply(new WordCount.CountWords()) + .apply(MapElements.via(new WordCount.FormatAsTextFn())); + + PAssert.that(output).containsInAnyOrder(EXPECTED_COUNT_SET); + + try { + p.run(); + fail("Should throw an exception when The provided Spark context is null"); + } catch (RuntimeException e){ + assert(e.getMessage().contains(PROVIDED_CONTEXT_EXCEPTION)); + } + } + + /** + * A SparkRunner with a stopped provided Spark context cannot run pipelines. + * @throws Exception + */ + @Test + public void testWithStoppedProvidedContext() throws Exception { + JavaSparkContext jsc = new JavaSparkContext("local[*]", "Existing_Context"); + // Stop the provided Spark context directly + jsc.stop(); + + SparkPipelineOptions options = PipelineOptionsFactory.as(SparkPipelineOptions.class); + options.setRunner(SparkRunner.class); + options.setUsesProvidedSparkContext(true); + options.setProvidedSparkContext(jsc); + + Pipeline p = Pipeline.create(options); + PCollection inputWords = p.apply(Create.of(WORDS).withCoder(StringUtf8Coder + .of())); + PCollection output = inputWords.apply(new WordCount.CountWords()) + .apply(MapElements.via(new WordCount.FormatAsTextFn())); + + PAssert.that(output).containsInAnyOrder(EXPECTED_COUNT_SET); + + try { + p.run(); + fail("Should throw an exception when The provided Spark context is stopped"); + } catch (RuntimeException e){ + assert(e.getMessage().contains(PROVIDED_CONTEXT_EXCEPTION)); + } + } + +} From 435054bef169046101fd6cf6b836209119b12260 Mon Sep 17 00:00:00 2001 From: Thomas Groh Date: Fri, 26 Aug 2016 15:28:21 -0700 Subject: [PATCH 086/346] Update Dataflow Container 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 e5b66147c3820..d067b61064e6b 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 @@ -206,9 +206,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-20160823"; + "dataflow.gcr.io/v1beta3/beam-java-batch:beam-master-20160826"; public static final String STREAMING_WORKER_HARNESS_CONTAINER_IMAGE = - "dataflow.gcr.io/v1beta3/beam-java-streaming:beam-master-20160823"; + "dataflow.gcr.io/v1beta3/beam-java-streaming:beam-master-20160826"; // The limit of CreateJob request size. private static final int CREATE_JOB_REQUEST_LIMIT_BYTES = 10 * 1024 * 1024; From cf9ce2ff27dfe13715119c5a9c04900f0e5e6e93 Mon Sep 17 00:00:00 2001 From: Mark Liu Date: Fri, 26 Aug 2016 16:13:17 -0700 Subject: [PATCH 087/346] [BEAM-572] Remove Spark Reference in WordCount --- .../org/apache/beam/examples/WordCount.java | 24 +------------------ .../org/apache/beam/examples/WordCountIT.java | 9 +++++++ 2 files changed, 10 insertions(+), 23 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 0275651bf288b..793ee4baceae5 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 @@ -18,7 +18,6 @@ package org.apache.beam.examples; import com.google.common.base.Strings; -import com.google.common.io.Resources; import java.io.IOException; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.io.TextIO; @@ -165,7 +164,7 @@ public PCollection> apply(PCollection lines) { */ public static interface WordCountOptions extends PipelineOptions { @Description("Path of the file to read from") - @Default.InstanceFactory(InputFactory.class) + @Default.String("gs://apache-beam-samples/shakespeare/kinglear.txt") String getInputFile(); void setInputFile(String value); @@ -194,27 +193,6 @@ public String create(PipelineOptions options) { } } } - - /** - * Return default input file path according to runner type. - * - *

                - *
              • SparkRunner: - * .../src/test/resources/LICENSE
              • - *
              • other runners: - * gs://apache-beam-samples/apache/LICENSE
              • - *
              - */ - public static class InputFactory implements DefaultValueFactory { - @Override - public String create(PipelineOptions options) { - if (options.getRunner().getName().contains("SparkRunner")) { - return Resources.getResource("LICENSE").getPath(); - } else { - return "gs://apache-beam-samples/apache/LICENSE"; - } - } - } } public static void main(String[] args) { 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 ca0c9d6bda09d..b0e0fe00628a4 100644 --- a/examples/java/src/test/java/org/apache/beam/examples/WordCountIT.java +++ b/examples/java/src/test/java/org/apache/beam/examples/WordCountIT.java @@ -18,6 +18,7 @@ package org.apache.beam.examples; +import com.google.common.io.Resources; import java.util.Date; import org.apache.beam.examples.WordCount.WordCountOptions; import org.apache.beam.sdk.options.Default; @@ -61,6 +62,14 @@ public void testE2EWordCount() throws Exception { options.setOnSuccessMatcher( new FileChecksumMatcher(options.getOutputChecksum(), options.getOutput() + "*")); + String e2eTestInputPath = "gs://apache-beam-samples/apache/LICENSE"; + // Spark runner currently doesn't support GCS I/O, change default input to: + // .../src/test/resources/LICENSE + if (options.getRunner().getName().contains("SparkRunner")) { + e2eTestInputPath = Resources.getResource("LICENSE").getPath(); + } + options.setInputFile(e2eTestInputPath); + WordCount.main(TestPipeline.convertToArgs(options)); } } From a58afd3db8cdce4f5135b78f3785c8b3ff7fb0ab Mon Sep 17 00:00:00 2001 From: Aviem Zur Date: Sun, 28 Aug 2016 11:41:58 +0300 Subject: [PATCH 088/346] Returned KafkaIO getWatermark log line in debug mode --- .../src/main/java/org/apache/beam/sdk/io/kafka/KafkaIO.java | 1 + 1 file changed, 1 insertion(+) 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 f6394220ceba7..f16a5aba5815e 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 @@ -1072,6 +1072,7 @@ private void updateLatestOffsets() { @Override public Instant getWatermark() { if (curRecord == null) { + LOG.debug("{}: getWatermark() : no records have been read yet.", name); return initialWatermark; } From 74d0195d15660727de80314b1658b498ceda00e1 Mon Sep 17 00:00:00 2001 From: Raghu Angadi Date: Mon, 29 Aug 2016 13:09:58 -0700 Subject: [PATCH 089/346] take advantage of setup/teardown for KafkaWriter --- .../org/apache/beam/sdk/io/kafka/KafkaIO.java | 25 +++++++++---------- 1 file changed, 12 insertions(+), 13 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 f16a5aba5815e..6769b3162a011 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 @@ -1325,16 +1325,12 @@ public KV apply(V element) { private static class KafkaWriter extends DoFn, Void> { - @StartBundle - public void startBundle(Context c) throws Exception { - // Producer initialization is fairly costly. Move this to future initialization api to avoid - // creating a producer for each bundle. - if (producer == null) { - if (producerFactoryFnOpt.isPresent()) { - producer = producerFactoryFnOpt.get().apply(producerConfig); - } else { - producer = new KafkaProducer(producerConfig); - } + @Setup + public void setup() { + if (producerFactoryFnOpt.isPresent()) { + producer = producerFactoryFnOpt.get().apply(producerConfig); + } else { + producer = new KafkaProducer(producerConfig); } } @@ -1349,13 +1345,16 @@ public void processElement(ProcessContext ctx) throws Exception { } @FinishBundle - public void finishBundle(Context c) throws Exception { + public void finishBundle(Context c) throws IOException { producer.flush(); - producer.close(); - producer = null; checkForFailures(); } + @Teardown + public void teardown() { + producer.close(); + } + /////////////////////////////////////////////////////////////////////////////////// private final String topic; From 00b4e95148eb98d7fea5877274f2fcf2252ac432 Mon Sep 17 00:00:00 2001 From: Thomas Groh Date: Fri, 5 Aug 2016 14:20:56 -0700 Subject: [PATCH 090/346] Add LeaderBoardTest This test exercises the PTransforms that make up the LeaderBoard example. This includes speculative and late trigger firings to produce team and individual scores on a global and fixed window basis. Refactor LeaderBoard to expose the team and user score calculations as composite PTransforms to enable this testing. --- .../examples/complete/game/LeaderBoard.java | 113 ++++-- .../complete/game/LeaderBoardTest.java | 362 ++++++++++++++++++ 2 files changed, 440 insertions(+), 35 deletions(-) create mode 100644 examples/java8/src/test/java/org/apache/beam/examples/complete/game/LeaderBoardTest.java 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 8dd4e39bee01f..13bbf4482c6cb 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 @@ -17,6 +17,7 @@ */ package org.apache.beam.examples.complete.game; +import com.google.common.annotations.VisibleForTesting; import java.util.HashMap; import java.util.Map; import java.util.TimeZone; @@ -32,6 +33,7 @@ import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.options.StreamingOptions; import org.apache.beam.sdk.options.Validation; +import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.ParDo; import org.apache.beam.sdk.transforms.windowing.AfterProcessingTime; import org.apache.beam.sdk.transforms.windowing.AfterWatermark; @@ -68,7 +70,7 @@ * here we're using an unbounded data source, which lets us provide speculative results, and allows * handling of late data, at much lower latency. We can use the early/speculative results to keep a * 'leaderboard' updated in near-realtime. Our handling of late data lets us generate correct - * results, e.g. for 'team prizes'. We're now outputing window results as they're + * results, e.g. for 'team prizes'. We're now outputting window results as they're * calculated, giving us much lower latency than with the previous batch examples. * *

              Run {@link injector.Injector} to generate pubsub data for this pipeline. The Injector @@ -186,50 +188,91 @@ public static void main(String[] args) throws Exception { .apply(PubsubIO.Read.timestampLabel(TIMESTAMP_ATTRIBUTE).topic(options.getTopic())) .apply("ParseGameEvent", ParDo.of(new ParseEventFn())); - // [START DocInclude_WindowAndTrigger] - // Extract team/score pairs from the event stream, using hour-long windows by default. - gameEvents - .apply("LeaderboardTeamFixedWindows", Window.into( - FixedWindows.of(Duration.standardMinutes(options.getTeamWindowDuration()))) - // We will get early (speculative) results as well as cumulative - // processing of late data. - .triggering( - AfterWatermark.pastEndOfWindow() - .withEarlyFirings(AfterProcessingTime.pastFirstElementInPane() - .plusDelayOf(FIVE_MINUTES)) - .withLateFirings(AfterProcessingTime.pastFirstElementInPane() - .plusDelayOf(TEN_MINUTES))) - .withAllowedLateness(Duration.standardMinutes(options.getAllowedLateness())) - .accumulatingFiredPanes()) - // Extract and sum teamname/score pairs from the event data. - .apply("ExtractTeamScore", new ExtractAndSumScore("team")) + gameEvents.apply("CalculateTeamScores", + new CalculateTeamScores( + Duration.standardMinutes(options.getTeamWindowDuration()), + Duration.standardMinutes(options.getAllowedLateness()))) // Write the results to BigQuery. .apply("WriteTeamScoreSums", new WriteWindowedToBigQuery>( options.getTableName() + "_team", configureWindowedTableWrite())); - // [END DocInclude_WindowAndTrigger] - - // [START DocInclude_ProcTimeTrigger] - // Extract user/score pairs from the event stream using processing time, via global windowing. - // Get periodic updates on all users' running scores. gameEvents - .apply("LeaderboardUserGlobalWindow", Window.into(new GlobalWindows()) - // Get periodic results every ten minutes. - .triggering(Repeatedly.forever(AfterProcessingTime.pastFirstElementInPane() - .plusDelayOf(TEN_MINUTES))) - .accumulatingFiredPanes() - .withAllowedLateness(Duration.standardMinutes(options.getAllowedLateness()))) - // Extract and sum username/score pairs from the event data. - .apply("ExtractUserScore", new ExtractAndSumScore("user")) + .apply( + "CalculateUserScores", + new CalculateUserScores(Duration.standardMinutes(options.getAllowedLateness()))) // Write the results to BigQuery. - .apply("WriteUserScoreSums", - new WriteToBigQuery>( - options.getTableName() + "_user", configureGlobalWindowBigQueryWrite())); - // [END DocInclude_ProcTimeTrigger] + .apply( + "WriteUserScoreSums", + new WriteToBigQuery>( + options.getTableName() + "_user", configureGlobalWindowBigQueryWrite())); // Run the pipeline and wait for the pipeline to finish; capture cancellation requests from the // command line. PipelineResult result = pipeline.run(); exampleUtils.waitToFinish(result); } + + /** + * Calculates scores for each team within the configured window duration. + */ + // [START DocInclude_WindowAndTrigger] + // Extract team/score pairs from the event stream, using hour-long windows by default. + @VisibleForTesting + static class CalculateTeamScores + extends PTransform, PCollection>> { + private final Duration teamWindowDuration; + private final Duration allowedLateness; + + CalculateTeamScores(Duration teamWindowDuration, Duration allowedLateness) { + this.teamWindowDuration = teamWindowDuration; + this.allowedLateness = allowedLateness; + } + + @Override + public PCollection> apply(PCollection infos) { + return infos.apply("LeaderboardTeamFixedWindows", + Window.into(FixedWindows.of(teamWindowDuration)) + // We will get early (speculative) results as well as cumulative + // processing of late data. + .triggering(AfterWatermark.pastEndOfWindow() + .withEarlyFirings(AfterProcessingTime.pastFirstElementInPane() + .plusDelayOf(FIVE_MINUTES)) + .withLateFirings(AfterProcessingTime.pastFirstElementInPane() + .plusDelayOf(TEN_MINUTES))) + .withAllowedLateness(allowedLateness) + .accumulatingFiredPanes()) + // Extract and sum teamname/score pairs from the event data. + .apply("ExtractTeamScore", new ExtractAndSumScore("team")); + } + } + // [END DocInclude_WindowAndTrigger] + + // [START DocInclude_ProcTimeTrigger] + /** + * Extract user/score pairs from the event stream using processing time, via global windowing. + * Get periodic updates on all users' running scores. + */ + @VisibleForTesting + static class CalculateUserScores + extends PTransform, PCollection>> { + private final Duration allowedLateness; + + CalculateUserScores(Duration allowedLateness) { + this.allowedLateness = allowedLateness; + } + + @Override + public PCollection> apply(PCollection input) { + return input.apply("LeaderboardUserGlobalWindow", + Window.into(new GlobalWindows()) + // Get periodic results every ten minutes. + .triggering(Repeatedly.forever(AfterProcessingTime.pastFirstElementInPane() + .plusDelayOf(TEN_MINUTES))) + .accumulatingFiredPanes() + .withAllowedLateness(allowedLateness)) + // Extract and sum username/score pairs from the event data. + .apply("ExtractUserScore", new ExtractAndSumScore("user")); + } + } + // [END DocInclude_ProcTimeTrigger] } 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 new file mode 100644 index 0000000000000..40cac36f5a6ec --- /dev/null +++ b/examples/java8/src/test/java/org/apache/beam/examples/complete/game/LeaderBoardTest.java @@ -0,0 +1,362 @@ +/* + * 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.complete.game; + +import static org.apache.beam.sdk.testing.PAssert.that; +import static org.hamcrest.Matchers.hasItem; +import static org.junit.Assert.assertThat; + +import com.google.common.collect.ImmutableMap; +import java.io.Serializable; +import org.apache.beam.examples.complete.game.LeaderBoard.CalculateTeamScores; +import org.apache.beam.examples.complete.game.LeaderBoard.CalculateUserScores; +import org.apache.beam.examples.complete.game.UserScore.GameActionInfo; +import org.apache.beam.sdk.coders.AvroCoder; +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.PTransform; +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.IntervalWindow; +import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.TimestampedValue; +import org.joda.time.Duration; +import org.joda.time.Instant; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +/** + * Tests for {@link LeaderBoard}. + */ +@RunWith(JUnit4.class) +public class LeaderBoardTest implements Serializable { + private static final Duration ALLOWED_LATENESS = Duration.standardHours(1); + private static final Duration TEAM_WINDOW_DURATION = Duration.standardMinutes(20); + private Instant baseTime = new Instant(0); + + /** + * Some example users, on two separate teams. + */ + private enum TestUser { + RED_ONE("scarlet", "red"), RED_TWO("burgundy", "red"), + BLUE_ONE("navy", "blue"), BLUE_TWO("sky", "blue"); + + private final String userName; + private final String teamName; + + TestUser(String userName, String teamName) { + this.userName = userName; + this.teamName = teamName; + } + + public String getUser() { + return userName; + } + + public String getTeam() { + return teamName; + } + } + + /** + * A test of the {@link CalculateTeamScores} {@link PTransform} when all of the elements arrive + * on time (ahead of the watermark). + */ + @Test + public void testTeamScoresOnTime() { + TestPipeline p = TestPipeline.create(); + + TestStream createEvents = TestStream.create(AvroCoder.of(GameActionInfo.class)) + // Start at the epoch + .advanceWatermarkTo(baseTime) + // add some elements ahead of the watermark + .addElements(event(TestUser.BLUE_ONE, 3, Duration.standardSeconds(3)), + event(TestUser.BLUE_ONE, 2, Duration.standardMinutes(1)), + event(TestUser.RED_TWO, 3, Duration.standardSeconds(22)), + event(TestUser.BLUE_TWO, 5, Duration.standardMinutes(3))) + // The watermark advances slightly, but not past the end of the window + .advanceWatermarkTo(baseTime.plus(Duration.standardMinutes(3))) + // Add some more on time elements + .addElements(event(TestUser.RED_ONE, 1, Duration.standardMinutes(4)), + event(TestUser.BLUE_ONE, 2, Duration.standardSeconds(270))) + // The window should close and emit an ON_TIME pane + .advanceWatermarkToInfinity(); + + PCollection> teamScores = p.apply(createEvents) + .apply(new CalculateTeamScores(TEAM_WINDOW_DURATION, ALLOWED_LATENESS)); + + String blueTeam = TestUser.BLUE_ONE.getTeam(); + String redTeam = TestUser.RED_ONE.getTeam(); + that(teamScores) + .inOnTimePane(new IntervalWindow(baseTime, TEAM_WINDOW_DURATION)) + .containsInAnyOrder(KV.of(blueTeam, 12), KV.of(redTeam, 4)); + + p.run(); + } + + /** + * A test of the {@link CalculateTeamScores} {@link PTransform} when all of the elements arrive + * on time, and the processing time advances far enough for speculative panes. + */ + @Test + public void testTeamScoresSpeculative() { + TestPipeline p = TestPipeline.create(); + + TestStream createEvents = TestStream.create(AvroCoder.of(GameActionInfo.class)) + // Start at the epoch + .advanceWatermarkTo(baseTime) + .addElements(event(TestUser.BLUE_ONE, 3, Duration.standardSeconds(3)), + event(TestUser.BLUE_ONE, 2, Duration.standardMinutes(1))) + // Some time passes within the runner, which causes a speculative pane containing the blue + // team's score to be emitted + .advanceProcessingTime(Duration.standardMinutes(10)) + .addElements(event(TestUser.RED_TWO, 5, Duration.standardMinutes(3))) + // Some additional time passes and we get a speculative pane for the red team + .advanceProcessingTime(Duration.standardMinutes(12)) + .addElements(event(TestUser.BLUE_TWO, 3, Duration.standardSeconds(22))) + // More time passes and a speculative pane containing a refined value for the blue pane is + // emitted + .advanceProcessingTime(Duration.standardMinutes(10)) + // Some more events occur + .addElements(event(TestUser.RED_ONE, 4, Duration.standardMinutes(4)), + event(TestUser.BLUE_TWO, 2, Duration.standardMinutes(2))) + // The window closes and we get an ON_TIME pane that contains all of the updates + .advanceWatermarkToInfinity(); + + PCollection> teamScores = p.apply(createEvents) + .apply(new CalculateTeamScores(TEAM_WINDOW_DURATION, ALLOWED_LATENESS)); + + String blueTeam = TestUser.BLUE_ONE.getTeam(); + String redTeam = TestUser.RED_ONE.getTeam(); + IntervalWindow window = new IntervalWindow(baseTime, TEAM_WINDOW_DURATION); + // The window contains speculative panes alongside the on-time pane + PAssert.that(teamScores) + .inWindow(window) + .containsInAnyOrder(KV.of(blueTeam, 10) /* The on-time blue pane */, + KV.of(redTeam, 9) /* The on-time red pane */, + KV.of(blueTeam, 5) /* The first blue speculative pane */, + KV.of(blueTeam, 8) /* The second blue speculative pane */, + KV.of(redTeam, 5) /* The red speculative pane */); + PAssert.that(teamScores) + .inOnTimePane(window) + .containsInAnyOrder(KV.of(blueTeam, 10), KV.of(redTeam, 9)); + + p.run(); + } + + /** + * A test where elements arrive behind the watermark (late data), but before the end of the + * window. These elements are emitted on time. + */ + @Test + public void testTeamScoresUnobservablyLate() { + TestPipeline p = TestPipeline.create(); + + BoundedWindow window = new IntervalWindow(baseTime, TEAM_WINDOW_DURATION); + TestStream createEvents = TestStream.create(AvroCoder.of(GameActionInfo.class)) + .advanceWatermarkTo(baseTime) + .addElements(event(TestUser.BLUE_ONE, 3, Duration.standardSeconds(3)), + event(TestUser.BLUE_TWO, 5, Duration.standardMinutes(8)), + event(TestUser.RED_ONE, 4, Duration.standardMinutes(2)), + event(TestUser.BLUE_ONE, 3, Duration.standardMinutes(5))) + .advanceWatermarkTo(baseTime.plus(TEAM_WINDOW_DURATION).minus(Duration.standardMinutes(1))) + // These events are late, but the window hasn't closed yet, so the elements are in the + // on-time pane + .addElements(event(TestUser.RED_TWO, 2, Duration.ZERO), + event(TestUser.RED_TWO, 5, Duration.standardMinutes(1)), + event(TestUser.BLUE_TWO, 2, Duration.standardSeconds(90)), + event(TestUser.RED_TWO, 3, Duration.standardMinutes(3))) + .advanceWatermarkTo(baseTime.plus(TEAM_WINDOW_DURATION).plus(Duration.standardMinutes(1))) + .advanceWatermarkToInfinity(); + PCollection> teamScores = p.apply(createEvents) + .apply(new CalculateTeamScores(TEAM_WINDOW_DURATION, ALLOWED_LATENESS)); + + String blueTeam = TestUser.BLUE_ONE.getTeam(); + String redTeam = TestUser.RED_ONE.getTeam(); + // The On Time pane contains the late elements that arrived before the end of the window + PAssert.that(teamScores) + .inOnTimePane(window) + .containsInAnyOrder(KV.of(redTeam, 14), KV.of(blueTeam, 13)); + + p.run(); + } + + /** + * A test where elements arrive behind the watermark (late data) after the watermark passes the + * end of the window, but before the maximum allowed lateness. These elements are emitted in a + * late pane. + */ + @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)) + .advanceWatermarkTo(baseTime) + .addElements(event(TestUser.BLUE_ONE, 3, Duration.standardSeconds(3)), + event(TestUser.BLUE_TWO, 5, Duration.standardMinutes(8))) + .advanceProcessingTime(Duration.standardMinutes(10)) + .advanceWatermarkTo(baseTime.plus(Duration.standardMinutes(3))) + .addElements(event(TestUser.RED_ONE, 3, Duration.standardMinutes(1)), + event(TestUser.RED_ONE, 4, Duration.standardMinutes(2)), + event(TestUser.BLUE_ONE, 3, Duration.standardMinutes(5))) + .advanceWatermarkTo(firstWindowCloses.minus(Duration.standardMinutes(1))) + // These events are late but should still appear in a late pane + .addElements(event(TestUser.RED_TWO, 2, Duration.ZERO), + event(TestUser.RED_TWO, 5, Duration.standardMinutes(1)), + event(TestUser.RED_TWO, 3, Duration.standardMinutes(3))) + // A late refinement is emitted due to the advance in processing time, but the window has + // not yet closed because the watermark has not advanced + .advanceProcessingTime(Duration.standardMinutes(12)) + // These elements should appear in the final pane + .addElements(event(TestUser.RED_TWO, 9, Duration.standardMinutes(1)), + event(TestUser.RED_TWO, 1, Duration.standardMinutes(3))) + .advanceWatermarkToInfinity(); + + PCollection> teamScores = p.apply(createEvents) + .apply(new CalculateTeamScores(TEAM_WINDOW_DURATION, ALLOWED_LATENESS)); + + BoundedWindow window = new IntervalWindow(baseTime, TEAM_WINDOW_DURATION); + String blueTeam = TestUser.BLUE_ONE.getTeam(); + String redTeam = TestUser.RED_ONE.getTeam(); + PAssert.that(teamScores) + .inWindow(window) + .satisfies((SerializableFunction>, Void>) input -> { + // The final sums need not exist in the same pane, but must appear in the output + // PCollection + assertThat(input, hasItem(KV.of(blueTeam, 11))); + assertThat(input, hasItem(KV.of(redTeam, 27))); + return null; + }); + PAssert.thatMap(teamScores) + // The closing behavior of CalculateTeamScores precludes an inFinalPane matcher + .inOnTimePane(window) + .isEqualTo(ImmutableMap.builder().put(redTeam, 7) + .put(blueTeam, 11) + .build()); + + // No final pane is emitted for the blue team, as all of their updates have been taken into + // account in earlier panes + PAssert.that(teamScores).inFinalPane(window).containsInAnyOrder(KV.of(redTeam, 27)); + + p.run(); + } + + /** + * A test where elements arrive beyond the maximum allowed lateness. These elements are dropped + * within {@link CalculateTeamScores} and do not impact the final result. + */ + @Test + public void testTeamScoresDroppablyLate() { + TestPipeline p = TestPipeline.create(); + + BoundedWindow window = new IntervalWindow(baseTime, TEAM_WINDOW_DURATION); + TestStream infos = TestStream.create(AvroCoder.of(GameActionInfo.class)) + .addElements(event(TestUser.BLUE_ONE, 12, Duration.ZERO), + event(TestUser.RED_ONE, 3, Duration.ZERO)) + .advanceWatermarkTo(window.maxTimestamp()) + .addElements(event(TestUser.RED_ONE, 4, Duration.standardMinutes(2)), + event(TestUser.BLUE_TWO, 3, Duration.ZERO), + event(TestUser.BLUE_ONE, 3, Duration.standardMinutes(3))) + // Move the watermark past the end of the allowed lateness plus the end of the window + .advanceWatermarkTo(baseTime.plus(ALLOWED_LATENESS) + .plus(TEAM_WINDOW_DURATION).plus(Duration.standardMinutes(1))) + // These elements within the expired window are droppably late, and will not appear in the + // output + .addElements( + event(TestUser.BLUE_TWO, 3, TEAM_WINDOW_DURATION.minus(Duration.standardSeconds(5))), + event(TestUser.RED_ONE, 7, Duration.standardMinutes(4))) + .advanceWatermarkToInfinity(); + PCollection> teamScores = p.apply(infos) + .apply(new CalculateTeamScores(TEAM_WINDOW_DURATION, ALLOWED_LATENESS)); + + String blueTeam = TestUser.BLUE_ONE.getTeam(); + String redTeam = TestUser.RED_ONE.getTeam(); + // Only one on-time pane and no late panes should be emitted + PAssert.that(teamScores) + .inWindow(window) + .containsInAnyOrder(KV.of(redTeam, 7), KV.of(blueTeam, 18)); + // 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(); + } + + /** + * A test where elements arrive both on-time and late in {@link CalculateUserScores}, which emits + * output into the {@link GlobalWindow}. All elements that arrive should be taken into account, + * even if they arrive later than the maximum allowed lateness. + */ + @Test + public void testUserScore() { + TestPipeline p = TestPipeline.create(); + + TestStream infos = + TestStream.create(AvroCoder.of(GameActionInfo.class)) + .addElements( + event(TestUser.BLUE_ONE, 12, Duration.ZERO), + event(TestUser.RED_ONE, 3, Duration.ZERO)) + .advanceProcessingTime(Duration.standardMinutes(7)) + .addElements( + event(TestUser.RED_ONE, 4, Duration.standardMinutes(2)), + event(TestUser.BLUE_TWO, 3, Duration.ZERO), + event(TestUser.BLUE_ONE, 3, Duration.standardMinutes(3))) + .advanceProcessingTime(Duration.standardMinutes(5)) + .advanceWatermarkTo(baseTime.plus(ALLOWED_LATENESS).plus(Duration.standardHours(12))) + // Late elements are always observable within the global window - they arrive before + // the window closes, so they will appear in a pane, even if they arrive after the + // allowed lateness, and are taken into account alongside on-time elements + .addElements( + event(TestUser.RED_ONE, 3, Duration.standardMinutes(7)), + event(TestUser.RED_ONE, 2, (ALLOWED_LATENESS).plus(Duration.standardHours(13)))) + .advanceProcessingTime(Duration.standardMinutes(6)) + .addElements(event(TestUser.BLUE_TWO, 5, Duration.standardMinutes(12))) + .advanceProcessingTime(Duration.standardMinutes(20)) + .advanceWatermarkToInfinity(); + + PCollection> userScores = + p.apply(infos).apply(new CalculateUserScores(ALLOWED_LATENESS)); + + // User scores are emitted in speculative panes in the Global Window - this matcher choice + // ensures that panes emitted by the watermark advancing to positive infinity are not included, + // as that will not occur outside of tests + that(userScores) + .inEarlyGlobalWindowPanes() + .containsInAnyOrder(KV.of(TestUser.BLUE_ONE.getUser(), 15), + KV.of(TestUser.RED_ONE.getUser(), 7), + KV.of(TestUser.RED_ONE.getUser(), 12), + KV.of(TestUser.BLUE_TWO.getUser(), 3), + KV.of(TestUser.BLUE_TWO.getUser(), 8)); + + p.run(); + } + + private TimestampedValue event( + TestUser user, + int score, + Duration baseTimeOffset) { + return TimestampedValue.of(new GameActionInfo(user.getUser(), + user.getTeam(), + score, + baseTime.plus(baseTimeOffset).getMillis()), baseTime.plus(baseTimeOffset)); + } +} From 8007bdf5c3e1d3ee0c2c697e049549e9742442aa Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Jean-Baptiste=20Onofr=C3=A9?= Date: Sat, 27 Aug 2016 14:01:34 +0200 Subject: [PATCH 091/346] [BEAM-569] Define maxNumRecords default value to Long.MAX_VALUE in JmsIO --- .../src/main/java/org/apache/beam/sdk/io/jms/JmsIO.java | 8 ++------ 1 file changed, 2 insertions(+), 6 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 29d0c5fac1d83..3107aab9fcff1 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 @@ -101,11 +101,11 @@ public class JmsIO { private static final Logger LOG = LoggerFactory.getLogger(JmsIO.class); public static Read read() { - return new Read(); + return new Read(null, null, null, Long.MAX_VALUE, null); } public static Write write() { - return new Write(); + return new Write(null, null, null); } /** @@ -185,8 +185,6 @@ public void populateDisplayData(DisplayData.Builder builder) { protected long maxNumRecords; protected Duration maxReadTime; - private Read() {} - private Read( ConnectionFactory connectionFactory, String queue, @@ -428,8 +426,6 @@ public Write withTopic(String topic) { return new Write(connectionFactory, queue, topic); } - private Write() {} - private Write(ConnectionFactory connectionFactory, String queue, String topic) { this.connectionFactory = connectionFactory; this.queue = queue; From 6ae4b6a3df5cf3b834505fcb3f21df0e90473a0f Mon Sep 17 00:00:00 2001 From: Aljoscha Krettek Date: Thu, 25 Aug 2016 11:00:39 +0200 Subject: [PATCH 092/346] Address comments of Flink Side-Input PR --- .../beam/runners/core/SideInputHandler.java | 6 +- .../beam/runners/flink/FlinkRunner.java | 86 +++++++++++++++++-- .../wrappers/streaming/DoFnOperator.java | 13 ++- .../streaming/WindowDoFnOperator.java | 2 - 4 files changed, 89 insertions(+), 18 deletions(-) diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/SideInputHandler.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/SideInputHandler.java index a97d3f306d0ee..851ed37902ea1 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/SideInputHandler.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/SideInputHandler.java @@ -60,7 +60,11 @@ public class SideInputHandler implements ReadyCheckingSideInputReader { /** The list of side inputs that we're handling. */ protected final Collection> sideInputs; - /** State internals that are scoped not to the key of a value but instead to one key group. */ + /** + * State internals that are scoped not to the key of a value but are global. The state can still + * be keep locally but if side inputs are broadcast to all parallel operators then all will + * have the same view of the state. + */ private final StateInternals stateInternals; /** 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 8b1f42e5447cd..d3c65c00773c1 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 @@ -25,8 +25,13 @@ import java.net.URLClassLoader; import java.util.ArrayList; import java.util.Arrays; +import java.util.HashSet; import java.util.List; import java.util.Map; +import java.util.Set; +import java.util.SortedSet; +import java.util.TreeSet; + import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.CoderRegistry; @@ -35,6 +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.transforms.Combine; import org.apache.beam.sdk.transforms.OldDoFn; import org.apache.beam.sdk.transforms.PTransform; @@ -47,6 +53,8 @@ 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.flink.api.common.JobExecutionResult; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -108,6 +116,7 @@ public static FlinkRunner fromOptions(PipelineOptions options) { private FlinkRunner(FlinkPipelineOptions options) { this.options = options; + this.ptransformViewsWithNonDeterministicKeyCoders = new HashSet<>(); ImmutableMap.Builder, Class> builder = ImmutableMap., Class>builder(); if (options.isStreaming()) { @@ -124,6 +133,8 @@ private FlinkRunner(FlinkPipelineOptions options) { @Override public FlinkRunnerResult run(Pipeline pipeline) { + logWarningIfPCollectionViewHasNonDeterministicKeyCoder(pipeline); + LOG.info("Executing pipeline using FlinkRunner."); FlinkPipelineExecutionEnvironment env = new FlinkPipelineExecutionEnvironment(options); @@ -176,6 +187,7 @@ public OutputT apply( PTransform customTransform = InstanceBuilder.ofType(customTransformClass) + .withArg(FlinkRunner.class, this) .withArg(transformClass, transform) .build(); @@ -223,6 +235,59 @@ protected static List detectClassPathResourcesToStage( return files; } + /** A set of {@link View}s with non-deterministic key coders. */ + Set> ptransformViewsWithNonDeterministicKeyCoders; + + /** + * Records that the {@link PTransform} requires a deterministic key coder. + */ + private void recordViewUsesNonDeterministicKeyCoder(PTransform ptransform) { + ptransformViewsWithNonDeterministicKeyCoders.add(ptransform); + } + + /** Outputs a warning about PCollection views without deterministic key coders. */ + private void logWarningIfPCollectionViewHasNonDeterministicKeyCoder(Pipeline pipeline) { + // We need to wait till this point to determine the names of the transforms since only + // at this time do we know the hierarchy of the transforms otherwise we could + // have just recorded the full names during apply time. + if (!ptransformViewsWithNonDeterministicKeyCoders.isEmpty()) { + final SortedSet ptransformViewNamesWithNonDeterministicKeyCoders = new TreeSet<>(); + pipeline.traverseTopologically(new Pipeline.PipelineVisitor() { + @Override + public void visitValue(PValue value, TransformTreeNode producer) { + } + + @Override + public void visitPrimitiveTransform(TransformTreeNode node) { + if (ptransformViewsWithNonDeterministicKeyCoders.contains(node.getTransform())) { + ptransformViewNamesWithNonDeterministicKeyCoders.add(node.getFullName()); + } + } + + @Override + public CompositeBehavior enterCompositeTransform(TransformTreeNode node) { + if (ptransformViewsWithNonDeterministicKeyCoders.contains(node.getTransform())) { + ptransformViewNamesWithNonDeterministicKeyCoders.add(node.getFullName()); + } + return CompositeBehavior.ENTER_TRANSFORM; + } + + @Override + public void leaveCompositeTransform(TransformTreeNode node) { + } + }); + + LOG.warn("Unable to use indexed implementation for View.AsMap and View.AsMultimap for {} " + + "because the key coder is not deterministic. Falling back to singleton implementation " + + "which may cause memory and/or performance problems. Future major versions of " + + "the Flink runner will require deterministic key coders.", + ptransformViewNamesWithNonDeterministicKeyCoders); + } + } + + + ///////////////////////////////////////////////////////////////////////////// + /** * Specialized implementation for * {@link org.apache.beam.sdk.transforms.View.AsMap View.AsMap} @@ -231,8 +296,11 @@ protected static List detectClassPathResourcesToStage( private static class StreamingViewAsMap extends PTransform>, PCollectionView>> { + private final FlinkRunner runner; + @SuppressWarnings("unused") // used via reflection in FlinkRunner#apply() - public StreamingViewAsMap(View.AsMap transform) { + public StreamingViewAsMap(FlinkRunner runner, View.AsMap transform) { + this.runner = runner; } @Override @@ -248,7 +316,7 @@ public PCollectionView> apply(PCollection> input) { try { inputCoder.getKeyCoder().verifyDeterministic(); } catch (Coder.NonDeterministicException e) { -// runner.recordViewUsesNonDeterministicKeyCoder(this); + runner.recordViewUsesNonDeterministicKeyCoder(this); } return input @@ -270,11 +338,14 @@ protected String getKindString() { private static class StreamingViewAsMultimap extends PTransform>, PCollectionView>>> { + private final FlinkRunner runner; + /** * Builds an instance of this class from the overridden transform. */ @SuppressWarnings("unused") // used via reflection in FlinkRunner#apply() - public StreamingViewAsMultimap(View.AsMultimap transform) { + public StreamingViewAsMultimap(FlinkRunner runner, View.AsMultimap transform) { + this.runner = runner; } @Override @@ -290,7 +361,7 @@ public PCollectionView>> apply(PCollection> input) { try { inputCoder.getKeyCoder().verifyDeterministic(); } catch (Coder.NonDeterministicException e) { -// runner.recordViewUsesNonDeterministicKeyCoder(this); + runner.recordViewUsesNonDeterministicKeyCoder(this); } return input @@ -315,7 +386,7 @@ private static class StreamingViewAsList * Builds an instance of this class from the overridden transform. */ @SuppressWarnings("unused") // used via reflection in FlinkRunner#apply() - public StreamingViewAsList(View.AsList transform) {} + public StreamingViewAsList(FlinkRunner runner, View.AsList transform) {} @Override public PCollectionView> apply(PCollection input) { @@ -346,7 +417,7 @@ private static class StreamingViewAsIterable * Builds an instance of this class from the overridden transform. */ @SuppressWarnings("unused") // used via reflection in FlinkRunner#apply() - public StreamingViewAsIterable(View.AsIterable transform) { } + public StreamingViewAsIterable(FlinkRunner runner, View.AsIterable transform) { } @Override public PCollectionView> apply(PCollection input) { @@ -386,7 +457,7 @@ private static class StreamingViewAsSingleton * Builds an instance of this class from the overridden transform. */ @SuppressWarnings("unused") // used via reflection in FlinkRunner#apply() - public StreamingViewAsSingleton(View.AsSingleton transform) { + public StreamingViewAsSingleton(FlinkRunner runner, View.AsSingleton transform) { this.transform = transform; } @@ -443,6 +514,7 @@ private static class StreamingCombineGloballyAsSingletonView */ @SuppressWarnings("unused") // used via reflection in FlinkRunner#apply() public StreamingCombineGloballyAsSingletonView( + FlinkRunner runner, Combine.GloballyAsSingletonView transform) { this.transform = transform; } 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 000d69f34823e..2c7ebc697709c 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 @@ -75,11 +75,12 @@ import org.apache.flink.streaming.runtime.tasks.StreamTaskState; /** - * Flink operator for executing {@link DoFn DoFns}. + * Flink operator for executing {@link OldDoFn DoFns}. * - * @param - * @param - * @param + * @param the input type of the {@link OldDoFn} + * @param the output type of the {@link OldDoFn} + * @param the output type of the operator, this can be different from the fn output type when we have + * side outputs */ public class DoFnOperator extends AbstractStreamOperator @@ -95,8 +96,6 @@ public class DoFnOperator protected final Collection> sideInputs; protected final Map> sideInputTagMapping; - protected final boolean hasSideInputs; - protected final WindowingStrategy windowingStrategy; protected final OutputManagerFactory outputManagerFactory; @@ -136,8 +135,6 @@ public DoFnOperator( this.windowingStrategy = windowingStrategy; this.outputManagerFactory = outputManagerFactory; - this.hasSideInputs = !sideInputs.isEmpty(); - this.pushedBackWatermarkDescriptor = new ReducingStateDescriptor<>( "pushed-back-elements-watermark-hold", 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 c6dde5197dce0..01cfa5b6a26bf 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 @@ -201,8 +201,6 @@ public void processWatermark1(Watermark mark) throws Exception { if (timer != null && timer.f1.getTimestamp().getMillis() < actualInputWatermark) { fire = true; - System.out.println("FIRING: " + timer); - watermarkTimersQueue.remove(); watermarkTimers.remove(timer); From 1524494f77ca8667d57b00651b39005524c71d64 Mon Sep 17 00:00:00 2001 From: Aljoscha Krettek Date: Wed, 31 Aug 2016 13:42:30 +0200 Subject: [PATCH 093/346] Fix condition in FlinkStreamingPipelineTranslator --- .../flink/translation/FlinkStreamingPipelineTranslator.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 b12745512c93e..284cd2321e20f 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 @@ -84,7 +84,7 @@ public void visitPrimitiveTransform(TransformTreeNode node) { StreamTransformTranslator translator = FlinkStreamingTransformTranslators.getTranslator(transform); - if (translator == null && applyCanTranslate(transform, node, translator)) { + if (translator == null || !applyCanTranslate(transform, node, translator)) { LOG.info(node.getTransform().getClass().toString()); throw new UnsupportedOperationException( "The transform " + transform + " is currently not supported."); From 798566c3e0f9bbd535dc9336206ee3a8053911a8 Mon Sep 17 00:00:00 2001 From: bchambers Date: Wed, 3 Aug 2016 13:38:43 -0700 Subject: [PATCH 094/346] Correct some accidental renames IDE over-eagerly replaced some occurrences of createAggregator with createAggregatorForDoFn. This corrects that. --- .../java/org/apache/beam/sdk/util/DoFnRunnerBase.java | 2 +- .../java/org/apache/beam/sdk/transforms/Aggregator.java | 8 ++++---- .../java/org/apache/beam/sdk/transforms/DoFnTester.java | 2 +- 3 files changed, 6 insertions(+), 6 deletions(-) diff --git a/runners/core-java/src/main/java/org/apache/beam/sdk/util/DoFnRunnerBase.java b/runners/core-java/src/main/java/org/apache/beam/sdk/util/DoFnRunnerBase.java index 04a0978b60301..f0cfd7455de5c 100644 --- a/runners/core-java/src/main/java/org/apache/beam/sdk/util/DoFnRunnerBase.java +++ b/runners/core-java/src/main/java/org/apache/beam/sdk/util/DoFnRunnerBase.java @@ -344,7 +344,7 @@ public void sideOutputWithTimestamp(TupleTag tag, T output, Instant times @Override protected Aggregator createAggregatorInternal( String name, CombineFn combiner) { - checkNotNull(combiner, "Combiner passed to createAggregatorForDoFn cannot be null"); + checkNotNull(combiner, "Combiner passed to createAggregatorInternal cannot be null"); return aggregatorFactory.createAggregatorForDoFn(fn.getClass(), stepContext, name, combiner); } } 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 67d399fbc8207..e8f624792439a 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 @@ -25,7 +25,7 @@ * to be combined across all bundles. * *

              Aggregators are created by calling - * {@link DoFn#createAggregator DoFn.createAggregatorForDoFn}, + * {@link DoFn#createAggregator DoFn.createAggregator}, * typically from the {@link DoFn} constructor. Elements can be added to the * {@code Aggregator} by calling {@link Aggregator#addValue}. * @@ -41,7 +41,7 @@ * private Aggregator myAggregator; * * public MyDoFn() { - * myAggregator = createAggregatorForDoFn("myAggregator", new Sum.SumIntegerFn()); + * myAggregator = createAggregator("myAggregator", new Sum.SumIntegerFn()); * } * * @ProcessElement @@ -89,9 +89,9 @@ Aggregator createAggregatorForDoFn( } // TODO: Consider the following additional API conveniences: - // - In addition to createAggregatorForDoFn(), consider adding getAggregator() to + // - In addition to createAggregator(), consider adding getAggregator() to // avoid the need to store the aggregator locally in a DoFn, i.e., create // if not already present. // - Add a shortcut for the most common aggregator: - // c.createAggregatorForDoFn("name", new Sum.SumIntegerFn()). + // c.createAggregator("name", new Sum.SumIntegerFn()). } 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 6801768c10967..b867a5582b97c 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 @@ -667,7 +667,7 @@ protected Aggregator createAggreg String name, CombineFn combiner) { throw new IllegalStateException("Aggregators should not be created within ProcessContext. " + "Instead, create an aggregator at OldDoFn construction time with" - + " createAggregatorForDoFn, and ensure they are set up by the time startBundle is" + + " createAggregator, and ensure they are set up by the time startBundle is" + " called with setupDelegateAggregators."); } } From 4251761de2eced90235696767cef941332e8427c Mon Sep 17 00:00:00 2001 From: Thomas Groh Date: Tue, 30 Aug 2016 14:17:50 -0700 Subject: [PATCH 095/346] Test that multiple instances of TestStream are supported Add KeyedResourcePool This interface represents some shared pool of values that may be used by at most one caller at a time. Add LockedKeyedResourcePool which has at most one value per key and at most one user per value at a time. Use KeyedResourcePool in TestStream --- .../runners/direct/KeyedResourcePool.java | 47 ++++ .../direct/LockedKeyedResourcePool.java | 95 ++++++++ .../direct/TestStreamEvaluatorFactory.java | 141 +++++++----- .../direct/LockedKeyedResourcePoolTest.java | 163 ++++++++++++++ .../TestStreamEvaluatorFactoryTest.java | 206 ++++++++++++++++++ .../beam/sdk/testing/TestStreamTest.java | 29 +++ 6 files changed, 623 insertions(+), 58 deletions(-) create mode 100644 runners/direct-java/src/main/java/org/apache/beam/runners/direct/KeyedResourcePool.java create mode 100644 runners/direct-java/src/main/java/org/apache/beam/runners/direct/LockedKeyedResourcePool.java create mode 100644 runners/direct-java/src/test/java/org/apache/beam/runners/direct/LockedKeyedResourcePoolTest.java create mode 100644 runners/direct-java/src/test/java/org/apache/beam/runners/direct/TestStreamEvaluatorFactoryTest.java diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/KeyedResourcePool.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/KeyedResourcePool.java new file mode 100644 index 0000000000000..b976b696925d1 --- /dev/null +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/KeyedResourcePool.java @@ -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. + */ + +package org.apache.beam.runners.direct; + +import com.google.common.base.Optional; +import java.util.concurrent.Callable; +import java.util.concurrent.ExecutionException; + +/** + * A pool of resources associated with specific keys. Implementations enforce specific use patterns, + * such as limiting the the number of outstanding elements available per key. + */ +interface KeyedResourcePool { + /** + * Tries to acquire a value for the provided key, loading it via the provided loader if necessary. + * + *

              If the returned {@link Optional} contains a value, the caller obtains ownership of that + * value. The value should be released back to this {@link KeyedResourcePool} after the + * caller no longer has use of it using {@link #release(Object, Object)}. + * + *

              The provided {@link Callable} must not return null; it may either return a non-null + * value or throw an exception. + */ + Optional tryAcquire(K key, Callable loader) throws ExecutionException; + + /** + * Release the provided value, relinquishing ownership of it. Future calls to + * {@link #tryAcquire(Object, Callable)} may return the released value. + */ + void release(K key, V value); +} diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/LockedKeyedResourcePool.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/LockedKeyedResourcePool.java new file mode 100644 index 0000000000000..8b1e0b17e618b --- /dev/null +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/LockedKeyedResourcePool.java @@ -0,0 +1,95 @@ +/* + * 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 com.google.common.base.Preconditions.checkNotNull; +import static com.google.common.base.Preconditions.checkState; + +import com.google.common.base.Optional; +import com.google.common.util.concurrent.ExecutionError; +import com.google.common.util.concurrent.UncheckedExecutionException; +import java.util.concurrent.Callable; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.ExecutionException; + +/** + * A {@link KeyedResourcePool} which is limited to at most one outstanding instance at a time for + * each key. + */ +class LockedKeyedResourcePool implements KeyedResourcePool { + /** + * A map from each key to an {@link Optional} of the associated value. At most one value is stored + * per key, and it is obtained by at most one thread at a time. + * + *

              For each key in this map: + * + *

                + *
              • If there is no associated value, then no value has been stored yet. + *
              • If the value is {@code Optional.absent()} then the value is currently in use. + *
              • If the value is {@code Optional.present()} then the contained value is available for use. + *
              + */ + public static LockedKeyedResourcePool create() { + return new LockedKeyedResourcePool<>(); + } + + private final ConcurrentMap> cache; + + private LockedKeyedResourcePool() { + cache = new ConcurrentHashMap<>(); + } + + @Override + public Optional tryAcquire(K key, Callable loader) throws ExecutionException { + Optional value = cache.replace(key, Optional.absent()); + if (value == null) { + // No value already existed, so populate the cache with the value returned by the loader + cache.putIfAbsent(key, Optional.of(load(loader))); + // Some other thread may obtain the result after the putIfAbsent, so retry acquisition + value = cache.replace(key, Optional.absent()); + } + return value; + } + + private V load(Callable loader) throws ExecutionException { + try { + return loader.call(); + } catch (Error t) { + throw new ExecutionError(t); + } catch (RuntimeException e) { + throw new UncheckedExecutionException(e); + } catch (Exception e) { + throw new ExecutionException(e); + } + } + + @Override + public void release(K key, V value) { + Optional replaced = cache.replace(key, Optional.of(value)); + checkNotNull(replaced, "Tried to release before a value was acquired"); + checkState( + !replaced.isPresent(), + "Released a value to a %s where there is already a value present for key %s (%s). " + + "At most one value may be present at a time.", + LockedKeyedResourcePool.class.getSimpleName(), + key, + replaced); + } +} 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 e9f37bab7567d..3dbd886fc75ed 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 @@ -22,12 +22,12 @@ import com.google.common.base.Supplier; import java.util.List; -import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.Callable; +import java.util.concurrent.ExecutionException; import java.util.concurrent.atomic.AtomicReference; 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.Pipeline; import org.apache.beam.sdk.runners.PipelineRunner; import org.apache.beam.sdk.testing.TestStream; import org.apache.beam.sdk.testing.TestStream.ElementEvent; @@ -49,43 +49,52 @@ import org.joda.time.Duration; import org.joda.time.Instant; -/** - * The {@link TransformEvaluatorFactory} for the {@link TestStream} primitive. - */ +/** The {@link TransformEvaluatorFactory} for the {@link TestStream} primitive. */ class TestStreamEvaluatorFactory implements TransformEvaluatorFactory { - private final AtomicBoolean inUse = new AtomicBoolean(false); - private final AtomicReference> evaluator = new AtomicReference<>(); + private final KeyedResourcePool, Evaluator> evaluators = + LockedKeyedResourcePool.create(); @Nullable @Override public TransformEvaluator forApplication( AppliedPTransform application, @Nullable CommittedBundle inputBundle, - EvaluationContext evaluationContext) throws Exception { + EvaluationContext evaluationContext) + throws Exception { return createEvaluator((AppliedPTransform) application, evaluationContext); } @Override public void cleanup() throws Exception {} + /** + * Returns the evaluator for the provided application of {@link TestStream}, or null if it is + * already in use. + * + *

              The documented behavior of {@link TestStream} requires the output of one event to travel + * completely through the pipeline before any additional event, so additional instances that have + * a separate collection of events cannot be created. + */ private TransformEvaluator createEvaluator( AppliedPTransform, TestStream> application, - EvaluationContext evaluationContext) { - if (evaluator.get() == null) { - Evaluator createdEvaluator = new Evaluator<>(application, evaluationContext, inUse); - evaluator.compareAndSet(null, createdEvaluator); - } - if (inUse.compareAndSet(false, true)) { - return evaluator.get(); - } else { - return null; - } + EvaluationContext evaluationContext) + throws ExecutionException { + return evaluators + .tryAcquire(application, new CreateEvaluator<>(application, evaluationContext, evaluators)) + .orNull(); } + /** + * Release the provided {@link Evaluator} after completing an evaluation. The next call to {@link + * #createEvaluator(AppliedPTransform, EvaluationContext)} with the {@link AppliedPTransform} will + * return this evaluator. + */ + private void completeEvaluation(Evaluator evaluator) {} + private static class Evaluator implements TransformEvaluator { private final AppliedPTransform, TestStream> application; private final EvaluationContext context; - private final AtomicBoolean inUse; + private final KeyedResourcePool, Evaluator> cache; private final List> events; private int index; private Instant currentWatermark; @@ -93,49 +102,48 @@ private static class Evaluator implements TransformEvaluator { private Evaluator( AppliedPTransform, TestStream> application, EvaluationContext context, - AtomicBoolean inUse) { + KeyedResourcePool, Evaluator> cache) { this.application = application; this.context = context; - this.inUse = inUse; + this.cache = cache; this.events = application.getTransform().getEvents(); index = 0; currentWatermark = BoundedWindow.TIMESTAMP_MIN_VALUE; } @Override - public void processElement(WindowedValue element) throws Exception { - } + public void processElement(WindowedValue element) throws Exception {} @Override public TransformResult finishBundle() throws Exception { - if (index >= events.size()) { - return StepTransformResult.withHold(application, BoundedWindow.TIMESTAMP_MAX_VALUE).build(); - } - Event event = events.get(index); - if (event.getType().equals(EventType.WATERMARK)) { - currentWatermark = ((WatermarkEvent) event).getWatermark(); - } - StepTransformResult.Builder result = - StepTransformResult.withHold(application, currentWatermark); - if (event.getType().equals(EventType.ELEMENT)) { - UncommittedBundle bundle = context.createRootBundle(application.getOutput()); - for (TimestampedValue elem : ((ElementEvent) event).getElements()) { - bundle.add(WindowedValue.timestampedValueInGlobalWindow(elem.getValue(), - elem.getTimestamp())); + try { + if (index >= events.size()) { + return StepTransformResult.withHold(application, BoundedWindow.TIMESTAMP_MAX_VALUE) + .build(); } - result.addOutput(bundle); - } - if (event.getType().equals(EventType.PROCESSING_TIME)) { - ((TestClock) context.getClock()) - .advance(((ProcessingTimeEvent) event).getProcessingTimeAdvance()); + Event event = events.get(index); + if (event.getType().equals(EventType.WATERMARK)) { + currentWatermark = ((WatermarkEvent) event).getWatermark(); + } + StepTransformResult.Builder result = + StepTransformResult.withHold(application, currentWatermark); + if (event.getType().equals(EventType.ELEMENT)) { + UncommittedBundle bundle = context.createRootBundle(application.getOutput()); + for (TimestampedValue elem : ((ElementEvent) event).getElements()) { + bundle.add( + WindowedValue.timestampedValueInGlobalWindow(elem.getValue(), elem.getTimestamp())); + } + result.addOutput(bundle); + } + if (event.getType().equals(EventType.PROCESSING_TIME)) { + ((TestClock) context.getClock()) + .advance(((ProcessingTimeEvent) event).getProcessingTimeAdvance()); + } + index++; + return result.build(); + } finally { + cache.release(application, this); } - index++; - checkState(inUse.compareAndSet(true, false), - "The InUse flag of a %s was changed while the source evaluator was executing. " - + "%s cannot be split or evaluated in parallel.", - TestStream.class.getSimpleName(), - TestStream.class.getSimpleName()); - return result.build(); } } @@ -181,20 +189,37 @@ private DirectTestStream(TestStream transform) { @Override public PCollection apply(PBegin input) { - setup(input.getPipeline()); - return PCollection.createPrimitiveOutputInternal( - input.getPipeline(), WindowingStrategy.globalDefault(), IsBounded.UNBOUNDED) - .setCoder(original.getValueCoder()); - } - - private void setup(Pipeline p) { - PipelineRunner runner = p.getRunner(); - checkState(runner instanceof DirectRunner, + PipelineRunner runner = input.getPipeline().getRunner(); + checkState( + runner instanceof DirectRunner, "%s can only be used when running with the %s", getClass().getSimpleName(), DirectRunner.class.getSimpleName()); ((DirectRunner) runner).setClockSupplier(new TestClockSupplier()); + return PCollection.createPrimitiveOutputInternal( + input.getPipeline(), WindowingStrategy.globalDefault(), IsBounded.UNBOUNDED) + .setCoder(original.getValueCoder()); } } } + + private static class CreateEvaluator implements Callable> { + private final AppliedPTransform, TestStream> application; + private final EvaluationContext evaluationContext; + private final KeyedResourcePool, Evaluator> evaluators; + + public CreateEvaluator( + AppliedPTransform, TestStream> application, + EvaluationContext evaluationContext, + KeyedResourcePool, Evaluator> evaluators) { + this.application = application; + this.evaluationContext = evaluationContext; + this.evaluators = evaluators; + } + + @Override + public Evaluator call() throws Exception { + return new Evaluator<>(application, evaluationContext, evaluators); + } + } } diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/LockedKeyedResourcePoolTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/LockedKeyedResourcePoolTest.java new file mode 100644 index 0000000000000..e1e24a37276ab --- /dev/null +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/LockedKeyedResourcePoolTest.java @@ -0,0 +1,163 @@ +/* + * 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.equalTo; +import static org.hamcrest.Matchers.is; +import static org.junit.Assert.assertThat; + +import com.google.common.base.Optional; +import com.google.common.util.concurrent.ExecutionError; +import com.google.common.util.concurrent.UncheckedExecutionException; +import java.util.concurrent.Callable; +import java.util.concurrent.ExecutionException; +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 LockedKeyedResourcePool}. + */ +@RunWith(JUnit4.class) +public class LockedKeyedResourcePoolTest { + @Rule public ExpectedException thrown = ExpectedException.none(); + private LockedKeyedResourcePool cache = + LockedKeyedResourcePool.create(); + + @Test + public void acquireReleaseAcquireLastLoadedOrReleased() throws ExecutionException { + Optional returned = cache.tryAcquire("foo", new Callable() { + @Override + public Integer call() throws Exception { + return 3; + } + }); + assertThat(returned.get(), equalTo(3)); + + cache.release("foo", 4); + Optional reacquired = cache.tryAcquire("foo", new Callable() { + @Override + public Integer call() throws Exception { + return 5; + } + }); + assertThat(reacquired.get(), equalTo(4)); + } + + @Test + public void acquireReleaseReleaseThrows() throws ExecutionException { + Optional returned = cache.tryAcquire("foo", new Callable() { + @Override + public Integer call() throws Exception { + return 3; + } + }); + assertThat(returned.get(), equalTo(3)); + + cache.release("foo", 4); + thrown.expect(IllegalStateException.class); + thrown.expectMessage("already a value present"); + thrown.expectMessage("At most one"); + cache.release("foo", 4); + } + + @Test + public void releaseBeforeAcquireThrows() { + thrown.expect(NullPointerException.class); + thrown.expectMessage("before a value was acquired"); + cache.release("bar", 3); + } + + @Test + public void multipleAcquireWithoutReleaseAbsent() throws ExecutionException { + Optional returned = cache.tryAcquire("foo", new Callable() { + @Override + public Integer call() throws Exception { + return 3; + } + }); + Optional secondReturned = cache.tryAcquire("foo", new Callable() { + @Override + public Integer call() throws Exception { + return 3; + } + }); + assertThat(secondReturned.isPresent(), is(false)); + } + + @Test + public void acquireMultipleKeysSucceeds() throws ExecutionException { + Optional returned = cache.tryAcquire("foo", new Callable() { + @Override + public Integer call() throws Exception { + return 3; + } + }); + Optional secondReturned = cache.tryAcquire("bar", new Callable() { + @Override + public Integer call() throws Exception { + return 4; + } + }); + + assertThat(returned.get(), equalTo(3)); + assertThat(secondReturned.get(), equalTo(4)); + } + + @Test + public void acquireThrowsExceptionWrapped() throws ExecutionException { + final Exception cause = new Exception("checkedException"); + thrown.expect(ExecutionException.class); + thrown.expectCause(equalTo(cause)); + cache.tryAcquire("foo", new Callable() { + @Override + public Integer call() throws Exception { + throw cause; + } + }); + } + + @Test + public void acquireThrowsRuntimeExceptionWrapped() throws ExecutionException { + final RuntimeException cause = new RuntimeException("UncheckedException"); + thrown.expect(UncheckedExecutionException.class); + thrown.expectCause(equalTo(cause)); + cache.tryAcquire("foo", new Callable() { + @Override + public Integer call() throws Exception { + throw cause; + } + }); + } + + @Test + public void acquireThrowsErrorWrapped() throws ExecutionException { + final Error cause = new Error("Error"); + thrown.expect(ExecutionError.class); + thrown.expectCause(equalTo(cause)); + cache.tryAcquire("foo", new Callable() { + @Override + public Integer call() throws Exception { + throw cause; + } + }); + } +} 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 new file mode 100644 index 0000000000000..7703881abf33a --- /dev/null +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/TestStreamEvaluatorFactoryTest.java @@ -0,0 +1,206 @@ +/* + * 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.equalTo; +import static org.hamcrest.Matchers.is; +import static org.hamcrest.Matchers.nullValue; +import static org.junit.Assert.assertThat; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +import com.google.common.collect.Iterables; +import org.apache.beam.sdk.coders.StringUtf8Coder; +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.windowing.BoundedWindow; +import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.PCollection; +import org.hamcrest.Matchers; +import org.joda.time.Instant; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +/** Tests for {@link TestStreamEvaluatorFactory}. */ +@RunWith(JUnit4.class) +public class TestStreamEvaluatorFactoryTest { + private TestStreamEvaluatorFactory factory = new TestStreamEvaluatorFactory(); + private BundleFactory bundleFactory = ImmutableListBundleFactory.create(); + + /** 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()) + .addElements(1, 2, 3) + .addElements(4, 5, 6) + .advanceWatermarkToInfinity()); + + EvaluationContext context = mock(EvaluationContext.class); + when(context.createRootBundle(streamVals)) + .thenReturn( + bundleFactory.createRootBundle(streamVals), bundleFactory.createRootBundle(streamVals)); + + TransformEvaluator firstEvaluator = + factory.forApplication(streamVals.getProducingTransformInternal(), null, context); + TransformResult firstResult = firstEvaluator.finishBundle(); + + TransformEvaluator secondEvaluator = + factory.forApplication(streamVals.getProducingTransformInternal(), null, context); + TransformResult secondResult = secondEvaluator.finishBundle(); + + TransformEvaluator thirdEvaluator = + factory.forApplication(streamVals.getProducingTransformInternal(), null, context); + TransformResult thirdResult = thirdEvaluator.finishBundle(); + + assertThat( + Iterables.getOnlyElement(firstResult.getOutputBundles()) + .commit(Instant.now()) + .getElements(), + Matchers.>containsInAnyOrder( + WindowedValue.valueInGlobalWindow(1), + WindowedValue.valueInGlobalWindow(2), + WindowedValue.valueInGlobalWindow(3))); + assertThat(firstResult.getWatermarkHold(), equalTo(BoundedWindow.TIMESTAMP_MIN_VALUE)); + + assertThat( + Iterables.getOnlyElement(secondResult.getOutputBundles()) + .commit(Instant.now()) + .getElements(), + Matchers.>containsInAnyOrder( + WindowedValue.valueInGlobalWindow(4), + WindowedValue.valueInGlobalWindow(5), + WindowedValue.valueInGlobalWindow(6))); + assertThat(secondResult.getWatermarkHold(), equalTo(BoundedWindow.TIMESTAMP_MIN_VALUE)); + + assertThat(Iterables.isEmpty(thirdResult.getOutputBundles()), is(true)); + assertThat(thirdResult.getWatermarkHold(), equalTo(BoundedWindow.TIMESTAMP_MAX_VALUE)); + } + + /** Demonstrates that at most one evaluator for an application is available at a time. */ + @Test + public void onlyOneEvaluatorAtATime() throws Exception { + TestPipeline p = TestPipeline.create(); + PCollection streamVals = + p.apply( + TestStream.create(VarIntCoder.of()).addElements(4, 5, 6).advanceWatermarkToInfinity()); + + EvaluationContext context = mock(EvaluationContext.class); + TransformEvaluator firstEvaluator = + factory.forApplication(streamVals.getProducingTransformInternal(), null, context); + + // create a second evaluator before the first is finished. The evaluator should not be available + TransformEvaluator secondEvaluator = + factory.forApplication(streamVals.getProducingTransformInternal(), null, context); + assertThat(secondEvaluator, is(nullValue())); + } + + /** + * Demonstrates that multiple applications of the same {@link TestStream} produce separate + * evaluators. + */ + @Test + public void multipleApplicationsMultipleEvaluators() throws Exception { + TestPipeline p = TestPipeline.create(); + TestStream stream = + TestStream.create(VarIntCoder.of()).addElements(2).advanceWatermarkToInfinity(); + PCollection firstVals = p.apply("Stream One", stream); + PCollection secondVals = p.apply("Stream A", stream); + + EvaluationContext context = mock(EvaluationContext.class); + when(context.createRootBundle(firstVals)).thenReturn(bundleFactory.createRootBundle(firstVals)); + when(context.createRootBundle(secondVals)) + .thenReturn(bundleFactory.createRootBundle(secondVals)); + + TransformEvaluator firstEvaluator = + factory.forApplication(firstVals.getProducingTransformInternal(), null, context); + // The two evaluators can exist independently + TransformEvaluator secondEvaluator = + factory.forApplication(secondVals.getProducingTransformInternal(), null, context); + + TransformResult firstResult = firstEvaluator.finishBundle(); + TransformResult secondResult = secondEvaluator.finishBundle(); + + assertThat( + Iterables.getOnlyElement(firstResult.getOutputBundles()) + .commit(Instant.now()) + .getElements(), + Matchers.>containsInAnyOrder(WindowedValue.valueInGlobalWindow(2))); + assertThat(firstResult.getWatermarkHold(), equalTo(BoundedWindow.TIMESTAMP_MIN_VALUE)); + + // They both produce equal results, and don't interfere with each other + assertThat( + Iterables.getOnlyElement(secondResult.getOutputBundles()) + .commit(Instant.now()) + .getElements(), + Matchers.>containsInAnyOrder(WindowedValue.valueInGlobalWindow(2))); + assertThat(secondResult.getWatermarkHold(), equalTo(BoundedWindow.TIMESTAMP_MIN_VALUE)); + } + + /** + * Demonstrates that multiple applications of different {@link TestStream} produce independent + * evaluators. + */ + @Test + public void multipleStreamsMultipleEvaluators() throws Exception { + TestPipeline p = TestPipeline.create(); + PCollection firstVals = + p.apply( + "Stream One", + TestStream.create(VarIntCoder.of()).addElements(2).advanceWatermarkToInfinity()); + PCollection secondVals = + p.apply( + "Stream A", + TestStream.create(StringUtf8Coder.of()) + .addElements("Two") + .advanceWatermarkToInfinity()); + + EvaluationContext context = mock(EvaluationContext.class); + when(context.createRootBundle(firstVals)).thenReturn(bundleFactory.createRootBundle(firstVals)); + when(context.createRootBundle(secondVals)) + .thenReturn(bundleFactory.createRootBundle(secondVals)); + + TransformEvaluator firstEvaluator = + factory.forApplication(firstVals.getProducingTransformInternal(), null, context); + // The two evaluators can exist independently + TransformEvaluator secondEvaluator = + factory.forApplication(secondVals.getProducingTransformInternal(), null, context); + + TransformResult firstResult = firstEvaluator.finishBundle(); + TransformResult secondResult = secondEvaluator.finishBundle(); + + assertThat( + Iterables.getOnlyElement(firstResult.getOutputBundles()) + .commit(Instant.now()) + .getElements(), + Matchers.>containsInAnyOrder(WindowedValue.valueInGlobalWindow(2))); + assertThat(firstResult.getWatermarkHold(), equalTo(BoundedWindow.TIMESTAMP_MIN_VALUE)); + + assertThat( + Iterables.getOnlyElement(secondResult.getOutputBundles()) + .commit(Instant.now()) + .getElements(), + Matchers.>containsInAnyOrder(WindowedValue.valueInGlobalWindow("Two"))); + assertThat(secondResult.getWatermarkHold(), equalTo(BoundedWindow.TIMESTAMP_MIN_VALUE)); + } +} 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 6457f910308f4..a1b4e4a8b3ede 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 @@ -264,6 +264,35 @@ public void testElementsAtAlmostPositiveInfinity() { p.run(); } + @Test + @Category(NeedsRunner.class) + public void testMultipleStreams() { + TestStream stream = TestStream.create(StringUtf8Coder.of()) + .addElements("foo", "bar") + .advanceWatermarkToInfinity(); + + TestStream other = + TestStream.create(VarIntCoder.of()).addElements(1, 2, 3, 4).advanceWatermarkToInfinity(); + + TestPipeline p = TestPipeline.create(); + PCollection createStrings = + p.apply("CreateStrings", stream) + .apply("WindowStrings", + Window.triggering(AfterPane.elementCountAtLeast(2)) + .withAllowedLateness(Duration.ZERO) + .accumulatingFiredPanes()); + PAssert.that(createStrings).containsInAnyOrder("foo", "bar"); + PCollection createInts = + p.apply("CreateInts", other) + .apply("WindowInts", + Window.triggering(AfterPane.elementCountAtLeast(4)) + .withAllowedLateness(Duration.ZERO) + .accumulatingFiredPanes()); + PAssert.that(createInts).containsInAnyOrder(1, 2, 3, 4); + + p.run(); + } + @Test public void testElementAtPositiveInfinityThrows() { Builder stream = From 28ad44d2ad5c995044966f8c70f6dcec91343eb6 Mon Sep 17 00:00:00 2001 From: Thomas Groh Date: Wed, 31 Aug 2016 15:34:21 -0700 Subject: [PATCH 096/346] Remove empty unused method in TestStreamEvaluatorFactory --- .../beam/runners/direct/TestStreamEvaluatorFactory.java | 7 ------- 1 file changed, 7 deletions(-) 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 3dbd886fc75ed..5fe771c52121a 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 @@ -84,13 +84,6 @@ private TransformEvaluator createEvaluator( .orNull(); } - /** - * Release the provided {@link Evaluator} after completing an evaluation. The next call to {@link - * #createEvaluator(AppliedPTransform, EvaluationContext)} with the {@link AppliedPTransform} will - * return this evaluator. - */ - private void completeEvaluation(Evaluator evaluator) {} - private static class Evaluator implements TransformEvaluator { private final AppliedPTransform, TestStream> application; private final EvaluationContext context; From 6ee7b620bf8e2ee07c0f30e9ff20363e79765405 Mon Sep 17 00:00:00 2001 From: Scott Wegner Date: Thu, 18 Aug 2016 13:56:34 -0700 Subject: [PATCH 097/346] Add Latest CombineFn and PTransforms Add DoFnTester support for specifying input timestamps --- .../apache/beam/sdk/coders/NullableCoder.java | 7 + .../beam/sdk/transforms/DoFnTester.java | 33 ++- .../apache/beam/sdk/transforms/Latest.java | 203 +++++++++++++++ .../beam/sdk/values/TimestampedValue.java | 14 ++ .../beam/sdk/transforms/DoFnTesterTest.java | 34 ++- .../beam/sdk/transforms/LatestFnTests.java | 233 ++++++++++++++++++ .../beam/sdk/transforms/LatestTest.java | 146 +++++++++++ .../beam/sdk/values/TimestampedValueTest.java | 83 +++++++ 8 files changed, 747 insertions(+), 6 deletions(-) create mode 100644 sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Latest.java create mode 100644 sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/LatestFnTests.java create mode 100644 sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/LatestTest.java create mode 100644 sdks/java/core/src/test/java/org/apache/beam/sdk/values/TimestampedValueTest.java diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/NullableCoder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/NullableCoder.java index 44aadbdd88ccb..9c6c7c0f694ef 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/NullableCoder.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/NullableCoder.java @@ -65,6 +65,13 @@ private NullableCoder(Coder valueCoder) { this.valueCoder = valueCoder; } + /** + * Returns the inner {@link Coder} wrapped by this {@link NullableCoder} instance. + */ + public Coder getValueCoder() { + return valueCoder; + } + @Override public void encode(@Nullable T value, OutputStream outStream, Context context) throws IOException, CoderException { 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 b867a5582b97c..0e018ba2fa422 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 @@ -17,6 +17,9 @@ */ package org.apache.beam.sdk.transforms; +import static com.google.common.base.Preconditions.checkNotNull; +import static com.google.common.base.Preconditions.checkState; + import com.google.common.base.Function; import com.google.common.base.MoreObjects; import com.google.common.collect.ImmutableList; @@ -221,9 +224,26 @@ private static void unwrapUserCodeException(UserCodeException e) throws Exceptio * been finished */ public void processElement(InputT element) throws Exception { - if (state == State.FINISHED) { - throw new IllegalStateException("finishBundle() has already been called"); - } + processTimestampedElement(TimestampedValue.atMinimumTimestamp(element)); + } + + /** + * Calls {@link OldDoFn#processElement} on the {@code OldDoFn} under test, in a + * context where {@link OldDoFn.ProcessContext#element} returns the + * given element and timestamp. + * + *

              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. + * + * @throws IllegalStateException if the {@code OldDoFn} under test has already + * been finished + */ + public void processTimestampedElement(TimestampedValue element) throws Exception { + checkNotNull(element, "Timestamped element cannot be null"); + checkState(state != State.FINISHED, "finishBundle() has already been called"); + if (state == State.UNSTARTED) { startBundle(); } @@ -522,10 +542,13 @@ private List> getOutputList(TupleTag tag) { private TestProcessContext createProcessContext( OldDoFn fn, - InputT elem) { + TimestampedValue elem) { + WindowedValue windowedValue = WindowedValue.timestampedValueInGlobalWindow( + elem.getValue(), elem.getTimestamp()); + return new TestProcessContext<>(fn, createContext(fn), - WindowedValue.valueInGlobalWindow(elem), + windowedValue, mainOutputTag, sideInputs); } 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 new file mode 100644 index 0000000000000..7f13649b783a5 --- /dev/null +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Latest.java @@ -0,0 +1,203 @@ +/* + * 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.transforms; + +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 java.util.Iterator; +import org.apache.beam.sdk.coders.CannotProvideCoderException; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.coders.CoderRegistry; +import org.apache.beam.sdk.coders.KvCoder; +import org.apache.beam.sdk.coders.NullableCoder; +import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.TimestampedValue; + +/** + * {@link PTransform} and {@link Combine.CombineFn} for computing the latest element + * in a {@link PCollection}. + * + *

              Example 1: compute the latest value for each session: + *

              
              + * PCollection input = ...;
              + * PCollection sessioned = input
              + *    .apply(Window.into(Sessions.withGapDuration(Duration.standardMinutes(5)));
              + * PCollection latestValues = sessioned.apply(Latest.globally());
              + * 
              + * + *

              Example 2: track a latest computed value in an aggregator: + *

              
              + * class MyDoFn extends DoFn {
              + *  private Aggregator, Double> latestValue =
              + *    createAggregator("latestValue", new Latest.LatestFn());
              + *
              + *  {@literal @}ProcessElement
              + *  public void processElement(ProcessContext c) {
              + *    double val = // ..
              + *    latestValue.addValue(TimestampedValue.of(val, c.timestamp()));
              + *    // ..
              + *  }
              + * }
              + * 
              + * + *

              For elements with the same timestamp, the element chosen for output is arbitrary. + */ +public class Latest { + // Do not instantiate + private Latest() {} + + /** + * A {@link Combine.CombineFn} that computes the latest element from a set of inputs. This is + * particularly useful as an {@link Aggregator}. + * + * @param Type of input element. + * @see Latest + */ + public static class LatestFn + extends Combine.CombineFn, TimestampedValue, T> { + /** Construct a new {@link LatestFn} instance. */ + public LatestFn() {} + + @Override + public TimestampedValue createAccumulator() { + return TimestampedValue.atMinimumTimestamp(null); + } + + @Override + public TimestampedValue addInput(TimestampedValue accumulator, + TimestampedValue input) { + checkNotNull(accumulator, "accumulator must be non-null"); + checkNotNull(input, "input must be non-null"); + + if (input.getTimestamp().isBefore(accumulator.getTimestamp())) { + return accumulator; + } else { + return input; + } + } + + @Override + public Coder> getAccumulatorCoder(CoderRegistry registry, + Coder> inputCoder) throws CannotProvideCoderException { + return NullableCoder.of(inputCoder); + } + + @Override + public Coder getDefaultOutputCoder(CoderRegistry registry, + Coder> inputCoder) throws CannotProvideCoderException { + checkState(inputCoder instanceof TimestampedValue.TimestampedValueCoder, + "inputCoder must be a TimestampedValueCoder, but was %s", inputCoder); + + TimestampedValue.TimestampedValueCoder inputTVCoder = + (TimestampedValue.TimestampedValueCoder) inputCoder; + return NullableCoder.of(inputTVCoder.getValueCoder()); + } + + @Override + public TimestampedValue mergeAccumulators(Iterable> accumulators) { + checkNotNull(accumulators, "accumulators must be non-null"); + + Iterator> iter = accumulators.iterator(); + if (!iter.hasNext()) { + return createAccumulator(); + } + + TimestampedValue merged = iter.next(); + while (iter.hasNext()) { + merged = addInput(merged, iter.next()); + } + + return merged; + } + + @Override + public T extractOutput(TimestampedValue accumulator) { + return accumulator.getValue(); + } + } + + /** + * Returns a {@link PTransform} that takes as input a {@link PCollection} and returns a + * {@link PCollection} whose contents is the latest element according to its event time, or + * {@literal null} if there are no elements. + * + * @param The type of the elements being combined. + */ + public static PTransform, PCollection> globally() { + return new Globally<>(); + } + + /** + * Returns a {@link PTransform} that takes as input a {@code PCollection>} and returns a + * {@code PCollection>} whose contents is the latest element per-key according to its + * event time. + * + * @param The key type of the elements being combined. + * @param The value type of the elements being combined. + */ + public static PTransform>, PCollection>> perKey() { + return new PerKey<>(); + } + + private static class Globally extends PTransform, PCollection> { + @Override + public PCollection apply(PCollection input) { + Coder inputCoder = input.getCoder(); + + return input + .apply("Reify Timestamps", ParDo.of( + new DoFn>() { + @ProcessElement + public void processElement(ProcessContext c) { + c.output(TimestampedValue.of(c.element(), c.timestamp())); + } + })).setCoder(TimestampedValue.TimestampedValueCoder.of(inputCoder)) + .apply("Latest Value", Combine.globally(new LatestFn())) + .setCoder(NullableCoder.of(inputCoder)); + } + } + + private static class PerKey + extends PTransform>, PCollection>> { + @Override + public PCollection> apply(PCollection> input) { + checkNotNull(input); + checkArgument(input.getCoder() instanceof KvCoder, + "Input specifiedCoder must be an instance of KvCoder, but was %s", input.getCoder()); + + @SuppressWarnings("unchecked") + KvCoder inputCoder = (KvCoder) input.getCoder(); + return input + .apply("Reify Timestamps", ParDo.of( + new DoFn, KV>>() { + @ProcessElement + public void processElement(ProcessContext c) { + c.output(KV.of(c.element().getKey(), TimestampedValue.of(c.element().getValue(), + c.timestamp()))); + } + })).setCoder(KvCoder.of( + inputCoder.getKeyCoder(), + TimestampedValue.TimestampedValueCoder.of(inputCoder.getValueCoder()))) + .apply("Latest Value", Combine., V>perKey(new LatestFn())) + .setCoder(inputCoder); + } + } +} diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/TimestampedValue.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/TimestampedValue.java index f2ad6168bb652..dd80fb2d06c55 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/TimestampedValue.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/TimestampedValue.java @@ -31,6 +31,7 @@ import org.apache.beam.sdk.coders.Coder; 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.util.PropertyNames; import org.joda.time.Instant; @@ -43,6 +44,13 @@ * @param the type of the value */ public class TimestampedValue { + /** + * Returns a new {@link TimestampedValue} with the + * {@link BoundedWindow#TIMESTAMP_MIN_VALUE minimum timestamp}. + */ + public static TimestampedValue atMinimumTimestamp(V value) { + return of(value, BoundedWindow.TIMESTAMP_MIN_VALUE); + } /** * Returns a new {@code TimestampedValue} with the given value and timestamp. @@ -136,6 +144,10 @@ public List> getCoderArguments() { return Arrays.>asList(valueCoder); } + public Coder getValueCoder() { + return valueCoder; + } + public static List getInstanceComponents(TimestampedValue exampleValue) { return Arrays.asList(exampleValue.getValue()); } @@ -147,6 +159,8 @@ public static List getInstanceComponents(TimestampedValue example private final Instant timestamp; protected TimestampedValue(V value, Instant timestamp) { + checkNotNull(timestamp, "timestamp must be non-null"); + this.value = value; this.timestamp = timestamp; } 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 2649be5aadf30..3ed30fdd0c67d 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 @@ -17,6 +17,7 @@ */ package org.apache.beam.sdk.transforms; +import static org.hamcrest.Matchers.contains; import static org.hamcrest.Matchers.containsInAnyOrder; import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.hasItems; @@ -35,7 +36,9 @@ import org.apache.beam.sdk.values.TimestampedValue; import org.hamcrest.Matchers; import org.joda.time.Instant; +import org.junit.Rule; import org.junit.Test; +import org.junit.rules.ExpectedException; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; @@ -44,6 +47,7 @@ */ @RunWith(JUnit4.class) public class DoFnTesterTest { + @Rule public ExpectedException thrown = ExpectedException.none(); @Test public void processElement() throws Exception { @@ -125,6 +129,16 @@ public void processElementsWithPeeks() throws Exception { assertTrue(deserializedDoFn.wasFinishBundleCalled()); } + @Test + public void processElementAfterFinish() throws Exception { + DoFnTester tester = DoFnTester.of(new CounterDoFn()); + tester.finishBundle(); + + thrown.expect(IllegalStateException.class); + thrown.expectMessage("finishBundle() has already been called"); + tester.processElement(1L); + } + @Test public void processBatch() throws Exception { CounterDoFn counterDoFn = new CounterDoFn(); @@ -145,7 +159,25 @@ public void processBatch() throws Exception { } @Test - public void processElementWithTimestamp() throws Exception { + public void processTimestampedElement() throws Exception { + DoFn> reifyTimestamps = new ReifyTimestamps(); + + DoFnTester> tester = DoFnTester.of(reifyTimestamps); + + TimestampedValue input = TimestampedValue.of(1L, new Instant(100)); + tester.processTimestampedElement(input); + assertThat(tester.takeOutputElements(), contains(input)); + } + + static class ReifyTimestamps extends DoFn> { + @ProcessElement + public void processElement(ProcessContext c) { + c.output(TimestampedValue.of(c.element(), c.timestamp())); + } + } + + @Test + public void processElementWithOutputTimestamp() throws Exception { CounterDoFn counterDoFn = new CounterDoFn(); DoFnTester tester = DoFnTester.of(counterDoFn); diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/LatestFnTests.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/LatestFnTests.java new file mode 100644 index 0000000000000..84b5b68d6b4f0 --- /dev/null +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/LatestFnTests.java @@ -0,0 +1,233 @@ +/* + * 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.transforms; + +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.instanceOf; +import static org.hamcrest.Matchers.isOneOf; +import static org.hamcrest.Matchers.nullValue; +import static org.junit.Assert.assertEquals; + +import com.google.common.collect.Lists; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Objects; +import org.apache.beam.sdk.coders.CannotProvideCoderException; +import org.apache.beam.sdk.coders.CoderRegistry; +import org.apache.beam.sdk.coders.NullableCoder; +import org.apache.beam.sdk.coders.VarLongCoder; +import org.apache.beam.sdk.values.TimestampedValue; +import org.joda.time.Instant; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.ExpectedException; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +/** + * Unit tests for {@link Latest.LatestFn}. + * */ +@RunWith(JUnit4.class) +public class LatestFnTests { + private static final Instant INSTANT = new Instant(100); + private static final long VALUE = 100 * INSTANT.getMillis(); + + private static final TimestampedValue TV = TimestampedValue.of(VALUE, INSTANT); + private static final TimestampedValue TV_MINUS_TEN = + TimestampedValue.of(VALUE - 10, INSTANT.minus(10)); + private static final TimestampedValue TV_PLUS_TEN = + TimestampedValue.of(VALUE + 10, INSTANT.plus(10)); + + @Rule + public final ExpectedException thrown = ExpectedException.none(); + + private final Latest.LatestFn fn = new Latest.LatestFn<>(); + private final Instant baseTimestamp = Instant.now(); + + @Test + public void testDefaultValue() { + assertThat(fn.defaultValue(), nullValue()); + } + + @Test + public void testCreateAccumulator() { + assertEquals(TimestampedValue.atMinimumTimestamp(null), fn.createAccumulator()); + } + + @Test + public void testAddInputInitialAdd() { + TimestampedValue input = TV; + assertEquals(input, fn.addInput(fn.createAccumulator(), input)); + } + + @Test + public void testAddInputMinTimestamp() { + TimestampedValue input = TimestampedValue.atMinimumTimestamp(1234L); + assertEquals(input, fn.addInput(fn.createAccumulator(), input)); + } + + @Test + public void testAddInputEarlierValue() { + assertEquals(TV, fn.addInput(TV, TV_MINUS_TEN)); + } + + @Test + public void testAddInputLaterValue() { + assertEquals(TV_PLUS_TEN, fn.addInput(TV, TV_PLUS_TEN)); + } + + @Test + public void testAddInputSameTimestamp() { + TimestampedValue accum = TimestampedValue.of(100L, INSTANT); + TimestampedValue input = TimestampedValue.of(200L, INSTANT); + + assertThat("Latest for values with the same timestamp is chosen arbitrarily", + fn.addInput(accum, input), isOneOf(accum, input)); + } + + @Test + public void testAddInputNullAccumulator() { + thrown.expect(NullPointerException.class); + thrown.expectMessage("accumulators"); + fn.addInput(null, TV); + } + + @Test + public void testAddInputNullInput() { + thrown.expect(NullPointerException.class); + thrown.expectMessage("input"); + fn.addInput(TV, null); + } + + @Test + public void testAddInputNullValue() { + TimestampedValue input = TimestampedValue.of(null, INSTANT.plus(10)); + assertEquals("Null values are allowed", input, fn.addInput(TV, input)); + } + + @Test + public void testMergeAccumulatorsMultipleValues() { + Iterable> accums = Lists.newArrayList( + TV, + TV_PLUS_TEN, + TV_MINUS_TEN + ); + + assertEquals(TV_PLUS_TEN, fn.mergeAccumulators(accums)); + } + + @Test + public void testMergeAccumulatorsSingleValue() { + assertEquals(TV, fn.mergeAccumulators(Lists.newArrayList(TV))); + } + + @Test + public void testMergeAccumulatorsEmptyIterable() { + ArrayList> emptyAccums = Lists.newArrayList(); + assertEquals(TimestampedValue.atMinimumTimestamp(null), fn.mergeAccumulators(emptyAccums)); + } + + @Test + public void testMergeAccumulatorsDefaultAccumulator() { + TimestampedValue defaultAccum = fn.createAccumulator(); + assertEquals(TV, fn.mergeAccumulators(Lists.newArrayList(TV, defaultAccum))); + } + + @Test + public void testMergeAccumulatorsAllDefaultAccumulators() { + TimestampedValue defaultAccum = fn.createAccumulator(); + assertEquals(defaultAccum, fn.mergeAccumulators( + Lists.newArrayList(defaultAccum, defaultAccum))); + } + + @Test + public void testMergeAccumulatorsNullIterable() { + thrown.expect(NullPointerException.class); + thrown.expectMessage("accumulators"); + fn.mergeAccumulators(null); + } + + @Test + public void testExtractOutput() { + assertEquals(TV.getValue(), fn.extractOutput(TV)); + } + + @Test + public void testExtractOutputDefaultAggregator() { + TimestampedValue accum = fn.createAccumulator(); + assertThat(fn.extractOutput(accum), nullValue()); + } + + @Test + public void testExtractOutputNullValue() { + TimestampedValue accum = TimestampedValue.of(null, baseTimestamp); + assertEquals(null, fn.extractOutput(accum)); + } + + @Test + public void testAggregator() throws Exception { + LatestAggregatorsFn doFn = new LatestAggregatorsFn<>(TV_MINUS_TEN.getValue()); + DoFnTester harness = DoFnTester.of(doFn); + for (TimestampedValue element : Arrays.asList(TV, TV_PLUS_TEN, TV_MINUS_TEN)) { + harness.processTimestampedElement(element); + } + + assertEquals(TV_PLUS_TEN.getValue(), harness.getAggregatorValue(doFn.allValuesAgg)); + assertEquals(TV_MINUS_TEN.getValue(), harness.getAggregatorValue(doFn.specialValueAgg)); + assertThat(harness.getAggregatorValue(doFn.noValuesAgg), nullValue()); + } + + @Test + public void testDefaultCoderHandlesNull() throws CannotProvideCoderException { + Latest.LatestFn fn = new Latest.LatestFn<>(); + + CoderRegistry registry = new CoderRegistry(); + TimestampedValue.TimestampedValueCoder inputCoder = + TimestampedValue.TimestampedValueCoder.of(VarLongCoder.of()); + + assertThat("Default output coder should handle null values", + fn.getDefaultOutputCoder(registry, inputCoder), instanceOf(NullableCoder.class)); + assertThat("Default accumulator coder should handle null values", + fn.getAccumulatorCoder(registry, inputCoder), instanceOf(NullableCoder.class)); + } + + static class LatestAggregatorsFn extends DoFn { + private final T specialValue; + LatestAggregatorsFn(T specialValue) { + this.specialValue = specialValue; + } + + Aggregator, T> allValuesAgg = + createAggregator("allValues", new Latest.LatestFn()); + + Aggregator, T> specialValueAgg = + createAggregator("oneValue", new Latest.LatestFn()); + + Aggregator, T> noValuesAgg = + createAggregator("noValues", new Latest.LatestFn()); + + @ProcessElement + public void processElement(ProcessContext c) { + TimestampedValue val = TimestampedValue.of(c.element(), c.timestamp()); + allValuesAgg.addValue(val); + if (Objects.equals(c.element(), specialValue)) { + specialValueAgg.addValue(val); + } + } + } +} 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 new file mode 100644 index 0000000000000..ce9ae376ad343 --- /dev/null +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/LatestTest.java @@ -0,0 +1,146 @@ +/* + * 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.transforms; + +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.instanceOf; +import static org.junit.Assert.assertEquals; + +import java.io.Serializable; +import java.util.concurrent.atomic.AtomicLong; + +import org.apache.beam.sdk.coders.AvroCoder; +import org.apache.beam.sdk.coders.BigEndianLongCoder; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.coders.KvCoder; +import org.apache.beam.sdk.coders.NullableCoder; +import org.apache.beam.sdk.coders.StringUtf8Coder; +import org.apache.beam.sdk.coders.VarLongCoder; +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.values.KV; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.TimestampedValue; + +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; + +/** + * Unit tests for {@link Latest} {@link PTransform} and {@link Combine.CombineFn}. + */ +@RunWith(JUnit4.class) +public class LatestTest implements Serializable { + @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)), + TimestampedValue.of("bar", new Instant(300)), + TimestampedValue.of("baz", new Instant(200)) + )) + .apply(Latest.globally()); + + PAssert.that(output).containsInAnyOrder("bar"); + p.run(); + } + + @Test + public void testGloballyOutputCoder() { + TestPipeline p = TestPipeline.create(); + BigEndianLongCoder inputCoder = BigEndianLongCoder.of(); + + PCollection output = + p.apply(Create.of(1L, 2L).withCoder(inputCoder)) + .apply(Latest.globally()); + + Coder outputCoder = output.getCoder(); + assertThat(outputCoder, instanceOf(NullableCoder.class)); + assertEquals(inputCoder, ((NullableCoder) outputCoder).getValueCoder()); + } + + @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())); + PCollection output = emptyInput.apply(Latest.globally()); + + PAssert.that(output).containsInAnyOrder((Long) null); + p.run(); + } + + @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)), + TimestampedValue.of(KV.of("B", "bar"), new Instant(300)), + TimestampedValue.of(KV.of("A", "baz"), new Instant(200)) + )) + .apply(Latest.perKey()); + + PAssert.that(output).containsInAnyOrder(KV.of("B", "bar"), KV.of("A", "baz")); + p.run(); + } + + @Test + public void testPerKeyOutputCoder() { + TestPipeline p = TestPipeline.create(); + KvCoder inputCoder = KvCoder.of( + AvroCoder.of(String.class), AvroCoder.of(Long.class)); + + PCollection> output = + p.apply(Create.of(KV.of("foo", 1L)).withCoder(inputCoder)) + .apply(Latest.perKey()); + + assertEquals("Should use input coder for outputs", inputCoder, output.getCoder()); + } + + @Test + @Category(NeedsRunner.class) + public void testPerKeyEmptyCollection() { + TestPipeline p = TestPipeline.create(); + PCollection> output = + p.apply(Create.>of().withCoder(KvCoder.of( + StringUtf8Coder.of(), StringUtf8Coder.of()))) + .apply(Latest.perKey()); + + PAssert.that(output).empty(); + p.run(); + } + + /** Helper method to easily create a timestamped value. */ + private static TimestampedValue timestamped(Instant timestamp) { + return TimestampedValue.of(uniqueLong.incrementAndGet(), timestamp); + } + private static final AtomicLong uniqueLong = new AtomicLong(); +} diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/values/TimestampedValueTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/values/TimestampedValueTest.java new file mode 100644 index 0000000000000..a982f3139ae9d --- /dev/null +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/values/TimestampedValueTest.java @@ -0,0 +1,83 @@ +/* + * 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 static org.junit.Assert.assertEquals; + +import com.google.common.testing.EqualsTester; + +import org.apache.beam.sdk.transforms.windowing.BoundedWindow; + +import org.joda.time.Instant; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.ExpectedException; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +/** + * Unit tests for {@link TimestampedValue}. + */ +@RunWith(JUnit4.class) +public class TimestampedValueTest { + @Rule + public ExpectedException thrown = ExpectedException.none(); + + @Test + public void testValues() { + Instant now = Instant.now(); + TimestampedValue tsv = TimestampedValue.of("foobar", now); + + assertEquals(now, tsv.getTimestamp()); + assertEquals("foobar", tsv.getValue()); + } + + @Test + public void testAtMinimumTimestamp() { + TimestampedValue tsv = TimestampedValue.atMinimumTimestamp("foobar"); + assertEquals(BoundedWindow.TIMESTAMP_MIN_VALUE, tsv.getTimestamp()); + } + + @Test + public void testNullTimestamp() { + thrown.expect(NullPointerException.class); + thrown.expectMessage("timestamp"); + TimestampedValue.of("foobar", null); + } + + @Test + public void testNullValue() { + TimestampedValue tsv = TimestampedValue.atMinimumTimestamp(null); + assertEquals(null, tsv.getValue()); + } + + @Test + public void testEquality() { + new EqualsTester() + .addEqualityGroup( + TimestampedValue.of("foo", new Instant(1000)), + TimestampedValue.of("foo", new Instant(1000))) + .addEqualityGroup(TimestampedValue.of("foo", new Instant(2000))) + .addEqualityGroup(TimestampedValue.of("bar", new Instant(1000))) + .addEqualityGroup( + TimestampedValue.of("foo", BoundedWindow.TIMESTAMP_MIN_VALUE), + TimestampedValue.atMinimumTimestamp("foo")) + .testEquals(); + } +} From 0312f15e95898f2fbd4dd4a4accfa9529f5efeee Mon Sep 17 00:00:00 2001 From: Vikas Kedigehalli Date: Mon, 29 Aug 2016 13:55:32 -0700 Subject: [PATCH 098/346] DatastoreIO SplitQueryFn integration test --- .../sdk/io/gcp/datastore/DatastoreV1.java | 9 +- .../sdk/io/gcp/datastore/DatastoreV1Test.java | 6 +- .../sdk/io/gcp/datastore/SplitQueryFnIT.java | 97 +++++++++++++++++++ 3 files changed, 107 insertions(+), 5 deletions(-) create mode 100644 sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/SplitQueryFnIT.java 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 8456e0287def3..e24bc809c1eb7 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 @@ -24,6 +24,7 @@ import static com.google.datastore.v1.PropertyFilter.Operator.EQUAL; import static com.google.datastore.v1.PropertyOrder.Direction.DESCENDING; import static com.google.datastore.v1.QueryResultBatch.MoreResultsType.NOT_FINISHED; +import static com.google.datastore.v1.client.DatastoreHelper.makeAndFilter; import static com.google.datastore.v1.client.DatastoreHelper.makeDelete; import static com.google.datastore.v1.client.DatastoreHelper.makeFilter; import static com.google.datastore.v1.client.DatastoreHelper.makeOrder; @@ -290,7 +291,7 @@ static long getEstimatedSizeBytes(Datastore datastore, Query query, @Nullable St throws DatastoreException { String ourKind = query.getKind(0).getName(); long latestTimestamp = queryLatestStatisticsTimestamp(datastore, namespace); - LOG.info("Latest stats timestamp : {}", latestTimestamp); + LOG.info("Latest stats timestamp for kind {} is {}", ourKind, latestTimestamp); Query.Builder queryBuilder = Query.newBuilder(); if (namespace == null) { @@ -298,8 +299,10 @@ static long getEstimatedSizeBytes(Datastore datastore, Query query, @Nullable St } else { queryBuilder.addKindBuilder().setName("__Stat_Ns_Kind__"); } - queryBuilder.setFilter(makeFilter("kind_name", EQUAL, makeValue(ourKind).build())); - queryBuilder.setFilter(makeFilter("timestamp", EQUAL, makeValue(latestTimestamp).build())); + + queryBuilder.setFilter(makeAndFilter( + makeFilter("kind_name", EQUAL, makeValue(ourKind).build()).build(), + makeFilter("timestamp", EQUAL, makeValue(latestTimestamp).build()).build())); RunQueryRequest request = makeRequest(queryBuilder.build(), namespace); diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/DatastoreV1Test.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/DatastoreV1Test.java index 138671d4a0078..d96c320dd94cb 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/DatastoreV1Test.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/DatastoreV1Test.java @@ -19,6 +19,7 @@ import static com.google.datastore.v1.PropertyFilter.Operator.EQUAL; import static com.google.datastore.v1.PropertyOrder.Direction.DESCENDING; +import static com.google.datastore.v1.client.DatastoreHelper.makeAndFilter; import static com.google.datastore.v1.client.DatastoreHelper.makeDelete; import static com.google.datastore.v1.client.DatastoreHelper.makeFilter; import static com.google.datastore.v1.client.DatastoreHelper.makeKey; @@ -805,8 +806,9 @@ private static Query makeStatKindQuery(String namespace, long timestamp) { } else { statQuery.addKindBuilder().setName("__Stat_Ns_Kind__"); } - statQuery.setFilter(makeFilter("kind_name", EQUAL, makeValue(KIND)).build()); - statQuery.setFilter(makeFilter("timestamp", EQUAL, makeValue(timestamp * 1000000L)).build()); + statQuery.setFilter(makeAndFilter( + makeFilter("kind_name", EQUAL, makeValue(KIND).build()).build(), + makeFilter("timestamp", EQUAL, makeValue(timestamp * 1000000L).build()).build())); return statQuery.build(); } diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/SplitQueryFnIT.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/SplitQueryFnIT.java new file mode 100644 index 0000000000000..72ab7c2535e4d --- /dev/null +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/SplitQueryFnIT.java @@ -0,0 +1,97 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.beam.sdk.io.gcp.datastore; + +import static org.apache.beam.sdk.io.gcp.datastore.DatastoreV1.Read.NUM_QUERY_SPLITS_MIN; +import static org.junit.Assert.assertEquals; + +import com.google.datastore.v1.Query; +import java.util.List; +import javax.annotation.Nullable; +import org.apache.beam.sdk.io.gcp.datastore.DatastoreV1.Read.SplitQueryFn; +import org.apache.beam.sdk.io.gcp.datastore.DatastoreV1.Read.V1Options; +import org.apache.beam.sdk.transforms.DoFnTester; +import org.apache.beam.sdk.values.KV; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +/** + * Integration tests for {@link DatastoreV1.Read.SplitQueryFn}. + * + *

              It is hard to mock the exact behavior of Cloud Datastore, especially for the statistics + * queries. Also the fact that DatastoreIO falls back gracefully when querying statistics fails, + * makes it hard to catch these issues in production. This test here ensures we interact with + * the Cloud Datastore directly, query the actual stats and verify that the SplitQueryFn generates + * the expected number of query splits. + * + *

              These tests are brittle as they rely on statistics data in Cloud Datastore. If the data + * gets lost or changes then they will begin failing and this test should be disabled. + * At the time of writing, the Cloud Datastore has the following statistics, + *

                + *
              • kind = sort_1G, entity_bytes = 2130000000, count = 10000000 + *
              • kind = shakespeare, entity_bytes = 26383451, count = 172948 + *
              + */ +@RunWith(JUnit4.class) +public class SplitQueryFnIT { + /** + * Tests {@link SplitQueryFn} to generate expected number of splits for a large dataset. + */ + @Test + public void testSplitQueryFnWithLargeDataset() throws Exception { + String projectId = "apache-beam-testing"; + String kind = "sort_1G"; + String namespace = null; + // Num splits is computed based on the entity_bytes size of the input_sort_1G kind reported by + // Datastore stats. + int expectedNumSplits = 32; + testSplitQueryFn(projectId, kind, namespace, expectedNumSplits); + } + + /** + * Tests {@link SplitQueryFn} to fallback to NUM_QUERY_SPLITS_MIN for a small dataset. + */ + @Test + public void testSplitQueryFnWithSmallDataset() throws Exception { + String projectId = "apache-beam-testing"; + String kind = "shakespeare"; + String namespace = null; + int expectedNumSplits = NUM_QUERY_SPLITS_MIN; + testSplitQueryFn(projectId, kind, namespace, expectedNumSplits); + } + + /** + * A helper method to test {@link SplitQueryFn} to generate the expected number of splits. + */ + private void testSplitQueryFn(String projectId, String kind, @Nullable String namespace, + int expectedNumSplits) throws Exception { + Query.Builder query = Query.newBuilder(); + query.addKindBuilder().setName(kind); + + SplitQueryFn splitQueryFn = new SplitQueryFn( + V1Options.from(projectId, query.build(), namespace), 0); + DoFnTester> doFnTester = DoFnTester.of(splitQueryFn); + + List> queries = doFnTester.processBundle(query.build()); + assertEquals(queries.size(), expectedNumSplits); + } + + // TODO (vikasrk): Create datasets under a different namespace and add tests. +} From f44fa2cb78e9531399edc71ae1387c6c9038e25c Mon Sep 17 00:00:00 2001 From: Vikas Kedigehalli Date: Thu, 1 Sep 2016 13:31:35 -0700 Subject: [PATCH 099/346] Cloud Datastore naming clean-up --- .../beam/examples/complete/AutoComplete.java | 12 ++--- .../examples/cookbook/DatastoreWordCount.java | 18 +++---- .../sdk/io/gcp/datastore/DatastoreIO.java | 6 +-- .../sdk/io/gcp/datastore/DatastoreV1.java | 52 +++++++++---------- .../sdk/io/gcp/datastore/package-info.java | 2 +- .../sdk/io/gcp/datastore/DatastoreV1Test.java | 6 +-- 6 files changed, 48 insertions(+), 48 deletions(-) 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 56c7855e45e09..2182e6dd0ea53 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 @@ -96,7 +96,7 @@ * --streaming * } * - *

              This will update the datastore every 10 seconds based on the last + *

              This will update the Cloud Datastore every 10 seconds based on the last * 30 minutes of data received. */ public class AutoComplete { @@ -380,7 +380,7 @@ static TableSchema getSchema() { /** * Takes as input a the top candidates per prefix, and emits an entity - * suitable for writing to Datastore. + * suitable for writing to Cloud Datastore. * *

              Note: We use ancestor keys for strong consistency. See the Cloud Datastore documentation on * @@ -431,7 +431,7 @@ private static interface Options Boolean getRecursive(); void setRecursive(Boolean value); - @Description("Datastore entity kind") + @Description("Cloud Datastore entity kind") @Default.String("autocomplete-demo") String getKind(); void setKind(String value); @@ -441,17 +441,17 @@ private static interface Options Boolean getOutputToBigQuery(); void setOutputToBigQuery(Boolean value); - @Description("Whether output to Datastore") + @Description("Whether output to Cloud Datastore") @Default.Boolean(false) Boolean getOutputToDatastore(); void setOutputToDatastore(Boolean value); - @Description("Datastore ancestor key") + @Description("Cloud Datastore ancestor key") @Default.String("root") String getDatastoreAncestorKey(); void setDatastoreAncestorKey(String value); - @Description("Datastore output project ID, defaults to project ID") + @Description("Cloud Datastore output project ID, defaults to project ID") String getOutputProject(); void setOutputProject(String value); } diff --git a/examples/java/src/main/java/org/apache/beam/examples/cookbook/DatastoreWordCount.java b/examples/java/src/main/java/org/apache/beam/examples/cookbook/DatastoreWordCount.java index eb2165f91e19b..434e9fb94b266 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/cookbook/DatastoreWordCount.java +++ b/examples/java/src/main/java/org/apache/beam/examples/cookbook/DatastoreWordCount.java @@ -47,11 +47,11 @@ /** * A WordCount example using DatastoreIO. * - *

              This example shows how to use DatastoreIO to read from Datastore and + *

              This example shows how to use DatastoreIO to read from Cloud Datastore and * write the results to Cloud Storage. Note that this example will write - * data to Datastore, which may incur charge for Datastore operations. + * data to Cloud Datastore, which may incur charge for Cloud Datastore operations. * - *

              To run this example, users need to use gcloud to get credential for Datastore: + *

              To run this example, users need to use gcloud to get credential for Cloud Datastore: *

              {@code
                * $ gcloud auth login
                * }
              @@ -150,7 +150,7 @@ public void processElement(ProcessContext c) { *

              Inherits standard configuration options. */ public static interface Options extends PipelineOptions { - @Description("Path of the file to read from and store to Datastore") + @Description("Path of the file to read from and store to Cloud Datastore") @Default.String("gs://apache-beam-samples/shakespeare/kinglear.txt") String getInput(); void setInput(String value); @@ -160,17 +160,17 @@ public static interface Options extends PipelineOptions { String getOutput(); void setOutput(String value); - @Description("Project ID to read from datastore") + @Description("Project ID to read from Cloud Datastore") @Validation.Required String getProject(); void setProject(String value); - @Description("Datastore Entity kind") + @Description("Cloud Datastore Entity kind") @Default.String("shakespeare-demo") String getKind(); void setKind(String value); - @Description("Datastore Namespace") + @Description("Cloud Datastore Namespace") String getNamespace(); void setNamespace(@Nullable String value); @@ -186,7 +186,7 @@ public static interface Options extends PipelineOptions { /** * An example that creates a pipeline to populate DatastoreIO from a - * text input. Forces use of DirectRunner for local execution mode. + * text input. Forces use of DirectRunner for local execution mode. */ public static void writeDataToDatastore(Options options) { Pipeline p = Pipeline.create(options); @@ -217,7 +217,7 @@ static Query makeAncestorKindQuery(Options options) { } /** - * An example that creates a pipeline to do DatastoreIO.Read from Datastore. + * An example that creates a pipeline to do DatastoreIO.Read from Cloud Datastore. */ public static void readDataFromDatastore(Options options) { Query query = makeAncestorKindQuery(options); diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/datastore/DatastoreIO.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/datastore/DatastoreIO.java index 5abf0153ac10a..c50c23a070daf 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/datastore/DatastoreIO.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/datastore/DatastoreIO.java @@ -22,7 +22,7 @@ /** *

              {@link DatastoreIO} provides an API for reading from and writing to * Google Cloud Datastore over different - * versions of the Datastore Client libraries. + * versions of the Cloud Datastore Client libraries. * *

              To use the v1 version see {@link DatastoreV1}. */ @@ -32,8 +32,8 @@ public class DatastoreIO { private DatastoreIO() {} /** - * Returns a {@link DatastoreV1} that provides an API for accessing Datastore through v1 version - * of Datastore Client library. + * Returns a {@link DatastoreV1} that provides an API for accessing Cloud Datastore through v1 + * version of Datastore Client library. */ public static DatastoreV1 v1() { return new DatastoreV1(); 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 e24bc809c1eb7..6bd03b5082688 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 @@ -91,13 +91,13 @@ * {@link Entity} objects. * *

              This API currently requires an authentication workaround. To use {@link DatastoreV1}, users - * must use the {@code gcloud} command line tool to get credentials for Datastore: + * must use the {@code gcloud} command line tool to get credentials for Cloud Datastore: *

                * $ gcloud auth login
                * 
              * - *

              To read a {@link PCollection} from a query to Datastore, use {@link DatastoreV1#read} and - * its methods {@link DatastoreV1.Read#withProjectId} and {@link DatastoreV1.Read#withQuery} to + *

              To read a {@link PCollection} from a query to Cloud Datastore, use {@link DatastoreV1#read} + * and its methods {@link DatastoreV1.Read#withProjectId} and {@link DatastoreV1.Read#withQuery} to * specify the project to query and the query to read from. You can optionally provide a namespace * to query within using {@link DatastoreV1.Read#withNamespace}. You could also optionally specify * how many splits you want for the query using {@link DatastoreV1.Read#withNumQuerySplits}. @@ -122,7 +122,7 @@ * {@link com.google.datastore.v1.Query.Builder#setLimit(Int32Value)}, then * all returned results will be read by a single Dataflow worker in order to ensure correct data. * - *

              To write a {@link PCollection} to a Datastore, use {@link DatastoreV1#write}, + *

              To write a {@link PCollection} to a Cloud Datastore, use {@link DatastoreV1#write}, * specifying the Cloud Datastore project to write to: * *

               {@code
              @@ -131,7 +131,7 @@
                * p.run();
                * } 
              * - *

              To delete a {@link PCollection} of {@link Entity Entities} from Datastore, use + *

              To delete a {@link PCollection} of {@link Entity Entities} from Cloud Datastore, use * {@link DatastoreV1#deleteEntity()}, specifying the Cloud Datastore project to write to: * *

               {@code
              @@ -140,8 +140,8 @@
                * p.run();
                * } 
              * - *

              To delete entities associated with a {@link PCollection} of {@link Key Keys} from Datastore, - * use {@link DatastoreV1#deleteKey}, specifying the Cloud Datastore project to write to: + *

              To delete entities associated with a {@link PCollection} of {@link Key Keys} from Cloud + * Datastore, use {@link DatastoreV1#deleteKey}, specifying the Cloud Datastore project to write to: * *

               {@code
                * PCollection entities = ...;
              @@ -169,7 +169,7 @@
                * more details.
                *
                * 

              Please see Cloud Datastore Sign Up - * for security and permission related information specific to Datastore. + * for security and permission related information specific to Cloud Datastore. * * @see org.apache.beam.sdk.runners.PipelineRunner */ @@ -180,7 +180,7 @@ public class DatastoreV1 { DatastoreV1() {} /** - * Datastore has a limit of 500 mutations per batch operation, so we flush + * Cloud Datastore has a limit of 500 mutations per batch operation, so we flush * changes to Datastore every 500 entities. */ @VisibleForTesting @@ -197,7 +197,7 @@ public DatastoreV1.Read read() { } /** - * A {@link PTransform} that reads the result rows of a Datastore query as {@code Entity} + * A {@link PTransform} that reads the result rows of a Cloud Datastore query as {@code Entity} * objects. * * @see DatastoreIO @@ -234,7 +234,7 @@ public static class Read extends PTransform> { /** * Computes the number of splits to be performed on the given query by querying the estimated - * size from Datastore. + * size from Cloud Datastore. */ static int getEstimatedNumSplits(Datastore datastore, Query query, @Nullable String namespace) { int numSplits; @@ -252,7 +252,7 @@ static int getEstimatedNumSplits(Datastore datastore, Query query, @Nullable Str } /** - * Datastore system tables with statistics are periodically updated. This method fetches + * Cloud Datastore system tables with statistics are periodically updated. This method fetches * the latest timestamp (in microseconds) of statistics update using the {@code __Stat_Total__} * table. */ @@ -281,7 +281,7 @@ private static long queryLatestStatisticsTimestamp(Datastore datastore, /** * Get the estimated size of the data returned by the given query. * - *

              Datastore provides no way to get a good estimate of how large the result of a query + *

              Cloud Datastore provides no way to get a good estimate of how large the result of a query * entity kind being queried, using the __Stat_Kind__ system table, assuming exactly 1 kind * is specified in the query. * @@ -357,7 +357,7 @@ private Read(@Nullable String projectId, @Nullable Query query, @Nullable String } /** - * Returns a new {@link DatastoreV1.Read} that reads from the Datastore for the specified + * Returns a new {@link DatastoreV1.Read} that reads from the Cloud Datastore for the specified * project. */ public DatastoreV1.Read withProjectId(String projectId) { @@ -400,7 +400,7 @@ public DatastoreV1.Read withNamespace(String namespace) { *

            3. If the {@code query} has a user limit set, then {@code numQuerySplits} will be * ignored and no split will be performed. *
            4. Under certain cases Cloud Datastore is unable to split query to the requested number of - * splits. In such cases we just use whatever the Datastore returns. + * splits. In such cases we just use whatever the Cloud Datastore returns. * */ public DatastoreV1.Read withNumQuerySplits(int numQuerySplits) { @@ -492,7 +492,7 @@ public String toString() { } /** - * A class for v1 Datastore related options. + * A class for v1 Cloud Datastore related options. */ @VisibleForTesting static class V1Options implements Serializable { @@ -608,7 +608,7 @@ public void populateDisplayData(Builder builder) { } /** - * A {@link DoFn} that reads entities from Datastore for each query. + * A {@link DoFn} that reads entities from Cloud Datastore for each query. */ @VisibleForTesting static class ReadFn extends DoFn { @@ -908,8 +908,8 @@ public void finishBundle(Context c) throws Exception { * *

              If a commit fails, it will be retried (up to {@link DatastoreWriterFn#MAX_RETRIES} * times). All mutations in the batch will be committed again, even if the commit was partially - * successful. If the retry limit is exceeded, the last exception from the Datastore will be - * thrown. + * successful. If the retry limit is exceeded, the last exception from the Cloud Datastore will + * be thrown. * * @throws DatastoreException if the commit fails or IOException or InterruptedException if * backing off between retries fails. @@ -953,7 +953,7 @@ public void populateDisplayData(Builder builder) { } /** - * Returns true if a Datastore key is complete. A key is complete if its last element + * Returns true if a Cloud Datastore key is complete. A key is complete if its last element * has either an id or a name. */ static boolean isValidKey(Key key) { @@ -974,7 +974,7 @@ static class UpsertFn extends SimpleFunction { public Mutation apply(Entity entity) { // Verify that the entity to write has a complete key. checkArgument(isValidKey(entity.getKey()), - "Entities to be written to the Datastore must have complete keys:\n%s", entity); + "Entities to be written to the Cloud Datastore must have complete keys:\n%s", entity); return makeUpsert(entity).build(); } @@ -995,7 +995,7 @@ static class DeleteEntityFn extends SimpleFunction { public Mutation apply(Entity entity) { // Verify that the entity to delete has a complete key. checkArgument(isValidKey(entity.getKey()), - "Entities to be deleted from the Datastore must have complete keys:\n%s", entity); + "Entities to be deleted from the Cloud Datastore must have complete keys:\n%s", entity); return makeDelete(entity.getKey()).build(); } @@ -1016,7 +1016,7 @@ static class DeleteKeyFn extends SimpleFunction { public Mutation apply(Key key) { // Verify that the entity to delete has a complete key. checkArgument(isValidKey(key), - "Keys to be deleted from the Datastore must be complete:\n%s", key); + "Keys to be deleted from the Cloud Datastore must be complete:\n%s", key); return makeDelete(key).build(); } @@ -1029,7 +1029,7 @@ public void populateDisplayData(Builder builder) { } /** - * A wrapper factory class for Datastore singleton classes {@link DatastoreFactory} and + * A wrapper factory class for Cloud Datastore singleton classes {@link DatastoreFactory} and * {@link QuerySplitter} * *

              {@link DatastoreFactory} and {@link QuerySplitter} are not java serializable, hence @@ -1038,7 +1038,7 @@ public void populateDisplayData(Builder builder) { @VisibleForTesting static class V1DatastoreFactory implements Serializable { - /** Builds a Datastore client for the given pipeline options and project. */ + /** Builds a Cloud Datastore client for the given pipeline options and project. */ public Datastore getDatastore(PipelineOptions pipelineOptions, String projectId) { DatastoreOptions.Builder builder = new DatastoreOptions.Builder() @@ -1055,7 +1055,7 @@ public Datastore getDatastore(PipelineOptions pipelineOptions, String projectId) return DatastoreFactory.get().create(builder.build()); } - /** Builds a Datastore {@link QuerySplitter}. */ + /** Builds a Cloud Datastore {@link QuerySplitter}. */ public QuerySplitter getQuerySplitter() { return DatastoreHelper.getQuerySplitter(); } diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/datastore/package-info.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/datastore/package-info.java index 1ca02668f3b40..4e5fbc30898c7 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/datastore/package-info.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/datastore/package-info.java @@ -19,6 +19,6 @@ /** *

              Provides an API for reading from and writing to * Google Cloud Datastore over different - * versions of the Datastore Client libraries. + * versions of the Cloud Datastore Client libraries. */ package org.apache.beam.sdk.io.gcp.datastore; diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/DatastoreV1Test.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/DatastoreV1Test.java index d96c320dd94cb..dd1904ae48a7c 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/DatastoreV1Test.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/DatastoreV1Test.java @@ -430,7 +430,7 @@ public void testAddEntitiesWithIncompleteKeys() throws Exception { UpsertFn upsertFn = new UpsertFn(); thrown.expect(IllegalArgumentException.class); - thrown.expectMessage("Entities to be written to the Datastore must have complete keys"); + thrown.expectMessage("Entities to be written to the Cloud Datastore must have complete keys"); upsertFn.apply(entity); } @@ -458,7 +458,7 @@ public void testDeleteEntitiesWithIncompleteKeys() throws Exception { DeleteEntityFn deleteEntityFn = new DeleteEntityFn(); thrown.expect(IllegalArgumentException.class); - thrown.expectMessage("Entities to be deleted from the Datastore must have complete keys"); + thrown.expectMessage("Entities to be deleted from the Cloud Datastore must have complete keys"); deleteEntityFn.apply(entity); } @@ -485,7 +485,7 @@ public void testDeleteIncompleteKeys() throws Exception { DeleteKeyFn deleteKeyFn = new DeleteKeyFn(); thrown.expect(IllegalArgumentException.class); - thrown.expectMessage("Keys to be deleted from the Datastore must be complete"); + thrown.expectMessage("Keys to be deleted from the Cloud Datastore must be complete"); deleteKeyFn.apply(key); } From 9943fd7d47819d522cef248d23c8db8f42981ad3 Mon Sep 17 00:00:00 2001 From: Ian Zhou Date: Thu, 18 Aug 2016 13:50:52 -0700 Subject: [PATCH 100/346] Fixed Combine display data --- .../apache/beam/sdk/transforms/Combine.java | 53 +++++++++++++++++-- .../beam/sdk/transforms/CombineTest.java | 19 +++++++ 2 files changed, 68 insertions(+), 4 deletions(-) diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Combine.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Combine.java index 26f0f660f0743..d432e15dc71e3 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 @@ -51,6 +51,7 @@ import org.apache.beam.sdk.transforms.CombineWithContext.KeyedCombineFnWithContext; import org.apache.beam.sdk.transforms.CombineWithContext.RequiresContextInternal; 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; import org.apache.beam.sdk.transforms.windowing.GlobalWindow; import org.apache.beam.sdk.transforms.windowing.GlobalWindows; @@ -1815,7 +1816,14 @@ public PerKeyWithHotKeyFanout withHotKeyFanout( */ public PerKeyWithHotKeyFanout withHotKeyFanout(final int hotKeyFanout) { return new PerKeyWithHotKeyFanout<>(name, fn, fnDisplayData, - new SerializableFunction() { + new SimpleFunction() { + @Override + public void populateDisplayData(Builder builder) { + super.populateDisplayData(builder); + builder.addIfNotDefault(DisplayData.item("fanout", hotKeyFanout) + .withLabel("Key Fanout Size"), 0); + } + @Override public Integer apply(K unused) { return hotKeyFanout; @@ -1904,7 +1912,7 @@ private PCollection> applyHelper(PCollection( inputCoder.getValueCoder(), accumCoder); - // A CombineFn's mergeAccumulator can be applied in a tree-like fashon. + // A CombineFn's mergeAccumulator can be applied in a tree-like fashion. // Here we shard the key using an integer nonce, combine on that partial // set of values, then drop the nonce and do a final combine of the // aggregates. We do this by splitting the original CombineFn into two, @@ -1944,6 +1952,16 @@ public Coder getAccumulatorCoder( throws CannotProvideCoderException { return accumCoder; } + + @Override + public void populateDisplayData(DisplayData.Builder builder) { + super.populateDisplayData(builder); + builder.add(DisplayData.item("fanoutFn", hotKeyFanout.getClass()) + .withLabel("Fanout Function")); + if (hotKeyFanout instanceof HasDisplayData) { + ((HasDisplayData) hotKeyFanout).populateDisplayData(builder); + } + } }; postCombine = new KeyedCombineFn, AccumT, OutputT>() { @@ -1988,6 +2006,15 @@ public Coder getAccumulatorCoder(CoderRegistry registry, Coder keyCod throws CannotProvideCoderException { return accumCoder; } + @Override + public void populateDisplayData(DisplayData.Builder builder) { + super.populateDisplayData(builder); + builder.add(DisplayData.item("fanoutFn", hotKeyFanout.getClass()) + .withLabel("Fanout Function")); + if (hotKeyFanout instanceof HasDisplayData) { + ((HasDisplayData) hotKeyFanout).populateDisplayData(builder); + } + } }; } else { final KeyedCombineFnWithContext keyedFnWithContext = @@ -2028,6 +2055,15 @@ public Coder getAccumulatorCoder( throws CannotProvideCoderException { return accumCoder; } + @Override + public void populateDisplayData(DisplayData.Builder builder) { + super.populateDisplayData(builder); + builder.add(DisplayData.item("fanoutFn", hotKeyFanout.getClass()) + .withLabel("Fanout Function")); + if (hotKeyFanout instanceof HasDisplayData) { + ((HasDisplayData) hotKeyFanout).populateDisplayData(builder); + } + } }; postCombine = new KeyedCombineFnWithContext, AccumT, OutputT>() { @@ -2073,6 +2109,15 @@ public Coder getAccumulatorCoder(CoderRegistry registry, Coder keyCod throws CannotProvideCoderException { return accumCoder; } + @Override + public void populateDisplayData(DisplayData.Builder builder) { + super.populateDisplayData(builder); + builder.add(DisplayData.item("fanoutFn", hotKeyFanout.getClass()) + .withLabel("Fanout Function")); + if (hotKeyFanout instanceof HasDisplayData) { + ((HasDisplayData) hotKeyFanout).populateDisplayData(builder); + } + } }; } @@ -2117,7 +2162,7 @@ public void processElement(ProcessContext c) { .setCoder(KvCoder.of(KvCoder.of(inputCoder.getKeyCoder(), VarIntCoder.of()), inputCoder.getValueCoder())) .setWindowingStrategyInternal(preCombineStrategy) - .apply("PreCombineHot", Combine.perKey(hotPreCombine)) + .apply("PreCombineHot", Combine.perKey(hotPreCombine, fnDisplayData)) .apply("StripNonce", MapElements.via( new SimpleFunction, AccumT>, KV>>() { @@ -2147,7 +2192,7 @@ public KV> apply(KV element) { // Combine the union of the pre-processed hot and cold key results. return PCollectionList.of(precombinedHot).and(preprocessedCold) .apply(Flatten.>>pCollections()) - .apply("PostCombine", Combine.perKey(postCombine)); + .apply("PostCombine", Combine.perKey(postCombine, fnDisplayData)); } @Override 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 77a1d6b03e69c..be061af51c3b0 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 @@ -731,6 +731,25 @@ public void testCombinePerKeyPrimitiveDisplayData() { displayData, hasItem(hasDisplayItem("combineFn", combineFn.getClass()))); } + @Test + @Category(RunnableOnService.class) + public void testCombinePerKeyWithHotKeyFanoutPrimitiveDisplayData() { + int hotKeyFanout = 2; + DisplayDataEvaluator evaluator = DisplayDataEvaluator.create(); + + CombineTest.UniqueInts combineFn = new CombineTest.UniqueInts(); + PTransform>, PCollection>>> combine = + Combine.>perKey(combineFn).withHotKeyFanout(hotKeyFanout); + + Set displayData = evaluator.displayDataForPrimitiveTransforms(combine, + KvCoder.of(VarIntCoder.of(), VarIntCoder.of())); + + assertThat("Combine.perKey.withHotKeyFanout should include the combineFn in its primitive " + + "transform", displayData, hasItem(hasDisplayItem("combineFn", combineFn.getClass()))); + assertThat("Combine.perKey.withHotKeyFanout(int) should include the fanout in its primitive " + + "transform", displayData, hasItem(hasDisplayItem("fanout", hotKeyFanout))); + } + //////////////////////////////////////////////////////////////////////////// // Test classes, for different kinds of combining fns. From 60d8cd90ab9f0947baaf7b50168f8a7b570215f8 Mon Sep 17 00:00:00 2001 From: Scott Wegner Date: Fri, 19 Aug 2016 13:32:45 -0700 Subject: [PATCH 101/346] Delegate populateDipslayData to wrapped combineFn's --- .../apache/beam/sdk/transforms/Combine.java | 39 ++++++------------- 1 file changed, 11 insertions(+), 28 deletions(-) diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Combine.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Combine.java index d432e15dc71e3..2b89372a47a84 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 @@ -1740,7 +1740,7 @@ protected SimpleCombineFn(SerializableFunction, V> combiner) { public static class PerKey extends PTransform>, PCollection>> { - private final transient PerKeyCombineFn fn; + private final PerKeyCombineFn fn; private final DisplayData.Item> fnDisplayData; private final boolean fewKeys; private final List> sideInputs; @@ -1820,8 +1820,8 @@ public PerKeyWithHotKeyFanout withHotKeyFanout(final int hot @Override public void populateDisplayData(Builder builder) { super.populateDisplayData(builder); - builder.addIfNotDefault(DisplayData.item("fanout", hotKeyFanout) - .withLabel("Key Fanout Size"), 0); + builder.add(DisplayData.item("fanout", hotKeyFanout) + .withLabel("Key Fanout Size")); } @Override @@ -1866,7 +1866,7 @@ public void populateDisplayData(DisplayData.Builder builder) { public static class PerKeyWithHotKeyFanout extends PTransform>, PCollection>> { - private final transient PerKeyCombineFn fn; + private final PerKeyCombineFn fn; private final DisplayData.Item> fnDisplayData; private final SerializableFunction hotKeyFanout; @@ -1955,12 +1955,7 @@ public Coder getAccumulatorCoder( @Override public void populateDisplayData(DisplayData.Builder builder) { - super.populateDisplayData(builder); - builder.add(DisplayData.item("fanoutFn", hotKeyFanout.getClass()) - .withLabel("Fanout Function")); - if (hotKeyFanout instanceof HasDisplayData) { - ((HasDisplayData) hotKeyFanout).populateDisplayData(builder); - } + builder.include(PerKeyWithHotKeyFanout.this); } }; postCombine = @@ -2008,12 +2003,7 @@ public Coder getAccumulatorCoder(CoderRegistry registry, Coder keyCod } @Override public void populateDisplayData(DisplayData.Builder builder) { - super.populateDisplayData(builder); - builder.add(DisplayData.item("fanoutFn", hotKeyFanout.getClass()) - .withLabel("Fanout Function")); - if (hotKeyFanout instanceof HasDisplayData) { - ((HasDisplayData) hotKeyFanout).populateDisplayData(builder); - } + builder.include(PerKeyWithHotKeyFanout.this); } }; } else { @@ -2057,12 +2047,7 @@ public Coder getAccumulatorCoder( } @Override public void populateDisplayData(DisplayData.Builder builder) { - super.populateDisplayData(builder); - builder.add(DisplayData.item("fanoutFn", hotKeyFanout.getClass()) - .withLabel("Fanout Function")); - if (hotKeyFanout instanceof HasDisplayData) { - ((HasDisplayData) hotKeyFanout).populateDisplayData(builder); - } + builder.include(PerKeyWithHotKeyFanout.this); } }; postCombine = @@ -2111,12 +2096,7 @@ public Coder getAccumulatorCoder(CoderRegistry registry, Coder keyCod } @Override public void populateDisplayData(DisplayData.Builder builder) { - super.populateDisplayData(builder); - builder.add(DisplayData.item("fanoutFn", hotKeyFanout.getClass()) - .withLabel("Fanout Function")); - if (hotKeyFanout instanceof HasDisplayData) { - ((HasDisplayData) hotKeyFanout).populateDisplayData(builder); - } + builder.include(PerKeyWithHotKeyFanout.this); } }; } @@ -2200,6 +2180,9 @@ public void populateDisplayData(DisplayData.Builder builder) { super.populateDisplayData(builder); Combine.populateDisplayData(builder, fn, fnDisplayData); + if (hotKeyFanout instanceof HasDisplayData) { + builder.include((HasDisplayData) hotKeyFanout); + } builder.add(DisplayData.item("fanoutFn", hotKeyFanout.getClass()) .withLabel("Fanout Function")); } From 4bf3a3b345d94ecea4c77ebdfaed9dd7ef0f39e5 Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Thu, 25 Aug 2016 14:57:26 -0700 Subject: [PATCH 102/346] Put classes in runners-core package into runners.core namespace --- .../util => runners/core}/AssignWindows.java | 2 +- .../core}/AssignWindowsDoFn.java | 4 +++- .../core}/BatchTimerInternals.java | 5 ++++- .../util => runners/core}/DoFnRunner.java | 4 +++- .../util => runners/core}/DoFnRunnerBase.java | 12 +++++++++-- .../util => runners/core}/DoFnRunners.java | 9 +++++++-- .../core}/ElementByteSizeObservable.java | 4 +++- .../GroupAlsoByWindowViaWindowSetDoFn.java | 5 +---- .../core}/GroupAlsoByWindowsDoFn.java | 5 ++++- ...GroupAlsoByWindowsViaOutputBufferDoFn.java | 5 ++++- .../core}/GroupByKeyViaGroupByKeyOnly.java | 5 ++++- .../core}/LateDataDroppingDoFnRunner.java | 8 +++++++- .../util => runners/core}/NonEmptyPanes.java | 3 ++- .../core}/PaneInfoTracker.java | 4 +++- .../core}/PeekingReiterator.java | 3 ++- .../core}/PushbackSideInputDoFnRunner.java | 5 ++++- .../{sdk/util => runners/core}/ReduceFn.java | 4 +++- .../core}/ReduceFnContextFactory.java | 8 +++++++- .../util => runners/core}/ReduceFnRunner.java | 18 +++++++++++++---- .../core}/SimpleDoFnRunner.java | 9 ++++++--- .../util => runners/core}/SystemReduceFn.java | 3 ++- .../util => runners/core}/TriggerRunner.java | 8 +++++++- .../util => runners/core}/WatermarkHold.java | 5 ++++- .../beam/sdk/util/common/package-info.java | 20 ------------------- .../apache/beam/sdk/util/package-info.java | 20 ------------------- .../core}/BatchTimerInternalsTest.java | 3 ++- .../core}/GroupAlsoByWindowsProperties.java | 4 +++- ...pAlsoByWindowsViaOutputBufferDoFnTest.java | 7 ++++--- .../core}/LateDataDroppingDoFnRunnerTest.java | 7 +++++-- .../PushbackSideInputDoFnRunnerTest.java | 6 +++++- .../core}/ReduceFnRunnerTest.java | 6 +++++- .../util => runners/core}/ReduceFnTester.java | 14 ++++++++++++- .../core}/SimpleDoFnRunnerTest.java | 6 +++++- .../GroupAlsoByWindowEvaluatorFactory.java | 8 ++++---- .../GroupByKeyOnlyEvaluatorFactory.java | 4 ++-- .../beam/runners/direct/ParDoEvaluator.java | 8 ++++---- .../UncommittedBundleOutputManager.java | 2 +- .../FlinkStreamingTransformTranslators.java | 2 +- .../wrappers/streaming/DoFnOperator.java | 6 +++--- .../streaming/WindowDoFnOperator.java | 2 +- .../beam/runners/spark/SparkRunner.java | 2 +- .../translation/TransformTranslator.java | 10 +++++----- .../StreamingTransformTranslator.java | 2 +- .../main/resources/beam/findbugs-filter.xml | 2 +- .../org/apache/beam/sdk/util/BitSetCoder.java | 2 +- 45 files changed, 172 insertions(+), 109 deletions(-) rename runners/core-java/src/main/java/org/apache/beam/{sdk/util => runners/core}/AssignWindows.java (97%) rename runners/core-java/src/main/java/org/apache/beam/{sdk/util => runners/core}/AssignWindowsDoFn.java (95%) rename runners/core-java/src/main/java/org/apache/beam/{sdk/util => runners/core}/BatchTimerInternals.java (97%) rename runners/core-java/src/main/java/org/apache/beam/{sdk/util => runners/core}/DoFnRunner.java (94%) rename runners/core-java/src/main/java/org/apache/beam/{sdk/util => runners/core}/DoFnRunnerBase.java (97%) rename runners/core-java/src/main/java/org/apache/beam/{sdk/util => runners/core}/DoFnRunners.java (94%) rename runners/core-java/src/main/java/org/apache/beam/{sdk/util/common => runners/core}/ElementByteSizeObservable.java (94%) rename runners/core-java/src/main/java/org/apache/beam/{sdk/util => runners/core}/GroupAlsoByWindowsDoFn.java (93%) rename runners/core-java/src/main/java/org/apache/beam/{sdk/util => runners/core}/GroupAlsoByWindowsViaOutputBufferDoFn.java (95%) rename runners/core-java/src/main/java/org/apache/beam/{sdk/util => runners/core}/GroupByKeyViaGroupByKeyOnly.java (98%) rename runners/core-java/src/main/java/org/apache/beam/{sdk/util => runners/core}/LateDataDroppingDoFnRunner.java (94%) rename runners/core-java/src/main/java/org/apache/beam/{sdk/util => runners/core}/NonEmptyPanes.java (98%) rename runners/core-java/src/main/java/org/apache/beam/{sdk/util => runners/core}/PaneInfoTracker.java (98%) rename runners/core-java/src/main/java/org/apache/beam/{sdk/util/common => runners/core}/PeekingReiterator.java (96%) rename runners/core-java/src/main/java/org/apache/beam/{sdk/util => runners/core}/PushbackSideInputDoFnRunner.java (96%) rename runners/core-java/src/main/java/org/apache/beam/{sdk/util => runners/core}/ReduceFn.java (97%) rename runners/core-java/src/main/java/org/apache/beam/{sdk/util => runners/core}/ReduceFnContextFactory.java (97%) rename runners/core-java/src/main/java/org/apache/beam/{sdk/util => runners/core}/ReduceFnRunner.java (98%) rename runners/core-java/src/main/java/org/apache/beam/{sdk/util => runners/core}/SimpleDoFnRunner.java (89%) rename runners/core-java/src/main/java/org/apache/beam/{sdk/util => runners/core}/SystemReduceFn.java (98%) rename runners/core-java/src/main/java/org/apache/beam/{sdk/util => runners/core}/TriggerRunner.java (96%) rename runners/core-java/src/main/java/org/apache/beam/{sdk/util => runners/core}/WatermarkHold.java (99%) delete mode 100644 runners/core-java/src/main/java/org/apache/beam/sdk/util/common/package-info.java delete mode 100644 runners/core-java/src/main/java/org/apache/beam/sdk/util/package-info.java rename runners/core-java/src/test/java/org/apache/beam/{sdk/util => runners/core}/BatchTimerInternalsTest.java (98%) rename runners/core-java/src/test/java/org/apache/beam/{sdk/util => runners/core}/GroupAlsoByWindowsProperties.java (99%) rename runners/core-java/src/test/java/org/apache/beam/{sdk/util => runners/core}/GroupAlsoByWindowsViaOutputBufferDoFnTest.java (94%) rename runners/core-java/src/test/java/org/apache/beam/{sdk/util => runners/core}/LateDataDroppingDoFnRunnerTest.java (93%) rename runners/core-java/src/test/java/org/apache/beam/{sdk/util => runners/core}/PushbackSideInputDoFnRunnerTest.java (97%) rename runners/core-java/src/test/java/org/apache/beam/{sdk/util => runners/core}/ReduceFnRunnerTest.java (99%) rename runners/core-java/src/test/java/org/apache/beam/{sdk/util => runners/core}/ReduceFnTester.java (98%) rename runners/core-java/src/test/java/org/apache/beam/{sdk/util => runners/core}/SimpleDoFnRunnerTest.java (93%) diff --git a/runners/core-java/src/main/java/org/apache/beam/sdk/util/AssignWindows.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/AssignWindows.java similarity index 97% rename from runners/core-java/src/main/java/org/apache/beam/sdk/util/AssignWindows.java rename to runners/core-java/src/main/java/org/apache/beam/runners/core/AssignWindows.java index af2805235af9d..f2387f57ba393 100644 --- a/runners/core-java/src/main/java/org/apache/beam/sdk/util/AssignWindows.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/AssignWindows.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 org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.ParDo; diff --git a/runners/core-java/src/main/java/org/apache/beam/sdk/util/AssignWindowsDoFn.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/AssignWindowsDoFn.java similarity index 95% rename from runners/core-java/src/main/java/org/apache/beam/sdk/util/AssignWindowsDoFn.java rename to runners/core-java/src/main/java/org/apache/beam/runners/core/AssignWindowsDoFn.java index 7e26253a829ad..0eb1667c3a1ce 100644 --- a/runners/core-java/src/main/java/org/apache/beam/sdk/util/AssignWindowsDoFn.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/AssignWindowsDoFn.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; @@ -26,6 +26,8 @@ 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.SystemDoFnInternal; +import org.apache.beam.sdk.values.PCollection; import org.joda.time.Instant; /** diff --git a/runners/core-java/src/main/java/org/apache/beam/sdk/util/BatchTimerInternals.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/BatchTimerInternals.java similarity index 97% rename from runners/core-java/src/main/java/org/apache/beam/sdk/util/BatchTimerInternals.java rename to runners/core-java/src/main/java/org/apache/beam/runners/core/BatchTimerInternals.java index f3e84a6861cb7..829dbde84a4c4 100644 --- a/runners/core-java/src/main/java/org/apache/beam/sdk/util/BatchTimerInternals.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/BatchTimerInternals.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.checkState; @@ -25,6 +25,9 @@ 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.joda.time.Instant; /** diff --git a/runners/core-java/src/main/java/org/apache/beam/sdk/util/DoFnRunner.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/DoFnRunner.java similarity index 94% rename from runners/core-java/src/main/java/org/apache/beam/sdk/util/DoFnRunner.java rename to runners/core-java/src/main/java/org/apache/beam/runners/core/DoFnRunner.java index 49206d19a2149..f4c8eea34e1e7 100644 --- a/runners/core-java/src/main/java/org/apache/beam/sdk/util/DoFnRunner.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/DoFnRunner.java @@ -15,11 +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.runners.core; 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/sdk/util/DoFnRunnerBase.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/DoFnRunnerBase.java similarity index 97% rename from runners/core-java/src/main/java/org/apache/beam/sdk/util/DoFnRunnerBase.java rename to runners/core-java/src/main/java/org/apache/beam/runners/core/DoFnRunnerBase.java index f0cfd7455de5c..71472da760c78 100644 --- a/runners/core-java/src/main/java/org/apache/beam/sdk/util/DoFnRunnerBase.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/DoFnRunnerBase.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; @@ -30,6 +30,8 @@ import java.util.List; import java.util.Map; 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; @@ -43,8 +45,14 @@ 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.DoFnRunners.OutputManager; 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.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.StateInternals; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TupleTag; diff --git a/runners/core-java/src/main/java/org/apache/beam/sdk/util/DoFnRunners.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/DoFnRunners.java similarity index 94% rename from runners/core-java/src/main/java/org/apache/beam/sdk/util/DoFnRunners.java rename to runners/core-java/src/main/java/org/apache/beam/runners/core/DoFnRunners.java index c4df7b222036f..7726374d85e4f 100644 --- a/runners/core-java/src/main/java/org/apache/beam/sdk/util/DoFnRunners.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/DoFnRunners.java @@ -15,15 +15,20 @@ * 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.util.List; + +import org.apache.beam.runners.core.DoFnRunner.ReduceFnExecutor; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.Aggregator.AggregatorFactory; import org.apache.beam.sdk.transforms.OldDoFn; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.DoFnRunner.ReduceFnExecutor; 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; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.TupleTag; diff --git a/runners/core-java/src/main/java/org/apache/beam/sdk/util/common/ElementByteSizeObservable.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/ElementByteSizeObservable.java similarity index 94% rename from runners/core-java/src/main/java/org/apache/beam/sdk/util/common/ElementByteSizeObservable.java rename to runners/core-java/src/main/java/org/apache/beam/runners/core/ElementByteSizeObservable.java index 613aa4b2bdd53..2380ba904df32 100644 --- a/runners/core-java/src/main/java/org/apache/beam/sdk/util/common/ElementByteSizeObservable.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/ElementByteSizeObservable.java @@ -15,7 +15,9 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.sdk.util.common; +package org.apache.beam.runners.core; + +import org.apache.beam.sdk.util.common.ElementByteSizeObserver; /** * An interface for things that allow observing the size in bytes of 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 7cdab005e73dc..b427037ef008b 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 @@ -17,16 +17,13 @@ */ package org.apache.beam.runners.core; +import org.apache.beam.runners.core.DoFnRunner.ReduceFnExecutor; import org.apache.beam.sdk.transforms.Aggregator; 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.DoFnRunner.ReduceFnExecutor; -import org.apache.beam.sdk.util.GroupAlsoByWindowsDoFn; import org.apache.beam.sdk.util.KeyedWorkItem; -import org.apache.beam.sdk.util.ReduceFnRunner; import org.apache.beam.sdk.util.SystemDoFnInternal; -import org.apache.beam.sdk.util.SystemReduceFn; import org.apache.beam.sdk.util.TimerInternals; import org.apache.beam.sdk.util.TimerInternals.TimerData; import org.apache.beam.sdk.util.WindowingStrategy; diff --git a/runners/core-java/src/main/java/org/apache/beam/sdk/util/GroupAlsoByWindowsDoFn.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/GroupAlsoByWindowsDoFn.java similarity index 93% rename from runners/core-java/src/main/java/org/apache/beam/sdk/util/GroupAlsoByWindowsDoFn.java rename to runners/core-java/src/main/java/org/apache/beam/runners/core/GroupAlsoByWindowsDoFn.java index f386dfba12201..9851449484028 100644 --- a/runners/core-java/src/main/java/org/apache/beam/sdk/util/GroupAlsoByWindowsDoFn.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/GroupAlsoByWindowsDoFn.java @@ -15,13 +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 org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.transforms.Aggregator; 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.SystemDoFnInternal; +import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.util.WindowingStrategy; import org.apache.beam.sdk.util.state.StateInternalsFactory; import org.apache.beam.sdk.values.KV; diff --git a/runners/core-java/src/main/java/org/apache/beam/sdk/util/GroupAlsoByWindowsViaOutputBufferDoFn.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/GroupAlsoByWindowsViaOutputBufferDoFn.java similarity index 95% rename from runners/core-java/src/main/java/org/apache/beam/sdk/util/GroupAlsoByWindowsViaOutputBufferDoFn.java rename to runners/core-java/src/main/java/org/apache/beam/runners/core/GroupAlsoByWindowsViaOutputBufferDoFn.java index cc418daec7e97..091ad33bd8de9 100644 --- a/runners/core-java/src/main/java/org/apache/beam/sdk/util/GroupAlsoByWindowsViaOutputBufferDoFn.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/GroupAlsoByWindowsViaOutputBufferDoFn.java @@ -15,12 +15,15 @@ * 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.List; 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; +import org.apache.beam.sdk.util.WindowingStrategy; 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/runners/core-java/src/main/java/org/apache/beam/sdk/util/GroupByKeyViaGroupByKeyOnly.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/GroupByKeyViaGroupByKeyOnly.java similarity index 98% rename from runners/core-java/src/main/java/org/apache/beam/sdk/util/GroupByKeyViaGroupByKeyOnly.java rename to runners/core-java/src/main/java/org/apache/beam/runners/core/GroupByKeyViaGroupByKeyOnly.java index fdad17a8bb0c7..b5214259a86b1 100644 --- a/runners/core-java/src/main/java/org/apache/beam/sdk/util/GroupByKeyViaGroupByKeyOnly.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/GroupByKeyViaGroupByKeyOnly.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,7 +31,10 @@ 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.ReifyTimestampsAndWindows; +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; diff --git a/runners/core-java/src/main/java/org/apache/beam/sdk/util/LateDataDroppingDoFnRunner.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/LateDataDroppingDoFnRunner.java similarity index 94% rename from runners/core-java/src/main/java/org/apache/beam/sdk/util/LateDataDroppingDoFnRunner.java rename to runners/core-java/src/main/java/org/apache/beam/runners/core/LateDataDroppingDoFnRunner.java index 08c670e4753f5..63a80d2294e99 100644 --- a/runners/core-java/src/main/java/org/apache/beam/sdk/util/LateDataDroppingDoFnRunner.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/LateDataDroppingDoFnRunner.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.base.Function; @@ -24,6 +24,12 @@ 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; +import org.apache.beam.sdk.util.WindowingStrategy; import org.apache.beam.sdk.values.KV; import org.joda.time.Instant; diff --git a/runners/core-java/src/main/java/org/apache/beam/sdk/util/NonEmptyPanes.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/NonEmptyPanes.java similarity index 98% rename from runners/core-java/src/main/java/org/apache/beam/sdk/util/NonEmptyPanes.java rename to runners/core-java/src/main/java/org/apache/beam/runners/core/NonEmptyPanes.java index e809c24062fd1..3e51dfb7e6d37 100644 --- a/runners/core-java/src/main/java/org/apache/beam/sdk/util/NonEmptyPanes.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/NonEmptyPanes.java @@ -15,11 +15,12 @@ * 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.coders.VarLongCoder; import org.apache.beam.sdk.transforms.Sum; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; +import org.apache.beam.sdk.util.WindowingStrategy; import org.apache.beam.sdk.util.WindowingStrategy.AccumulationMode; import org.apache.beam.sdk.util.state.AccumulatorCombiningState; import org.apache.beam.sdk.util.state.MergingStateAccessor; diff --git a/runners/core-java/src/main/java/org/apache/beam/sdk/util/PaneInfoTracker.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/PaneInfoTracker.java similarity index 98% rename from runners/core-java/src/main/java/org/apache/beam/sdk/util/PaneInfoTracker.java rename to runners/core-java/src/main/java/org/apache/beam/runners/core/PaneInfoTracker.java index 90c10b5c7449e..114f5e6313a40 100644 --- a/runners/core-java/src/main/java/org/apache/beam/sdk/util/PaneInfoTracker.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/PaneInfoTracker.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.checkState; @@ -25,6 +25,8 @@ import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.transforms.windowing.PaneInfo.PaneInfoCoder; import org.apache.beam.sdk.transforms.windowing.PaneInfo.Timing; +import org.apache.beam.sdk.util.TimerInternals; +import org.apache.beam.sdk.util.WindowTracing; import org.apache.beam.sdk.util.state.ReadableState; import org.apache.beam.sdk.util.state.StateAccessor; import org.apache.beam.sdk.util.state.StateTag; diff --git a/runners/core-java/src/main/java/org/apache/beam/sdk/util/common/PeekingReiterator.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/PeekingReiterator.java similarity index 96% rename from runners/core-java/src/main/java/org/apache/beam/sdk/util/common/PeekingReiterator.java rename to runners/core-java/src/main/java/org/apache/beam/runners/core/PeekingReiterator.java index 1e3c17f1a6e31..fcdff3bac906f 100644 --- a/runners/core-java/src/main/java/org/apache/beam/sdk/util/common/PeekingReiterator.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/PeekingReiterator.java @@ -15,12 +15,13 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.sdk.util.common; +package org.apache.beam.runners.core; import static com.google.common.base.Preconditions.checkNotNull; import static com.google.common.base.Preconditions.checkState; import java.util.NoSuchElementException; +import org.apache.beam.sdk.util.common.Reiterator; /** * A {@link Reiterator} that supports one-element lookahead during iteration. diff --git a/runners/core-java/src/main/java/org/apache/beam/sdk/util/PushbackSideInputDoFnRunner.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/PushbackSideInputDoFnRunner.java similarity index 96% rename from runners/core-java/src/main/java/org/apache/beam/sdk/util/PushbackSideInputDoFnRunner.java rename to runners/core-java/src/main/java/org/apache/beam/runners/core/PushbackSideInputDoFnRunner.java index d9f1fbffccb38..deeac3c2ee8da 100644 --- a/runners/core-java/src/main/java/org/apache/beam/sdk/util/PushbackSideInputDoFnRunner.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/PushbackSideInputDoFnRunner.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.ImmutableList; import com.google.common.collect.Iterables; @@ -23,7 +23,10 @@ import java.util.Collections; import java.util.HashSet; 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.WindowedValue; import org.apache.beam.sdk.values.PCollectionView; /** diff --git a/runners/core-java/src/main/java/org/apache/beam/sdk/util/ReduceFn.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/ReduceFn.java similarity index 97% rename from runners/core-java/src/main/java/org/apache/beam/sdk/util/ReduceFn.java rename to runners/core-java/src/main/java/org/apache/beam/runners/core/ReduceFn.java index 8135a5beca530..bb20226fcb33d 100644 --- a/runners/core-java/src/main/java/org/apache/beam/sdk/util/ReduceFn.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/ReduceFn.java @@ -15,11 +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.runners.core; import java.io.Serializable; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; +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.StateAccessor; diff --git a/runners/core-java/src/main/java/org/apache/beam/sdk/util/ReduceFnContextFactory.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/ReduceFnContextFactory.java similarity index 97% rename from runners/core-java/src/main/java/org/apache/beam/sdk/util/ReduceFnContextFactory.java rename to runners/core-java/src/main/java/org/apache/beam/runners/core/ReduceFnContextFactory.java index 5d27d51ad2f43..2043f143609e8 100644 --- a/runners/core-java/src/main/java/org/apache/beam/sdk/util/ReduceFnContextFactory.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/ReduceFnContextFactory.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; import static com.google.common.base.Preconditions.checkNotNull; @@ -28,7 +28,13 @@ 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.TimeDomain; +import org.apache.beam.sdk.util.TimerInternals; import org.apache.beam.sdk.util.TimerInternals.TimerData; +import org.apache.beam.sdk.util.Timers; +import org.apache.beam.sdk.util.WindowingInternals; +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; diff --git a/runners/core-java/src/main/java/org/apache/beam/sdk/util/ReduceFnRunner.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/ReduceFnRunner.java similarity index 98% rename from runners/core-java/src/main/java/org/apache/beam/sdk/util/ReduceFnRunner.java rename to runners/core-java/src/main/java/org/apache/beam/runners/core/ReduceFnRunner.java index 7c3e4d749a1f3..96d764aa62ea4 100644 --- a/runners/core-java/src/main/java/org/apache/beam/sdk/util/ReduceFnRunner.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/ReduceFnRunner.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; import static com.google.common.base.Preconditions.checkState; @@ -31,6 +31,9 @@ import java.util.Map; import java.util.Set; import javax.annotation.Nullable; +import org.apache.beam.runners.core.GroupByKeyViaGroupByKeyOnly.GroupByKeyOnly; +import org.apache.beam.runners.core.ReduceFnContextFactory.OnTriggerCallbacks; +import org.apache.beam.runners.core.ReduceFnContextFactory.StateStyle; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.Aggregator; import org.apache.beam.sdk.transforms.windowing.AfterWatermark; @@ -41,10 +44,17 @@ 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.GroupByKeyViaGroupByKeyOnly.GroupByKeyOnly; -import org.apache.beam.sdk.util.ReduceFnContextFactory.OnTriggerCallbacks; -import org.apache.beam.sdk.util.ReduceFnContextFactory.StateStyle; +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.TimeDomain; +import org.apache.beam.sdk.util.TimerInternals; import org.apache.beam.sdk.util.TimerInternals.TimerData; +import org.apache.beam.sdk.util.TriggerContextFactory; +import org.apache.beam.sdk.util.WindowTracing; +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.WindowingStrategy.AccumulationMode; import org.apache.beam.sdk.util.state.ReadableState; import org.apache.beam.sdk.util.state.StateInternals; diff --git a/runners/core-java/src/main/java/org/apache/beam/sdk/util/SimpleDoFnRunner.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/SimpleDoFnRunner.java similarity index 89% rename from runners/core-java/src/main/java/org/apache/beam/sdk/util/SimpleDoFnRunner.java rename to runners/core-java/src/main/java/org/apache/beam/runners/core/SimpleDoFnRunner.java index 1ebe5a874e74a..df74ed3b9a7b7 100644 --- a/runners/core-java/src/main/java/org/apache/beam/sdk/util/SimpleDoFnRunner.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/SimpleDoFnRunner.java @@ -15,14 +15,17 @@ * 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.util.List; +import org.apache.beam.runners.core.DoFnRunners.OutputManager; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.Aggregator.AggregatorFactory; import org.apache.beam.sdk.transforms.OldDoFn; -import org.apache.beam.sdk.util.DoFnRunners.OutputManager; 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; import org.apache.beam.sdk.values.TupleTag; /** @@ -31,7 +34,7 @@ * @param the type of the {@link OldDoFn} (main) input elements * @param the type of the {@link OldDoFn} (main) output elements */ -public class SimpleDoFnRunner extends DoFnRunnerBase{ +public class SimpleDoFnRunner extends DoFnRunnerBase { protected SimpleDoFnRunner(PipelineOptions options, OldDoFn fn, SideInputReader sideInputReader, diff --git a/runners/core-java/src/main/java/org/apache/beam/sdk/util/SystemReduceFn.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/SystemReduceFn.java similarity index 98% rename from runners/core-java/src/main/java/org/apache/beam/sdk/util/SystemReduceFn.java rename to runners/core-java/src/main/java/org/apache/beam/runners/core/SystemReduceFn.java index 28177a811cd3a..3948d9e61ece7 100644 --- a/runners/core-java/src/main/java/org/apache/beam/sdk/util/SystemReduceFn.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/SystemReduceFn.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 edu.umd.cs.findbugs.annotations.SuppressFBWarnings; @@ -25,6 +25,7 @@ import org.apache.beam.sdk.transforms.CombineWithContext.KeyedCombineFnWithContext; import org.apache.beam.sdk.transforms.GroupByKey; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; +import org.apache.beam.sdk.util.AppliedCombineFn; import org.apache.beam.sdk.util.state.AccumulatorCombiningState; import org.apache.beam.sdk.util.state.BagState; import org.apache.beam.sdk.util.state.CombiningState; diff --git a/runners/core-java/src/main/java/org/apache/beam/sdk/util/TriggerRunner.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/TriggerRunner.java similarity index 96% rename from runners/core-java/src/main/java/org/apache/beam/sdk/util/TriggerRunner.java rename to runners/core-java/src/main/java/org/apache/beam/runners/core/TriggerRunner.java index a53fb8c175135..8d0f322545854 100644 --- a/runners/core-java/src/main/java/org/apache/beam/sdk/util/TriggerRunner.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/TriggerRunner.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.checkState; @@ -28,6 +28,12 @@ import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.DefaultTrigger; import org.apache.beam.sdk.transforms.windowing.Trigger; +import org.apache.beam.sdk.util.BitSetCoder; +import org.apache.beam.sdk.util.ExecutableTrigger; +import org.apache.beam.sdk.util.FinishedTriggers; +import org.apache.beam.sdk.util.FinishedTriggersBitSet; +import org.apache.beam.sdk.util.Timers; +import org.apache.beam.sdk.util.TriggerContextFactory; import org.apache.beam.sdk.util.state.MergingStateAccessor; import org.apache.beam.sdk.util.state.StateAccessor; import org.apache.beam.sdk.util.state.StateTag; diff --git a/runners/core-java/src/main/java/org/apache/beam/sdk/util/WatermarkHold.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/WatermarkHold.java similarity index 99% rename from runners/core-java/src/main/java/org/apache/beam/sdk/util/WatermarkHold.java rename to runners/core-java/src/main/java/org/apache/beam/runners/core/WatermarkHold.java index 7404e1bc8ba3a..7d0b608f8cd91 100644 --- a/runners/core-java/src/main/java/org/apache/beam/sdk/util/WatermarkHold.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/WatermarkHold.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.checkState; @@ -28,6 +28,9 @@ import org.apache.beam.sdk.transforms.windowing.OutputTimeFns; import org.apache.beam.sdk.transforms.windowing.PaneInfo.Timing; import org.apache.beam.sdk.transforms.windowing.Window.ClosingBehavior; +import org.apache.beam.sdk.util.TimerInternals; +import org.apache.beam.sdk.util.WindowTracing; +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.StateMerging; diff --git a/runners/core-java/src/main/java/org/apache/beam/sdk/util/common/package-info.java b/runners/core-java/src/main/java/org/apache/beam/sdk/util/common/package-info.java deleted file mode 100644 index 1ea8af812cd49..0000000000000 --- a/runners/core-java/src/main/java/org/apache/beam/sdk/util/common/package-info.java +++ /dev/null @@ -1,20 +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. - */ - -/** Defines utilities shared by multiple PipelineRunner implementations. */ -package org.apache.beam.sdk.util.common; diff --git a/runners/core-java/src/main/java/org/apache/beam/sdk/util/package-info.java b/runners/core-java/src/main/java/org/apache/beam/sdk/util/package-info.java deleted file mode 100644 index b4772f35d33c6..0000000000000 --- a/runners/core-java/src/main/java/org/apache/beam/sdk/util/package-info.java +++ /dev/null @@ -1,20 +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. - */ - -/** Defines utilities that can be used by Beam runners. */ -package org.apache.beam.sdk.util; diff --git a/runners/core-java/src/test/java/org/apache/beam/sdk/util/BatchTimerInternalsTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/BatchTimerInternalsTest.java similarity index 98% rename from runners/core-java/src/test/java/org/apache/beam/sdk/util/BatchTimerInternalsTest.java rename to runners/core-java/src/test/java/org/apache/beam/runners/core/BatchTimerInternalsTest.java index 20a9852b09580..122e60ce6131f 100644 --- a/runners/core-java/src/test/java/org/apache/beam/sdk/util/BatchTimerInternalsTest.java +++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/BatchTimerInternalsTest.java @@ -15,8 +15,9 @@ * 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.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; diff --git a/runners/core-java/src/test/java/org/apache/beam/sdk/util/GroupAlsoByWindowsProperties.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/GroupAlsoByWindowsProperties.java similarity index 99% rename from runners/core-java/src/test/java/org/apache/beam/sdk/util/GroupAlsoByWindowsProperties.java rename to runners/core-java/src/test/java/org/apache/beam/runners/core/GroupAlsoByWindowsProperties.java index 215cd4c38cb31..d1e0c68143921 100644 --- a/runners/core-java/src/test/java/org/apache/beam/sdk/util/GroupAlsoByWindowsProperties.java +++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/GroupAlsoByWindowsProperties.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.hamcrest.Matchers.contains; import static org.hamcrest.Matchers.containsInAnyOrder; @@ -41,6 +41,8 @@ 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.WindowedValue; +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; diff --git a/runners/core-java/src/test/java/org/apache/beam/sdk/util/GroupAlsoByWindowsViaOutputBufferDoFnTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/GroupAlsoByWindowsViaOutputBufferDoFnTest.java similarity index 94% rename from runners/core-java/src/test/java/org/apache/beam/sdk/util/GroupAlsoByWindowsViaOutputBufferDoFnTest.java rename to runners/core-java/src/test/java/org/apache/beam/runners/core/GroupAlsoByWindowsViaOutputBufferDoFnTest.java index a1586c8bf2d9d..1fad1fbe5d6cf 100644 --- a/runners/core-java/src/test/java/org/apache/beam/sdk/util/GroupAlsoByWindowsViaOutputBufferDoFnTest.java +++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/GroupAlsoByWindowsViaOutputBufferDoFnTest.java @@ -15,12 +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.runners.core; +import org.apache.beam.runners.core.GroupAlsoByWindowsProperties.GroupAlsoByWindowsDoFnFactory; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.StringUtf8Coder; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.GroupAlsoByWindowsProperties.GroupAlsoByWindowsDoFnFactory; +import org.apache.beam.sdk.util.WindowingStrategy; import org.apache.beam.sdk.util.state.StateInternalsFactory; import org.junit.Test; import org.junit.runner.RunWith; @@ -43,7 +44,7 @@ public BufferingGABWViaOutputBufferDoFnFactory(Coder inputCoder) { @Override public - GroupAlsoByWindowsDoFn, W> forStrategy( + GroupAlsoByWindowsDoFn, W> forStrategy( WindowingStrategy windowingStrategy, StateInternalsFactory stateInternalsFactory) { return new GroupAlsoByWindowsViaOutputBufferDoFn, W>( diff --git a/runners/core-java/src/test/java/org/apache/beam/sdk/util/LateDataDroppingDoFnRunnerTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/LateDataDroppingDoFnRunnerTest.java similarity index 93% rename from runners/core-java/src/test/java/org/apache/beam/sdk/util/LateDataDroppingDoFnRunnerTest.java rename to runners/core-java/src/test/java/org/apache/beam/runners/core/LateDataDroppingDoFnRunnerTest.java index c63e43ef55b03..1cf05b6707290 100644 --- a/runners/core-java/src/test/java/org/apache/beam/sdk/util/LateDataDroppingDoFnRunnerTest.java +++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/LateDataDroppingDoFnRunnerTest.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.hamcrest.Matchers.containsInAnyOrder; import static org.junit.Assert.assertEquals; @@ -25,12 +25,15 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.Iterables; import java.util.Arrays; +import org.apache.beam.runners.core.LateDataDroppingDoFnRunner.LateDataFilter; import org.apache.beam.sdk.transforms.Aggregator; import org.apache.beam.sdk.transforms.Combine.CombineFn; import org.apache.beam.sdk.transforms.Sum; import org.apache.beam.sdk.transforms.windowing.FixedWindows; import org.apache.beam.sdk.transforms.windowing.PaneInfo; -import org.apache.beam.sdk.util.LateDataDroppingDoFnRunner.LateDataFilter; +import org.apache.beam.sdk.util.TimerInternals; +import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.util.WindowingStrategy; import org.joda.time.Duration; import org.joda.time.Instant; import org.junit.Before; diff --git a/runners/core-java/src/test/java/org/apache/beam/sdk/util/PushbackSideInputDoFnRunnerTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/PushbackSideInputDoFnRunnerTest.java similarity index 97% rename from runners/core-java/src/test/java/org/apache/beam/sdk/util/PushbackSideInputDoFnRunnerTest.java rename to runners/core-java/src/test/java/org/apache/beam/runners/core/PushbackSideInputDoFnRunnerTest.java index f8ad291d9bc95..59a7c92e7d218 100644 --- a/runners/core-java/src/test/java/org/apache/beam/sdk/util/PushbackSideInputDoFnRunnerTest.java +++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/PushbackSideInputDoFnRunnerTest.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.hamcrest.Matchers.containsInAnyOrder; import static org.hamcrest.Matchers.emptyIterable; @@ -27,6 +27,7 @@ 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; @@ -35,6 +36,9 @@ 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.IdentitySideInputWindowFn; +import org.apache.beam.sdk.util.ReadyCheckingSideInputReader; +import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionView; import org.hamcrest.Matchers; diff --git a/runners/core-java/src/test/java/org/apache/beam/sdk/util/ReduceFnRunnerTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/ReduceFnRunnerTest.java similarity index 99% rename from runners/core-java/src/test/java/org/apache/beam/sdk/util/ReduceFnRunnerTest.java rename to runners/core-java/src/test/java/org/apache/beam/runners/core/ReduceFnRunnerTest.java index 647495cc344a3..4d5680c5ea974 100644 --- a/runners/core-java/src/test/java/org/apache/beam/sdk/util/ReduceFnRunnerTest.java +++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/ReduceFnRunnerTest.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; import static org.apache.beam.sdk.WindowMatchers.isSingleWindowedValue; @@ -65,6 +65,10 @@ import org.apache.beam.sdk.transforms.windowing.Trigger; import org.apache.beam.sdk.transforms.windowing.Window.ClosingBehavior; import org.apache.beam.sdk.transforms.windowing.WindowFn; +import org.apache.beam.sdk.util.SideInputReader; +import org.apache.beam.sdk.util.TimeDomain; +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.values.PCollectionView; import org.apache.beam.sdk.values.TimestampedValue; diff --git a/runners/core-java/src/test/java/org/apache/beam/sdk/util/ReduceFnTester.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/ReduceFnTester.java similarity index 98% rename from runners/core-java/src/test/java/org/apache/beam/sdk/util/ReduceFnTester.java rename to runners/core-java/src/test/java/org/apache/beam/runners/core/ReduceFnTester.java index 24e33ddf310ce..45062fbb8c826 100644 --- a/runners/core-java/src/test/java/org/apache/beam/sdk/util/ReduceFnTester.java +++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/ReduceFnTester.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; @@ -43,6 +43,7 @@ import java.util.PriorityQueue; import java.util.Set; import javax.annotation.Nullable; + import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.CoderRegistry; import org.apache.beam.sdk.coders.IterableCoder; @@ -63,7 +64,18 @@ import org.apache.beam.sdk.transforms.windowing.Trigger; import org.apache.beam.sdk.transforms.windowing.Window.ClosingBehavior; import org.apache.beam.sdk.transforms.windowing.WindowFn; +import org.apache.beam.sdk.util.AppliedCombineFn; +import org.apache.beam.sdk.util.ExecutableTrigger; +import org.apache.beam.sdk.util.NullSideInputReader; +import org.apache.beam.sdk.util.SerializableUtils; +import org.apache.beam.sdk.util.SideInputReader; +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.WindowTracing; +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.WindowingStrategy.AccumulationMode; import org.apache.beam.sdk.util.state.InMemoryStateInternals; import org.apache.beam.sdk.util.state.State; diff --git a/runners/core-java/src/test/java/org/apache/beam/sdk/util/SimpleDoFnRunnerTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/SimpleDoFnRunnerTest.java similarity index 93% rename from runners/core-java/src/test/java/org/apache/beam/sdk/util/SimpleDoFnRunnerTest.java rename to runners/core-java/src/test/java/org/apache/beam/runners/core/SimpleDoFnRunnerTest.java index 156b4a92ca17b..adb0aac7db2c7 100644 --- a/runners/core-java/src/test/java/org/apache/beam/sdk/util/SimpleDoFnRunnerTest.java +++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/SimpleDoFnRunnerTest.java @@ -15,15 +15,19 @@ * 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.hamcrest.Matchers.is; import static org.mockito.Mockito.mock; import java.util.Arrays; import java.util.List; + 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; import org.apache.beam.sdk.values.TupleTag; import org.junit.Rule; import org.junit.Test; 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 2da70bbe56289..c08c229964f96 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 @@ -20,6 +20,10 @@ import com.google.common.collect.ImmutableMap; import java.util.Collections; import org.apache.beam.runners.core.GroupAlsoByWindowViaWindowSetDoFn; +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.SystemReduceFn; import org.apache.beam.runners.direct.DirectExecutionContext.DirectStepContext; import org.apache.beam.runners.direct.DirectGroupByKey.DirectGroupAlsoByWindow; import org.apache.beam.runners.direct.DirectRunner.CommittedBundle; @@ -28,11 +32,7 @@ import org.apache.beam.sdk.transforms.OldDoFn; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.GroupByKeyViaGroupByKeyOnly; -import org.apache.beam.sdk.util.GroupByKeyViaGroupByKeyOnly.GroupAlsoByWindow; -import org.apache.beam.sdk.util.GroupByKeyViaGroupByKeyOnly.GroupByKeyOnly; import org.apache.beam.sdk.util.KeyedWorkItem; -import org.apache.beam.sdk.util.SystemReduceFn; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.WindowingStrategy; import org.apache.beam.sdk.util.state.StateInternals; 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 f085a39fda9bf..17dc0bea3c9c7 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 @@ -25,6 +25,8 @@ import java.util.HashMap; import java.util.List; import java.util.Map; +import org.apache.beam.runners.core.GroupByKeyViaGroupByKeyOnly; +import org.apache.beam.runners.core.GroupByKeyViaGroupByKeyOnly.GroupByKeyOnly; import org.apache.beam.runners.direct.DirectGroupByKey.DirectGroupByKeyOnly; import org.apache.beam.runners.direct.DirectRunner.CommittedBundle; import org.apache.beam.runners.direct.DirectRunner.UncommittedBundle; @@ -34,8 +36,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.GroupByKeyViaGroupByKeyOnly; -import org.apache.beam.sdk.util.GroupByKeyViaGroupByKeyOnly.GroupByKeyOnly; import org.apache.beam.sdk.util.KeyedWorkItem; import org.apache.beam.sdk.util.KeyedWorkItems; import org.apache.beam.sdk.util.WindowedValue; 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 85a1c6af13858..99ab22ab12bc7 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 @@ -23,15 +23,15 @@ import java.util.HashMap; import java.util.List; import java.util.Map; +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.PushbackSideInputDoFnRunner; import org.apache.beam.runners.direct.DirectExecutionContext.DirectStepContext; import org.apache.beam.runners.direct.DirectRunner.CommittedBundle; import org.apache.beam.runners.direct.DirectRunner.UncommittedBundle; import org.apache.beam.sdk.transforms.AppliedPTransform; import org.apache.beam.sdk.transforms.OldDoFn; -import org.apache.beam.sdk.util.DoFnRunner; -import org.apache.beam.sdk.util.DoFnRunners; -import org.apache.beam.sdk.util.DoFnRunners.OutputManager; -import org.apache.beam.sdk.util.PushbackSideInputDoFnRunner; import org.apache.beam.sdk.util.ReadyCheckingSideInputReader; 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/UncommittedBundleOutputManager.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/UncommittedBundleOutputManager.java index 41f7e8d64b396..d40dc110df59e 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/UncommittedBundleOutputManager.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/UncommittedBundleOutputManager.java @@ -18,9 +18,9 @@ package org.apache.beam.runners.direct; import java.util.Map; +import org.apache.beam.runners.core.DoFnRunners.OutputManager; import org.apache.beam.runners.direct.DirectRunner.CommittedBundle; import org.apache.beam.runners.direct.DirectRunner.UncommittedBundle; -import org.apache.beam.sdk.util.DoFnRunners.OutputManager; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.TupleTag; 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 07e2191d4e0c2..3719fa8e5d4c7 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 @@ -63,7 +63,7 @@ import org.apache.beam.sdk.transforms.windowing.WindowFn; import org.apache.beam.sdk.util.AppliedCombineFn; import org.apache.beam.sdk.util.Reshuffle; -import org.apache.beam.sdk.util.SystemReduceFn; +import org.apache.beam.runners.core.SystemReduceFn; 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/wrappers/streaming/DoFnOperator.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DoFnOperator.java index 2c7ebc697709c..3b0fccc2b3650 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,11 +40,11 @@ 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.DoFnRunner; -import org.apache.beam.sdk.util.DoFnRunners; +import org.apache.beam.runners.core.DoFnRunner; +import org.apache.beam.runners.core.DoFnRunners; import org.apache.beam.sdk.util.ExecutionContext; import org.apache.beam.sdk.util.NullSideInputReader; -import org.apache.beam.sdk.util.PushbackSideInputDoFnRunner; +import org.apache.beam.runners.core.PushbackSideInputDoFnRunner; import org.apache.beam.sdk.util.SideInputReader; 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/WindowDoFnOperator.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/WindowDoFnOperator.java index 01cfa5b6a26bf..b8931161045e7 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 @@ -43,7 +43,7 @@ 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.SystemReduceFn; +import org.apache.beam.runners.core.SystemReduceFn; 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/SparkRunner.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/SparkRunner.java index 9f1a83996fb76..03db8117aae44 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 org.apache.beam.runners.core.GroupByKeyViaGroupByKeyOnly; import org.apache.beam.runners.spark.translation.EvaluationContext; import org.apache.beam.runners.spark.translation.SparkContextFactory; import org.apache.beam.runners.spark.translation.SparkPipelineEvaluator; @@ -36,7 +37,6 @@ import org.apache.beam.sdk.transforms.Create; import org.apache.beam.sdk.transforms.GroupByKey; import org.apache.beam.sdk.transforms.PTransform; -import org.apache.beam.sdk.util.GroupByKeyViaGroupByKeyOnly; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PInput; import org.apache.beam.sdk.values.POutput; 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 eaceb852ce344..8341c6d84a24d 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 @@ -37,6 +37,11 @@ import org.apache.avro.mapred.AvroKey; import org.apache.avro.mapreduce.AvroJob; import org.apache.avro.mapreduce.AvroKeyInputFormat; +import org.apache.beam.runners.core.AssignWindowsDoFn; +import org.apache.beam.runners.core.GroupAlsoByWindowsViaOutputBufferDoFn; +import org.apache.beam.runners.core.GroupByKeyViaGroupByKeyOnly.GroupAlsoByWindow; +import org.apache.beam.runners.core.GroupByKeyViaGroupByKeyOnly.GroupByKeyOnly; +import org.apache.beam.runners.core.SystemReduceFn; import org.apache.beam.runners.spark.coders.CoderHelpers; import org.apache.beam.runners.spark.io.hadoop.HadoopIO; import org.apache.beam.runners.spark.io.hadoop.ShardNameTemplateHelper; @@ -61,11 +66,6 @@ import org.apache.beam.sdk.transforms.windowing.GlobalWindows; import org.apache.beam.sdk.transforms.windowing.Window; import org.apache.beam.sdk.transforms.windowing.WindowFn; -import org.apache.beam.sdk.util.AssignWindowsDoFn; -import org.apache.beam.sdk.util.GroupAlsoByWindowsViaOutputBufferDoFn; -import org.apache.beam.sdk.util.GroupByKeyViaGroupByKeyOnly.GroupAlsoByWindow; -import org.apache.beam.sdk.util.GroupByKeyViaGroupByKeyOnly.GroupByKeyOnly; -import org.apache.beam.sdk.util.SystemReduceFn; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.WindowedValue.WindowedValueCoder; import org.apache.beam.sdk.util.WindowingStrategy; 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 43dcef665fb2a..c55be3dee2d92 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 @@ -28,6 +28,7 @@ import java.util.Map; import java.util.Set; import kafka.serializer.Decoder; +import org.apache.beam.runners.core.AssignWindowsDoFn; import org.apache.beam.runners.spark.io.ConsoleIO; import org.apache.beam.runners.spark.io.CreateStream; import org.apache.beam.runners.spark.io.KafkaIO; @@ -49,7 +50,6 @@ import org.apache.beam.sdk.transforms.windowing.SlidingWindows; import org.apache.beam.sdk.transforms.windowing.Window; import org.apache.beam.sdk.transforms.windowing.WindowFn; -import org.apache.beam.sdk.util.AssignWindowsDoFn; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollectionList; 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 cd50408fd395e..3c016907a3b36 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 @@ -110,7 +110,7 @@ - + diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/BitSetCoder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/BitSetCoder.java index fde90af683697..72524bd446e84 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/BitSetCoder.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/BitSetCoder.java @@ -28,7 +28,7 @@ /** * Coder for the BitSet used to track child-trigger finished states. */ -class BitSetCoder extends AtomicCoder { +public class BitSetCoder extends AtomicCoder { private static final BitSetCoder INSTANCE = new BitSetCoder(); private static final ByteArrayCoder byteArrayCoder = ByteArrayCoder.of(); From c92e45dd4019e613a7670e4bb0e1fcc4b7e2c462 Mon Sep 17 00:00:00 2001 From: Pei He Date: Thu, 25 Aug 2016 14:20:30 -0700 Subject: [PATCH 103/346] Remove the DataflowRunner instructions from examples --- .../beam/examples/DebuggingWordCount.java | 16 ++++++------ .../beam/examples/MinimalWordCount.java | 7 +++--- .../beam/examples/WindowedWordCount.java | 22 ++++++---------- .../org/apache/beam/examples/WordCount.java | 22 +++++----------- .../beam/examples/complete/AutoComplete.java | 25 +++++++------------ .../complete/StreamingWordExtract.java | 4 +-- .../apache/beam/examples/complete/TfIdf.java | 18 +++++-------- .../complete/TopWikipediaSessions.java | 12 +++------ .../examples/complete/TrafficMaxLaneFlow.java | 4 +-- .../beam/examples/complete/TrafficRoutes.java | 4 +-- .../examples/cookbook/BigQueryTornadoes.java | 18 +++---------- .../cookbook/CombinePerKeyExamples.java | 18 +++---------- .../examples/cookbook/DatastoreWordCount.java | 17 ++++++------- .../beam/examples/cookbook/DeDupExample.java | 16 ++++++------ .../examples/cookbook/FilterExamples.java | 21 +++------------- .../beam/examples/cookbook/JoinExamples.java | 18 +++---------- .../examples/cookbook/MaxPerKeyExamples.java | 19 +++----------- .../examples/cookbook/TriggerExample.java | 16 ++++++------ 18 files changed, 92 insertions(+), 185 deletions(-) diff --git a/examples/java/src/main/java/org/apache/beam/examples/DebuggingWordCount.java b/examples/java/src/main/java/org/apache/beam/examples/DebuggingWordCount.java index be3aa419b5379..eb382278346ef 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/DebuggingWordCount.java +++ b/examples/java/src/main/java/org/apache/beam/examples/DebuggingWordCount.java @@ -46,12 +46,12 @@ * *

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

              New Concepts: *

                *   1. Logging to Cloud Logging
              - *   2. Controlling Dataflow worker log levels
              + *   2. Controlling worker log levels
                *   3. Creating a custom aggregator
                *   4. Testing your Pipeline via PAssert
                * 
              @@ -62,12 +62,14 @@ * } *
            5. * - *

              To execute this pipeline using the Dataflow service and the additional logging discussed - * below, specify pipeline configuration: + *

              To change the runner, specify: + *

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

              To use the additional logging discussed below, specify: *

              {@code
              - *   --project=YOUR_PROJECT_ID
              - *   --tempLocation=gs://YOUR_TEMP_DIRECTORY
              - *   --runner=BlockingDataflowRunner
                *   --workerLogLevelOverrides={"org.apache.beam.examples":"DEBUG"}
                * }
                * 
              diff --git a/examples/java/src/main/java/org/apache/beam/examples/MinimalWordCount.java b/examples/java/src/main/java/org/apache/beam/examples/MinimalWordCount.java index f28a20cf94491..f772dd5bc7684 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/MinimalWordCount.java +++ b/examples/java/src/main/java/org/apache/beam/examples/MinimalWordCount.java @@ -66,12 +66,11 @@ public static void main(String[] args) { // In order to run your pipeline, you need to make following runner specific changes: // - // CHANGE 1/3: Select a Beam runner, such as BlockingDataflowRunner - // or FlinkRunner. + // CHANGE 1/3: Select a Beam runner, such as DataflowRunner or FlinkRunner. // CHANGE 2/3: Specify runner-required options. - // For BlockingDataflowRunner, set project and temp location as follows: + // For DataflowRunner, set project and temp location as follows: // DataflowPipelineOptions dataflowOptions = options.as(DataflowPipelineOptions.class); - // dataflowOptions.setRunner(BlockingDataflowRunner.class); + // dataflowOptions.setRunner(DataflowRunner.class); // dataflowOptions.setProject("SET_YOUR_PROJECT_ID_HERE"); // dataflowOptions.setTempLocation("gs://SET_YOUR_BUCKET_NAME_HERE/AND_TEMP_DIRECTORY"); // For FlinkRunner, set the runner as follows. See {@code FlinkPipelineOptions} 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 7af354cee0fab..c8bd9d322003b 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 @@ -54,7 +54,7 @@ * *

              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 the Dataflow service; defining DoFns; creating a custom aggregator; + * and using a selected runner; defining DoFns; creating a custom aggregator; * user-defined PTransforms; defining PipelineOptions. * *

              New Concepts: @@ -66,19 +66,13 @@ * 5. Writing to BigQuery * * - *

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

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

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

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

              {@code
              - *   --project=YOUR_PROJECT_ID
              - *   --tempLocation=gs://YOUR_TEMP_DIRECTORY
              - *   --runner=BlockingDataflowRunner
              + *   --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}, @@ -86,7 +80,7 @@ * *

              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 dataflow-examples} must already exist in your project. + * 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 @@ -190,7 +184,7 @@ public static void main(String[] args) throws IOException { Pipeline pipeline = Pipeline.create(options); /** - * Concept #1: the Dataflow SDK lets us run the same pipeline with either a bounded or + * Concept #1: the Beam SDK lets us run the same pipeline with either a bounded or * unbounded input source. */ PCollection input = pipeline @@ -229,7 +223,7 @@ public static void main(String[] args) throws IOException { PipelineResult result = pipeline.run(); - // dataflowUtils will try to cancel the pipeline before the program exists. + // ExampleUtils will try to cancel the pipeline before the program exists. exampleUtils.waitToFinish(result); } } 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 793ee4baceae5..498b0693877e6 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 @@ -48,8 +48,8 @@ * pipeline, for introduction of additional concepts. * *

              For a detailed walkthrough of this example, see - * - * https://cloud.google.com/dataflow/java-sdk/wordcount-example + * + * http://beam.incubator.apache.org/use/walkthroughs/ * * *

              Basic concepts, also in the MinimalWordCount example: @@ -66,27 +66,17 @@ *

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

              {@code
              - *   --project=YOUR_PROJECT_ID
              - * }
              - * 
              - * and a local output file or output prefix on GCS: + * To execute this pipeline locally, specify a local output file or output prefix on GCS: *
              {@code
                *   --output=[YOUR_LOCAL_FILE | gs://YOUR_OUTPUT_PREFIX]
                * }
              * - *

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

              To change the runner, specify: *

              {@code
              - *   --project=YOUR_PROJECT_ID
              - *   --tempLocation=gs://YOUR_TEMP_DIRECTORY
              - *   --runner=BlockingDataflowRunner
              + *   --runner=YOUR_SELECTED_RUNNER
                * }
                * 
              - * and an output prefix on GCS: - *
              {@code
              - *   --output=gs://YOUR_OUTPUT_PREFIX
              - * }
              + * See examples/java/README.md for instructions about how to configure different runners. * *

              The input file defaults to {@code gs://apache-beam-samples/shakespeare/kinglear.txt} * and can be overridden with {@code --inputFile}. 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 2182e6dd0ea53..c3ac6149c4619 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 @@ -77,24 +77,17 @@ *

              Concepts: Using the same pipeline in both streaming and batch, combiners, * composite transforms. * - *

              To execute this pipeline using the Dataflow service in batch mode, - * specify pipeline configuration: + *

              To execute this pipeline in streaming mode, specify: *

              {@code
              - *   --project=YOUR_PROJECT_ID
              - *   --tempLocation=gs://YOUR_TEMP_DIRECTORY
              - *   --runner=DataflowRunner
              - *   --inputFile=gs://path/to/input*.txt
              + *   --streaming
                * }
              * - *

              To execute this pipeline using the Dataflow service in streaming mode, - * specify pipeline configuration: + *

              To change the runner, specify: *

              {@code
              - *   --project=YOUR_PROJECT_ID
              - *   --tempLocation=gs://YOUR_TEMP_DIRECTORY
              - *   --runner=DataflowRunner
              - *   --inputFile=gs://YOUR_INPUT_DIRECTORY/*.txt
              - *   --streaming
              - * }
              + * --runner=YOUR_SELECTED_RUNNER + * } + * + * See examples/java/README.md for instructions about how to configure different runners. * *

              This will update the Cloud Datastore every 10 seconds based on the last * 30 minutes of data received. @@ -417,7 +410,7 @@ public void processElement(ProcessContext c) { /** * Options supported by this class. * - *

              Inherits standard Dataflow configuration options. + *

              Inherits standard Beam example configuration options. */ private static interface Options extends ExampleOptions, ExampleBigQueryTableOptions, StreamingOptions { @@ -510,7 +503,7 @@ public static void main(String[] args) throws IOException { // Run the pipeline. PipelineResult result = p.run(); - // dataflowUtils will try to cancel the pipeline and the injector before the program exists. + // ExampleUtils will try to cancel the pipeline and the injector before the program exists. exampleUtils.waitToFinish(result); } } diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/StreamingWordExtract.java b/examples/java/src/main/java/org/apache/beam/examples/complete/StreamingWordExtract.java index 869ea69b8ae47..e8d895057fd5f 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/complete/StreamingWordExtract.java +++ b/examples/java/src/main/java/org/apache/beam/examples/complete/StreamingWordExtract.java @@ -44,7 +44,7 @@ * a BigQuery table. * *

              The example is configured to use the default BigQuery table from the example common package - * (there are no defaults for a general Dataflow pipeline). + * (there are no defaults for a general Beam pipeline). * You can override them by using the {@literal --bigQueryDataset}, and {@literal --bigQueryTable} * options. If the BigQuery table do not exist, the example will try to create them. * @@ -141,7 +141,7 @@ public static void main(String[] args) throws IOException { PipelineResult result = pipeline.run(); - // dataflowUtils will try to cancel the pipeline before the program exists. + // ExampleUtils will try to cancel the pipeline before the program exists. exampleUtils.waitToFinish(result); } } 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 6684553c41eb0..59bbd49b55e3f 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 @@ -65,23 +65,17 @@ * *

              Concepts: joining data; side inputs; logging * - *

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

              {@code
              - *   --project=YOUR_PROJECT_ID
              - * }
              - * and a local output file or output prefix on GCS: + *

              To execute this pipeline locally, specify a local output file or output prefix on GCS: *

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

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

              To change the runner, specify: *

              {@code
              - *   --project=YOUR_PROJECT_ID
              - *   --tempLocation=gs://YOUR_TEMP_DIRECTORY
              - *   --runner=BlockingDataflowRunner
              - * and an output prefix on GCS:
              - *   --output=gs://YOUR_OUTPUT_PREFIX
              - * }
              + * --runner=YOUR_SELECTED_RUNNER + * } + * + * See examples/java/README.md for instructions about how to configure different runners. * *

              The default input is {@code gs://apache-beam-samples/shakespeare/} and can be overridden with * {@code --input}. 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 d597258d3b923..0f594d7ff9733 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 @@ -52,17 +52,13 @@ *

              It is not recommended to execute this pipeline locally, given the size of the default input * data. * - *

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

              To execute this pipeline using a selected runner and an output prefix on GCS, specify: *

              {@code
              - *   --project=YOUR_PROJECT_ID
              - *   --tempLocation=gs://YOUR_TEMP_DIRECTORY
              - *   --runner=BlockingDataflowRunner
              + *   --runner=YOUR_SELECTED_RUNNER
              + *   --output=gs://YOUR_OUTPUT_PREFIX
                * }
                * 
              - * and an output prefix on GCS: - *
              {@code
              - *   --output=gs://YOUR_OUTPUT_PREFIX
              - * }
              + * See examples/java/README.md for instructions about how to configure different runners. * *

              The default input is {@code gs://apache-beam-samples/wikipedia_edits/*.json} and can be * overridden with {@code --input}. 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 e4569600c05c6..0c367d484a79f 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 @@ -66,7 +66,7 @@ *

              The pipeline reads traffic sensor data from {@literal --inputFile}. * *

              The example is configured to use the default BigQuery table from the example common package - * (there are no defaults for a general Dataflow pipeline). + * (there are no defaults for a general Beam pipeline). * You can override them by using the {@literal --bigQueryDataset}, and {@literal --bigQueryTable} * options. If the BigQuery table do not exist, the example will try to create them. * @@ -354,7 +354,7 @@ public static void main(String[] args) throws IOException { // Run the pipeline. PipelineResult result = pipeline.run(); - // dataflowUtils will try to cancel the pipeline and the injector before the program exists. + // ExampleUtils will try to cancel the pipeline and the injector before the program exists. exampleUtils.waitToFinish(result); } 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 95336c644a79f..14cee4dd2946c 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 @@ -69,7 +69,7 @@ *

              The pipeline reads traffic sensor data from {@literal --inputFile}. * *

              The example is configured to use the default BigQuery table from the example common package - * (there are no defaults for a general Dataflow pipeline). + * (there are no defaults for a general Beam pipeline). * You can override them by using the {@literal --bigQueryDataset}, and {@literal --bigQueryTable} * options. If the BigQuery table do not exist, the example will try to create them. * @@ -365,7 +365,7 @@ public static void main(String[] args) throws IOException { // Run the pipeline. PipelineResult result = pipeline.run(); - // dataflowUtils will try to cancel the pipeline and the injector before the program exists. + // ExampleUtils will try to cancel the pipeline and the injector before the program exists. exampleUtils.waitToFinish(result); } 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 439cf020aaa9f..1e4918d370258 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 @@ -45,27 +45,17 @@ *

              Note: Before running this example, you must create a BigQuery dataset to contain your output * table. * - *

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

              {@code
              - *   --project=YOUR_PROJECT_ID
              - * }
              - * 
              - * and the BigQuery table for the output, with the form + *

              To execute this pipeline locally, specify the BigQuery table for the output with the form: *

              {@code
                *   --output=YOUR_PROJECT_ID:DATASET_ID.TABLE_ID
                * }
              * - *

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

              To change the runner, specify: *

              {@code
              - *   --project=YOUR_PROJECT_ID
              - *   --tempLocation=gs://YOUR_TEMP_DIRECTORY
              - *   --runner=BlockingDataflowRunner
              + *   --runner=YOUR_SELECTED_RUNNER
                * }
                * 
              - * and the BigQuery table for the output: - *
              {@code
              - *   --output=YOUR_PROJECT_ID:DATASET_ID.TABLE_ID
              - * }
              + * See examples/java/README.md for instructions about how to configure different runners. * *

              The BigQuery input table defaults to {@code clouddataflow-readonly:samples.weather_stations} * and can be overridden with {@code --input}. 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 1d280a6e154b5..fc11ac9c2c9d4 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 @@ -52,27 +52,17 @@ *

              Note: Before running this example, you must create a BigQuery dataset to contain your output * table. * - *

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

              {@code
              - *   --project=YOUR_PROJECT_ID
              - * }
              - * 
              - * and the BigQuery table for the output: + *

              To execute this pipeline locally, specify the BigQuery table for the output: *

              {@code
                *   --output=YOUR_PROJECT_ID:DATASET_ID.TABLE_ID
                * }
              * - *

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

              To change the runner, specify: *

              {@code
              - *   --project=YOUR_PROJECT_ID
              - *   --tempLocation=gs://
              - *   --runner=BlockingDataflowRunner
              + *   --runner=YOUR_SELECTED_RUNNER
                * }
                * 
              - * and the BigQuery table for the output: - *
              {@code
              - *   --output=YOUR_PROJECT_ID:DATASET_ID.TABLE_ID
              - * }
              + * See examples/java/README.md for instructions about how to configure different runners. * *

              The BigQuery input table defaults to {@code publicdata:samples.shakespeare} and can * be overridden with {@code --input}. diff --git a/examples/java/src/main/java/org/apache/beam/examples/cookbook/DatastoreWordCount.java b/examples/java/src/main/java/org/apache/beam/examples/cookbook/DatastoreWordCount.java index 434e9fb94b266..c0066e698ed3d 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/cookbook/DatastoreWordCount.java +++ b/examples/java/src/main/java/org/apache/beam/examples/cookbook/DatastoreWordCount.java @@ -58,14 +58,15 @@ * *

              To run this pipeline locally, the following options must be provided: *

              {@code
              - *   --project=YOUR_PROJECT_ID
                *   --output=[YOUR_LOCAL_FILE | gs://YOUR_OUTPUT_PATH]
                * }
              * - *

              To run this example using Dataflow service, you must additionally - * provide either {@literal --tempLocation} or {@literal --tempLocation}, and - * select one of the Dataflow pipeline runners, eg - * {@literal --runner=BlockingDataflowRunner}. + *

              To change the runner, specify: + *

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

              Note: this example creates entities with Ancestor keys to ensure that all * entities created are in the same entity group. Similarly, the query used to read from the Cloud @@ -239,13 +240,9 @@ public static void readDataFromDatastore(Options options) { } /** - * An example to demo how to use {@link DatastoreIO}. The runner here is - * customizable, which means users could pass either {@code DirectRunner} - * or {@code DataflowRunner} in the pipeline options. + * An example to demo how to use {@link DatastoreIO}. */ public static void main(String args[]) { - // The options are used in two places, for Dataflow service, and - // building DatastoreIO.Read object Options options = PipelineOptionsFactory.fromArgs(args).withValidation().as(Options.class); if (!options.isReadOnly()) { diff --git a/examples/java/src/main/java/org/apache/beam/examples/cookbook/DeDupExample.java b/examples/java/src/main/java/org/apache/beam/examples/cookbook/DeDupExample.java index 57917109addd1..594d52dab4892 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/cookbook/DeDupExample.java +++ b/examples/java/src/main/java/org/apache/beam/examples/cookbook/DeDupExample.java @@ -35,17 +35,15 @@ * Demonstrates {@link org.apache.beam.sdk.io.TextIO.Read}/ * {@link RemoveDuplicates}/{@link org.apache.beam.sdk.io.TextIO.Write}. * - *

              To execute this pipeline locally, specify general pipeline configuration: - * --project=YOUR_PROJECT_ID - * and a local output file or output prefix on GCS: + *

              To execute this pipeline locally, specify a local output file or output prefix on GCS: * --output=[YOUR_LOCAL_FILE | gs://YOUR_OUTPUT_PREFIX] * - *

              To execute this pipeline using the Dataflow service, specify pipeline configuration: - * --project=YOUR_PROJECT_ID - * --tempLocation=gs://YOUR_TEMP_DIRECTORY - * --runner=BlockingDataflowRunner - * and an output prefix on GCS: - * --output=gs://YOUR_OUTPUT_PREFIX + *

              To change the runner, specify: + *

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

              The input defaults to {@code gs://apache-beam-samples/shakespeare/*} and can be * overridden with {@code --input}. 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 6c42520ff7c6e..01d668bb43acc 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 @@ -54,12 +54,7 @@ *

              Note: Before running this example, you must create a BigQuery dataset to contain your output * table. * - *

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

              {@code
              - *   --project=YOUR_PROJECT_ID
              - * }
              - * 
              - * and the BigQuery table for the output: + *

              To execute this pipeline locally, specify the BigQuery table for the output: *

              {@code
                *   --output=YOUR_PROJECT_ID:DATASET_ID.TABLE_ID
                *   [--monthFilter=]
              @@ -67,20 +62,12 @@
                * 
              * where optional parameter {@code --monthFilter} is set to a number 1-12. * - *

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

              To change the runner, specify: *

              {@code
              - *   --project=YOUR_PROJECT_ID
              - *   --tempLocation=gs://YOUR_TEMP_DIRECTORY
              - *   --runner=BlockingDataflowRunner
              + *   --runner=YOUR_SELECTED_RUNNER
                * }
                * 
              - * and the BigQuery table for the output: - *
              {@code
              - *   --output=YOUR_PROJECT_ID:DATASET_ID.TABLE_ID
              - *   [--monthFilter=]
              - * }
              - * 
              - * where optional parameter {@code --monthFilter} is set to a number 1-12. + * See examples/java/README.md for instructions about how to configure different runners. * *

              The BigQuery input table defaults to {@code clouddataflow-readonly:samples.weather_stations} * and can be overridden with {@code --input}. diff --git a/examples/java/src/main/java/org/apache/beam/examples/cookbook/JoinExamples.java b/examples/java/src/main/java/org/apache/beam/examples/cookbook/JoinExamples.java index 1b91bf1e0615b..799cad3f1e9ea 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/cookbook/JoinExamples.java +++ b/examples/java/src/main/java/org/apache/beam/examples/cookbook/JoinExamples.java @@ -41,27 +41,17 @@ * *

              Concepts: Join operation; multiple input sources. * - *

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

              {@code
              - *   --project=YOUR_PROJECT_ID
              - * }
              - * 
              - * and a local output file or output prefix on GCS: + *

              To execute this pipeline locally, specify a local output file or output prefix on GCS: *

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

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

              To change the runner, specify: *

              {@code
              - *   --project=YOUR_PROJECT_ID
              - *   --tempLocation=gs://YOUR_TEMP_DIRECTORY
              - *   --runner=BlockingDataflowRunner
              + *   --runner=YOUR_SELECTED_RUNNER
                * }
                * 
              - * and an output prefix on GCS: - *
              {@code
              - *   --output=gs://YOUR_OUTPUT_PREFIX
              - * }
              + * See examples/java/README.md for instructions about how to configure different runners. */ public class JoinExamples { 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 3772a7bc5b86a..3a4fa2676f96d 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 @@ -46,27 +46,16 @@ *

              Note: Before running this example, you must create a BigQuery dataset to contain your output * table. * - *

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

              {@code
              - *   --project=YOUR_PROJECT_ID
              - * }
              - * 
              - * and the BigQuery table for the output, with the form + *

              To execute this pipeline locally, specify the BigQuery table for the output with the form: *

              {@code
                *   --output=YOUR_PROJECT_ID:DATASET_ID.TABLE_ID
                * }
              * - *

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

              {@code
              - *   --project=YOUR_PROJECT_ID
              - *   --tempLocation=gs://YOUR_TEMP_DIRECTORY
              - *   --runner=BlockingDataflowRunner
              - * }
              - * 
              - * and the BigQuery table for the output: + *

              To change the runner, specify: *

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

              The BigQuery input table defaults to {@code clouddataflow-readonly:samples.weather_stations } * and can be overridden with {@code --input}. 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 263054138dd64..68d4d329f302c 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 @@ -73,15 +73,13 @@ * 4. Combining late data and speculative estimates * * - *

              Before running this example, it will be useful to familiarize yourself with Dataflow triggers + *

              Before running this example, it will be useful to familiarize yourself with Beam triggers * and understand the concept of 'late data', - * See: - * https://cloud.google.com/dataflow/model/triggers and - * - * https://cloud.google.com/dataflow/model/windowing#Advanced + * See: + * http://beam.incubator.apache.org/use/walkthroughs/ * *

              The example is configured to use the default BigQuery table from the example common package - * (there are no defaults for a general Dataflow pipeline). + * (there are no defaults for a general Beam pipeline). * You can override them by using the {@code --bigQueryDataset}, and {@code --bigQueryTable} * options. If the BigQuery table do not exist, the example will try to create them. * @@ -155,7 +153,7 @@ public class TriggerExample { * 5 | 60 | 10:27:20 | 10:27:25 * 5 | 60 | 10:29:00 | 11:11:00 * - *

              Dataflow tracks a watermark which records up to what point in event time the data is + *

              Beam tracks a watermark which records up to what point in event time the data is * complete. For the purposes of the example, we'll assume the watermark is approximately 15m * behind the current processing time. In practice, the actual value would vary over time based * on the systems knowledge of the current delay and contents of the backlog (data @@ -176,7 +174,7 @@ static class CalculateTotalFlow public PCollectionList apply(PCollection> flowInfo) { // Concept #1: The default triggering behavior - // By default Dataflow uses a trigger which fires when the watermark has passed the end of the + // By default Beam uses a trigger which fires when the watermark has passed the end of the // window. This would be written {@code Repeatedly.forever(AfterWatermark.pastEndOfWindow())}. // The system also defaults to dropping late data -- data which arrives after the watermark @@ -459,7 +457,7 @@ public static void main(String[] args) throws Exception { PipelineResult result = pipeline.run(); - // dataflowUtils will try to cancel the pipeline and the injector before the program exits. + // ExampleUtils will try to cancel the pipeline and the injector before the program exits. exampleUtils.waitToFinish(result); } From 3f48566618552c4b0fa026aa3a75ef6f1875da63 Mon Sep 17 00:00:00 2001 From: Dan Halperin Date: Wed, 24 Aug 2016 22:35:26 -0700 Subject: [PATCH 104/346] FluentBackoff: a replacement for a variety of custom backoff implementations We have 3 different backoff classes, which don't really have that much different functionality. Add a single, flexible backoff implementation that can be used to replace all three classes. Additionally, this new backoff actually supports more functionality than any of the other three did -- you can limit retries, cap the exponential growth of an individual backoff, and cap the cumulative time spent in backoff; prior implementations did not allow all 3. This also makes the parameters self-obvious (Duration, not number-that-is-also-millis) where appropriate. This initial PR should have no functional changes. * Implement FluentBackoff * Replace other custom BackOff implementations with FluentBackoff --- .../beam/examples/common/ExampleUtils.java | 7 +- .../runners/dataflow/DataflowPipelineJob.java | 72 +++--- .../runners/dataflow/util/PackageUtil.java | 31 ++- .../dataflow/DataflowPipelineJobTest.java | 32 ++- .../io/BoundedReadFromUnboundedSource.java | 8 +- ...temptAndTimeBoundedExponentialBackOff.java | 172 ------------- .../AttemptBoundedExponentialBackOff.java | 85 ------- .../apache/beam/sdk/util/FluentBackoff.java | 229 ++++++++++++++++++ .../org/apache/beam/sdk/util/GcsUtil.java | 18 +- .../IntervalBoundedExponentialBackOff.java | 1 + ...tAndTimeBoundedExponentialBackOffTest.java | 212 ---------------- .../AttemptBoundedExponentialBackOffTest.java | 84 ------- .../beam/sdk/util/FluentBackoffTest.java | 226 +++++++++++++++++ .../org/apache/beam/sdk/util/GcsUtilTest.java | 10 +- .../beam/sdk/io/gcp/bigquery/BigQueryIO.java | 21 +- .../io/gcp/bigquery/BigQueryServicesImpl.java | 152 ++++++------ .../bigquery/BigQueryTableRowIterator.java | 5 +- .../sdk/io/gcp/datastore/DatastoreV1.java | 26 +- .../bigquery/BigQueryServicesImplTest.java | 17 +- .../beam/sdk/io/gcp/datastore/V1TestUtil.java | 9 +- 20 files changed, 675 insertions(+), 742 deletions(-) delete mode 100644 sdks/java/core/src/main/java/org/apache/beam/sdk/util/AttemptAndTimeBoundedExponentialBackOff.java delete mode 100644 sdks/java/core/src/main/java/org/apache/beam/sdk/util/AttemptBoundedExponentialBackOff.java create mode 100644 sdks/java/core/src/main/java/org/apache/beam/sdk/util/FluentBackoff.java delete mode 100644 sdks/java/core/src/test/java/org/apache/beam/sdk/util/AttemptAndTimeBoundedExponentialBackOffTest.java delete mode 100644 sdks/java/core/src/test/java/org/apache/beam/sdk/util/AttemptBoundedExponentialBackOffTest.java create mode 100644 sdks/java/core/src/test/java/org/apache/beam/sdk/util/FluentBackoffTest.java diff --git a/examples/java/src/main/java/org/apache/beam/examples/common/ExampleUtils.java b/examples/java/src/main/java/org/apache/beam/examples/common/ExampleUtils.java index eadb580a257c4..2e8dcf696887a 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/common/ExampleUtils.java +++ b/examples/java/src/main/java/org/apache/beam/examples/common/ExampleUtils.java @@ -45,8 +45,9 @@ 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.AttemptBoundedExponentialBackOff; +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, @@ -79,7 +80,9 @@ public ExampleUtils(PipelineOptions options) { */ public void setup() throws IOException { Sleeper sleeper = Sleeper.DEFAULT; - BackOff backOff = new AttemptBoundedExponentialBackOff(3, 200); + BackOff backOff = + FluentBackoff.DEFAULT + .withMaxRetries(3).withInitialBackoff(Duration.millis(200)).backoff(); Throwable lastException = null; try { do { 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 9a515fa6140f3..dad59f2797412 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 @@ -34,7 +34,6 @@ import java.net.SocketTimeoutException; import java.util.List; import java.util.Map; -import java.util.concurrent.TimeUnit; import javax.annotation.Nullable; import org.apache.beam.runners.dataflow.internal.DataflowAggregatorTransforms; import org.apache.beam.runners.dataflow.internal.DataflowMetricUpdateExtractor; @@ -44,8 +43,7 @@ import org.apache.beam.sdk.AggregatorValues; import org.apache.beam.sdk.PipelineResult; import org.apache.beam.sdk.transforms.Aggregator; -import org.apache.beam.sdk.util.AttemptAndTimeBoundedExponentialBackOff; -import org.apache.beam.sdk.util.AttemptBoundedExponentialBackOff; +import org.apache.beam.sdk.util.FluentBackoff; import org.joda.time.Duration; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -95,14 +93,27 @@ public class DataflowPipelineJob implements PipelineResult { /** * The polling interval for job status and messages information. */ - static final long MESSAGES_POLLING_INTERVAL = TimeUnit.SECONDS.toMillis(2); - static final long STATUS_POLLING_INTERVAL = TimeUnit.SECONDS.toMillis(2); + static final Duration MESSAGES_POLLING_INTERVAL = Duration.standardSeconds(2); + static final Duration STATUS_POLLING_INTERVAL = Duration.standardSeconds(2); + + static final double DEFAULT_BACKOFF_EXPONENT = 1.5; /** - * The amount of polling attempts for job status and messages information. + * The amount of polling retries for job status and messages information. */ - static final int MESSAGES_POLLING_ATTEMPTS = 12; - static final int STATUS_POLLING_ATTEMPTS = 5; + static final int MESSAGES_POLLING_RETRIES = 11; + static final int STATUS_POLLING_RETRIES = 4; + + private static final FluentBackoff MESSAGES_BACKOFF_FACTORY = + FluentBackoff.DEFAULT + .withInitialBackoff(MESSAGES_POLLING_INTERVAL) + .withMaxRetries(MESSAGES_POLLING_RETRIES) + .withExponent(DEFAULT_BACKOFF_EXPONENT); + protected static final FluentBackoff STATUS_BACKOFF_FACTORY = + FluentBackoff.DEFAULT + .withInitialBackoff(STATUS_POLLING_INTERVAL) + .withMaxRetries(STATUS_POLLING_RETRIES) + .withExponent(DEFAULT_BACKOFF_EXPONENT); /** * Constructs the job. @@ -214,21 +225,23 @@ State waitUntilFinish( MonitoringUtil monitor = new MonitoringUtil(projectId, dataflowOptions.getDataflowClient()); long lastTimestamp = 0; - BackOff backoff = - duration.getMillis() > 0 - ? new AttemptAndTimeBoundedExponentialBackOff( - MESSAGES_POLLING_ATTEMPTS, - MESSAGES_POLLING_INTERVAL, - duration.getMillis(), - AttemptAndTimeBoundedExponentialBackOff.ResetPolicy.ATTEMPTS, - nanoClock) - : new AttemptBoundedExponentialBackOff( - MESSAGES_POLLING_ATTEMPTS, MESSAGES_POLLING_INTERVAL); + BackOff backoff; + if (!duration.isLongerThan(Duration.ZERO)) { + backoff = MESSAGES_BACKOFF_FACTORY.backoff(); + } else { + backoff = MESSAGES_BACKOFF_FACTORY.withMaxCumulativeBackoff(duration).backoff(); + } + + // This function tracks the cumulative time from the *first request* to enforce the wall-clock + // limit. Any backoff instance could, at best, track the the time since the first attempt at a + // given request. Thus, we need to track the cumulative time ourselves. + long startNanos = nanoClock.nanoTime(); + State state; do { // Get the state of the job before listing messages. This ensures we always fetch job // messages after the job finishes to ensure we have all them. - state = getStateWithRetries(1, sleeper); + state = getStateWithRetries(STATUS_BACKOFF_FACTORY.withMaxRetries(0).backoff(), sleeper); boolean hasError = state == State.UNKNOWN; if (messageHandler != null && !hasError) { @@ -250,7 +263,16 @@ State waitUntilFinish( } if (!hasError) { + // Reset the backoff. backoff.reset(); + // If duration is set, update the new cumulative sleep time to be the remaining + // part of the total input sleep duration. + if (duration.isLongerThan(Duration.ZERO)) { + long nanosConsumed = nanoClock.nanoTime() - startNanos; + Duration consumed = Duration.millis((nanosConsumed + 999999) / 1000000); + backoff = + MESSAGES_BACKOFF_FACTORY.withMaxCumulativeBackoff(duration.minus(consumed)).backoff(); + } // Check if the job is done. if (state.isTerminal()) { return state; @@ -287,7 +309,7 @@ public State getState() { return terminalState; } - return getStateWithRetries(STATUS_POLLING_ATTEMPTS, Sleeper.DEFAULT); + return getStateWithRetries(STATUS_BACKOFF_FACTORY.backoff(), Sleeper.DEFAULT); } /** @@ -299,7 +321,7 @@ public State getState() { * @return The state of the job or State.UNKNOWN in case of failure. */ @VisibleForTesting - State getStateWithRetries(int attempts, Sleeper sleeper) { + State getStateWithRetries(BackOff attempts, Sleeper sleeper) { if (terminalState != null) { return terminalState; } @@ -318,17 +340,13 @@ State getStateWithRetries(int attempts, Sleeper sleeper) { * Attempts to get the underlying {@link Job}. Uses exponential backoff on failure up to the * maximum number of passed in attempts. * - * @param attempts The amount of attempts to make. + * @param backoff the {@link BackOff} used to control retries. * @param sleeper Object used to do the sleeps between attempts. * @return The underlying {@link Job} object. * @throws IOException When the maximum number of retries is exhausted, the last exception is * thrown. */ - @VisibleForTesting - Job getJobWithRetries(int attempts, Sleeper sleeper) throws IOException { - AttemptBoundedExponentialBackOff backoff = - new AttemptBoundedExponentialBackOff(attempts, STATUS_POLLING_INTERVAL); - + private Job getJobWithRetries(BackOff backoff, Sleeper sleeper) throws IOException { // Retry loop ends in return or throw while (true) { try { diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/PackageUtil.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/PackageUtil.java index bf1f666c32c5e..6d910baba3724 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/PackageUtil.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/PackageUtil.java @@ -18,7 +18,7 @@ package org.apache.beam.runners.dataflow.util; import com.fasterxml.jackson.core.Base64Variants; -import com.google.api.client.util.BackOffUtils; +import com.google.api.client.util.BackOff; import com.google.api.client.util.Sleeper; import com.google.api.services.dataflow.model.DataflowPackage; import com.google.cloud.hadoop.util.ApiErrorExtractor; @@ -37,10 +37,11 @@ import java.util.Collection; import java.util.List; import java.util.Objects; -import org.apache.beam.sdk.util.AttemptBoundedExponentialBackOff; +import org.apache.beam.sdk.util.FluentBackoff; import org.apache.beam.sdk.util.IOChannelUtils; import org.apache.beam.sdk.util.MimeTypes; import org.apache.beam.sdk.util.ZipFiles; +import org.joda.time.Duration; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -54,11 +55,15 @@ public class PackageUtil { /** * The initial interval to use between package staging attempts. */ - private static final long INITIAL_BACKOFF_INTERVAL_MS = 5000L; + private static final Duration INITIAL_BACKOFF_INTERVAL = Duration.standardSeconds(5); /** - * The maximum number of attempts when staging a file. + * The maximum number of retries when staging a file. */ - private static final int MAX_ATTEMPTS = 5; + private static final int MAX_RETRIES = 4; + + private static final FluentBackoff BACKOFF_FACTORY = + FluentBackoff.DEFAULT + .withMaxRetries(MAX_RETRIES).withInitialBackoff(INITIAL_BACKOFF_INTERVAL); /** * Translates exceptions from API calls. @@ -199,9 +204,7 @@ static List stageClasspathElements( } // Upload file, retrying on failure. - AttemptBoundedExponentialBackOff backoff = new AttemptBoundedExponentialBackOff( - MAX_ATTEMPTS, - INITIAL_BACKOFF_INTERVAL_MS); + BackOff backoff = BACKOFF_FACTORY.backoff(); while (true) { try { LOG.debug("Uploading classpath element {} to {}", classpathElement, target); @@ -219,15 +222,17 @@ static List stageClasspathElements( + "'gcloud auth login'.", classpathElement, target); LOG.error(errorMessage); throw new IOException(errorMessage, e); - } else if (!backoff.atMaxAttempts()) { - LOG.warn("Upload attempt failed, sleeping before retrying staging of classpath: {}", - classpathElement, e); - BackOffUtils.next(retrySleeper, backoff); - } else { + } + long sleep = backoff.nextBackOffMillis(); + if (sleep == BackOff.STOP) { // Rethrow last error, to be included as a cause in the catch below. LOG.error("Upload failed, will NOT retry staging of classpath: {}", classpathElement, e); throw e; + } else { + LOG.warn("Upload attempt failed, sleeping before retrying staging of classpath: {}", + classpathElement, e); + retrySleeper.sleep(sleep); } } } 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 22b5400b0fdf4..226140a672451 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 @@ -60,7 +60,6 @@ import org.apache.beam.sdk.transforms.Combine.CombineFn; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.Sum; -import org.apache.beam.sdk.util.AttemptBoundedExponentialBackOff; import org.apache.beam.sdk.values.PInput; import org.apache.beam.sdk.values.POutput; import org.joda.time.Duration; @@ -111,22 +110,21 @@ public void setup() { * AttemptBoundedExponentialBackOff given the number of retries and * an initial polling interval. * - * @param pollingIntervalMillis The initial polling interval given. - * @param attempts The number of attempts made + * @param pollingInterval The initial polling interval given. + * @param retries The number of retries made * @param timeSleptMillis The amount of time slept by the clock. This is checked * against the valid interval. */ - void checkValidInterval(long pollingIntervalMillis, int attempts, long timeSleptMillis) { + private void checkValidInterval(Duration pollingInterval, int retries, long timeSleptMillis) { long highSum = 0; long lowSum = 0; - for (int i = 1; i < attempts; i++) { + for (int i = 0; i < retries; i++) { double currentInterval = - pollingIntervalMillis - * Math.pow(AttemptBoundedExponentialBackOff.DEFAULT_MULTIPLIER, i - 1); - double offset = - AttemptBoundedExponentialBackOff.DEFAULT_RANDOMIZATION_FACTOR * currentInterval; - highSum += Math.round(currentInterval + offset); - lowSum += Math.round(currentInterval - offset); + pollingInterval.getMillis() + * Math.pow(DataflowPipelineJob.DEFAULT_BACKOFF_EXPONENT, i); + double randomOffset = 0.5 * currentInterval; + highSum += Math.round(currentInterval + randomOffset); + lowSum += Math.round(currentInterval - randomOffset); } assertThat(timeSleptMillis, allOf(greaterThanOrEqualTo(lowSum), lessThanOrEqualTo(highSum))); } @@ -228,7 +226,7 @@ public void testWaitToFinishFail() throws Exception { assertEquals(null, state); long timeDiff = TimeUnit.NANOSECONDS.toMillis(fastClock.nanoTime() - startTime); checkValidInterval(DataflowPipelineJob.MESSAGES_POLLING_INTERVAL, - DataflowPipelineJob.MESSAGES_POLLING_ATTEMPTS, timeDiff); + DataflowPipelineJob.MESSAGES_POLLING_RETRIES, timeDiff); } @Test @@ -246,8 +244,8 @@ public void testWaitToFinishTimeFail() throws Exception { State state = job.waitUntilFinish(Duration.millis(4), null, fastClock, fastClock); assertEquals(null, state); long timeDiff = TimeUnit.NANOSECONDS.toMillis(fastClock.nanoTime() - startTime); - // Should only sleep for the 4 ms remaining. - assertEquals(timeDiff, 4L); + // Should only have slept for the 4 ms allowed. + assertEquals(4L, timeDiff); } @Test @@ -268,7 +266,7 @@ public void testGetStateReturnsServiceState() throws Exception { assertEquals( State.RUNNING, - job.getStateWithRetries(DataflowPipelineJob.STATUS_POLLING_ATTEMPTS, fastClock)); + job.getStateWithRetries(DataflowPipelineJob.STATUS_BACKOFF_FACTORY.backoff(), fastClock)); } @Test @@ -286,10 +284,10 @@ public void testGetStateWithExceptionReturnsUnknown() throws Exception { long startTime = fastClock.nanoTime(); assertEquals( State.UNKNOWN, - job.getStateWithRetries(DataflowPipelineJob.STATUS_POLLING_ATTEMPTS, fastClock)); + job.getStateWithRetries(DataflowPipelineJob.STATUS_BACKOFF_FACTORY.backoff(), fastClock)); long timeDiff = TimeUnit.NANOSECONDS.toMillis(fastClock.nanoTime() - startTime); checkValidInterval(DataflowPipelineJob.STATUS_POLLING_INTERVAL, - DataflowPipelineJob.STATUS_POLLING_ATTEMPTS, timeDiff); + DataflowPipelineJob.STATUS_POLLING_RETRIES, timeDiff); } @Test 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 28d77468bbc0e..b41c655c31bc1 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 @@ -34,7 +34,7 @@ import org.apache.beam.sdk.transforms.RemoveDuplicates; import org.apache.beam.sdk.transforms.SerializableFunction; import org.apache.beam.sdk.transforms.display.DisplayData; -import org.apache.beam.sdk.util.IntervalBoundedExponentialBackOff; +import org.apache.beam.sdk.util.FluentBackoff; import org.apache.beam.sdk.util.ValueWithRecordId; import org.apache.beam.sdk.values.PBegin; import org.apache.beam.sdk.values.PCollection; @@ -52,6 +52,10 @@ class BoundedReadFromUnboundedSource extends PTransform source; private final long maxNumRecords; private final Duration maxReadTime; + private static final FluentBackoff BACKOFF_FACTORY = + FluentBackoff.DEFAULT + .withInitialBackoff(Duration.millis(10)) + .withMaxBackoff(Duration.standardSeconds(10)); /** * Returns a new {@link BoundedReadFromUnboundedSource} that reads a bounded amount @@ -241,7 +245,7 @@ public boolean advance() throws IOException { private boolean advanceWithBackoff() throws IOException { // Try reading from the source with exponential backoff - BackOff backoff = new IntervalBoundedExponentialBackOff(10000L, 10L); + BackOff backoff = BACKOFF_FACTORY.backoff(); long nextSleep = backoff.nextBackOffMillis(); while (nextSleep != BackOff.STOP) { if (endTime != null && Instant.now().isAfter(endTime)) { diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/AttemptAndTimeBoundedExponentialBackOff.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/AttemptAndTimeBoundedExponentialBackOff.java deleted file mode 100644 index d8050e0a32198..0000000000000 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/AttemptAndTimeBoundedExponentialBackOff.java +++ /dev/null @@ -1,172 +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; - -import static com.google.common.base.Preconditions.checkArgument; - -import com.google.api.client.util.BackOff; -import com.google.api.client.util.NanoClock; -import edu.umd.cs.findbugs.annotations.SuppressFBWarnings; -import java.util.concurrent.TimeUnit; - -/** - * Extension of {@link AttemptBoundedExponentialBackOff} that bounds the total time that the backoff - * is happening as well as the amount of retries. Acts exactly as a AttemptBoundedExponentialBackOff - * unless the time interval has expired since the object was created. At this point, it will always - * return BackOff.STOP. Calling reset() resets both the timer and the number of retry attempts, - * unless a custom ResetPolicy (ResetPolicy.ATTEMPTS or ResetPolicy.TIMER) is passed to the - * constructor. - * - *

              Implementation is not thread-safe. - */ -public class AttemptAndTimeBoundedExponentialBackOff extends AttemptBoundedExponentialBackOff { - private long endTimeMillis; - private long maximumTotalWaitTimeMillis; - private ResetPolicy resetPolicy; - private final NanoClock nanoClock; - // NanoClock.SYSTEM has a max elapsed time of 292 years or 2^63 ns. Here, we choose 2^53 ns as - // a smaller but still huge limit. - private static final long MAX_ELAPSED_TIME_MILLIS = 1L << 53; - - /** - * A ResetPolicy controls the behavior of this BackOff when reset() is called. By default, both - * the number of attempts and the time bound for the BackOff are reset, but an alternative - * ResetPolicy may be set to only reset one of these two. - */ - public static enum ResetPolicy { - ALL, - ATTEMPTS, - TIMER - } - - /** - * Constructs an instance of AttemptAndTimeBoundedExponentialBackoff. - * - * @param maximumNumberOfAttempts The maximum number of attempts it will make. - * @param initialIntervalMillis The original interval to wait between attempts in milliseconds. - * @param maximumTotalWaitTimeMillis The maximum total time that this object will - * allow more attempts in milliseconds. - */ - public AttemptAndTimeBoundedExponentialBackOff( - int maximumNumberOfAttempts, long initialIntervalMillis, long maximumTotalWaitTimeMillis) { - this( - maximumNumberOfAttempts, - initialIntervalMillis, - maximumTotalWaitTimeMillis, - ResetPolicy.ALL, - NanoClock.SYSTEM); - } - - /** - * Constructs an instance of AttemptAndTimeBoundedExponentialBackoff. - * - * @param maximumNumberOfAttempts The maximum number of attempts it will make. - * @param initialIntervalMillis The original interval to wait between attempts in milliseconds. - * @param maximumTotalWaitTimeMillis The maximum total time that this object will - * allow more attempts in milliseconds. - * @param resetPolicy The ResetPolicy specifying the properties of this BackOff that are subject - * to being reset. - */ - public AttemptAndTimeBoundedExponentialBackOff( - int maximumNumberOfAttempts, - long initialIntervalMillis, - long maximumTotalWaitTimeMillis, - ResetPolicy resetPolicy) { - this( - maximumNumberOfAttempts, - initialIntervalMillis, - maximumTotalWaitTimeMillis, - resetPolicy, - NanoClock.SYSTEM); - } - - /** - * Constructs an instance of AttemptAndTimeBoundedExponentialBackoff. - * - * @param maximumNumberOfAttempts The maximum number of attempts it will make. - * @param initialIntervalMillis The original interval to wait between attempts in milliseconds. - * @param maximumTotalWaitTimeMillis The maximum total time that this object will - * allow more attempts in milliseconds. - * @param resetPolicy The ResetPolicy specifying the properties of this BackOff that are subject - * to being reset. - * @param nanoClock clock used to measure the time that has passed. - */ - public AttemptAndTimeBoundedExponentialBackOff( - int maximumNumberOfAttempts, - long initialIntervalMillis, - long maximumTotalWaitTimeMillis, - ResetPolicy resetPolicy, - NanoClock nanoClock) { - super(maximumNumberOfAttempts, initialIntervalMillis); - checkArgument( - maximumTotalWaitTimeMillis > 0, "Maximum total wait time must be greater than zero."); - checkArgument( - maximumTotalWaitTimeMillis < MAX_ELAPSED_TIME_MILLIS, - "Maximum total wait time must be less than " + MAX_ELAPSED_TIME_MILLIS + " milliseconds"); - checkArgument(resetPolicy != null, "resetPolicy may not be null"); - checkArgument(nanoClock != null, "nanoClock may not be null"); - this.maximumTotalWaitTimeMillis = maximumTotalWaitTimeMillis; - this.resetPolicy = resetPolicy; - this.nanoClock = nanoClock; - // Set the end time for this BackOff. Note that we cannot simply call reset() here since the - // resetPolicy may not be set to reset the time bound. - endTimeMillis = getTimeMillis() + maximumTotalWaitTimeMillis; - } - - @Override - @SuppressFBWarnings(value = "UR_UNINIT_READ_CALLED_FROM_SUPER_CONSTRUCTOR", - justification = "Explicitly handled in implementation.") - public void reset() { - // reset() is called in the constructor of the parent class before resetPolicy and nanoClock are - // set. In this case, we call the parent class's reset() method and return. - if (resetPolicy == null) { - super.reset(); - return; - } - // Reset the number of attempts. - if (resetPolicy == ResetPolicy.ALL || resetPolicy == ResetPolicy.ATTEMPTS) { - super.reset(); - } - // Reset the time bound. - if (resetPolicy == ResetPolicy.ALL || resetPolicy == ResetPolicy.TIMER) { - endTimeMillis = getTimeMillis() + maximumTotalWaitTimeMillis; - } - } - - public void setEndtimeMillis(long endTimeMillis) { - this.endTimeMillis = endTimeMillis; - } - - @Override - public long nextBackOffMillis() { - if (atMaxAttempts()) { - return BackOff.STOP; - } - long backoff = Math.min(super.nextBackOffMillis(), endTimeMillis - getTimeMillis()); - return (backoff > 0 ? backoff : BackOff.STOP); - } - - private long getTimeMillis() { - return TimeUnit.NANOSECONDS.toMillis(nanoClock.nanoTime()); - } - - @Override - public boolean atMaxAttempts() { - return super.atMaxAttempts() || getTimeMillis() >= endTimeMillis; - } -} diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/AttemptBoundedExponentialBackOff.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/AttemptBoundedExponentialBackOff.java deleted file mode 100644 index 5707293bad06d..0000000000000 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/AttemptBoundedExponentialBackOff.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 org.apache.beam.sdk.util; - -import static com.google.common.base.Preconditions.checkArgument; - -import com.google.api.client.util.BackOff; - - -/** - * Implementation of {@link BackOff} that increases the back off period for each retry attempt - * using a randomization function that grows exponentially. - * - *

              Example: The initial interval is .5 seconds and the maximum number of retries is 10. - * For 10 tries the sequence will be (values in seconds): - * - *

              - * retry#      retry_interval     randomized_interval
              - * 1             0.5                [0.25,   0.75]
              - * 2             0.75               [0.375,  1.125]
              - * 3             1.125              [0.562,  1.687]
              - * 4             1.687              [0.8435, 2.53]
              - * 5             2.53               [1.265,  3.795]
              - * 6             3.795              [1.897,  5.692]
              - * 7             5.692              [2.846,  8.538]
              - * 8             8.538              [4.269, 12.807]
              - * 9            12.807              [6.403, 19.210]
              - * 10           {@link BackOff#STOP}
              - * 
              - * - *

              Implementation is not thread-safe. - */ -public class AttemptBoundedExponentialBackOff implements BackOff { - public static final double DEFAULT_MULTIPLIER = 1.5; - public static final double DEFAULT_RANDOMIZATION_FACTOR = 0.5; - private final int maximumNumberOfAttempts; - private final long initialIntervalMillis; - private int currentAttempt; - - public AttemptBoundedExponentialBackOff(int maximumNumberOfAttempts, long initialIntervalMillis) { - checkArgument(maximumNumberOfAttempts > 0, - "Maximum number of attempts must be greater than zero."); - checkArgument(initialIntervalMillis > 0, "Initial interval must be greater than zero."); - this.maximumNumberOfAttempts = maximumNumberOfAttempts; - this.initialIntervalMillis = initialIntervalMillis; - reset(); - } - - @Override - public void reset() { - currentAttempt = 1; - } - - @Override - public long nextBackOffMillis() { - if (currentAttempt >= maximumNumberOfAttempts) { - return BackOff.STOP; - } - double currentIntervalMillis = initialIntervalMillis - * Math.pow(DEFAULT_MULTIPLIER, currentAttempt - 1); - double randomOffset = (Math.random() * 2 - 1) - * DEFAULT_RANDOMIZATION_FACTOR * currentIntervalMillis; - currentAttempt += 1; - return Math.round(currentIntervalMillis + randomOffset); - } - - public boolean atMaxAttempts() { - return currentAttempt >= maximumNumberOfAttempts; - } -} diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/FluentBackoff.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/FluentBackoff.java new file mode 100644 index 0000000000000..479d7a81451b3 --- /dev/null +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/FluentBackoff.java @@ -0,0 +1,229 @@ +/* + * 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 com.google.api.client.util.BackOff; +import com.google.common.base.MoreObjects; +import org.joda.time.Duration; + +/** + * A fluent builder for {@link BackOff} objects that allows customization of the retry algorithm. + * + * @see #DEFAULT for the default configuration parameters. + */ +public final class FluentBackoff { + + private static final double DEFAULT_EXPONENT = 1.5; + private static final double DEFAULT_RANDOMIZATION_FACTOR = 0.5; + private static final Duration DEFAULT_MIN_BACKOFF = Duration.standardSeconds(1); + private static final Duration DEFAULT_MAX_BACKOFF = Duration.standardDays(1000); + private static final int DEFAULT_MAX_RETRIES = Integer.MAX_VALUE; + private static final Duration DEFAULT_MAX_CUM_BACKOFF = Duration.standardDays(1000); + + private final double exponent; + private final Duration initialBackoff; + private final Duration maxBackoff; + private final Duration maxCumulativeBackoff; + private final int maxRetries; + + /** + * By default the {@link BackOff} created by this builder will use exponential backoff (base + * exponent 1.5) with an initial backoff of 1 second. These parameters can be overridden with + * {@link #withExponent(double)} and {@link #withInitialBackoff(Duration)}, + * respectively, and the maximum backoff after exponential increase can be capped using {@link + * FluentBackoff#withMaxBackoff(Duration)}. + * + *

              The default {@link BackOff} does not limit the number of retries. To limit the backoff, the + * maximum total number of retries can be set using {@link #withMaxRetries(int)}. The + * total time spent in backoff can be time-bounded as well by configuring {@link + * #withMaxCumulativeBackoff(Duration)}. If either of these limits are reached, calls + * to {@link BackOff#nextBackOffMillis()} will return {@link BackOff#STOP} to signal that no more + * retries should continue. + */ + public static final FluentBackoff DEFAULT = new FluentBackoff( + DEFAULT_EXPONENT, + DEFAULT_MIN_BACKOFF, DEFAULT_MAX_BACKOFF, DEFAULT_MAX_CUM_BACKOFF, + DEFAULT_MAX_RETRIES); + + /** + * Instantiates a {@link BackOff} that will obey the current configuration. + * + * @see FluentBackoff + */ + public BackOff backoff() { + return new BackoffImpl(this); + } + + /** + * Returns a copy of this {@link FluentBackoff} that instead uses the specified exponent to + * control the exponential growth of delay. + * + *

              Does not modify this object. + * + * @see FluentBackoff + */ + public FluentBackoff withExponent(double exponent) { + checkArgument(exponent > 0, "exponent %s must be greater than 0", exponent); + return new FluentBackoff( + exponent, initialBackoff, maxBackoff, maxCumulativeBackoff, maxRetries); + } + + /** + * Returns a copy of this {@link FluentBackoff} that instead uses the specified initial backoff + * duration. + * + *

              Does not modify this object. + * + * @see FluentBackoff + */ + public FluentBackoff withInitialBackoff(Duration initialBackoff) { + checkArgument( + initialBackoff.isLongerThan(Duration.ZERO), + "initialBackoff %s must be at least 1 millisecond", + initialBackoff); + return new FluentBackoff( + exponent, initialBackoff, maxBackoff, maxCumulativeBackoff, maxRetries); + } + + /** + * Returns a copy of this {@link FluentBackoff} that limits the maximum backoff of an individual + * attempt to the specified duration. + * + *

              Does not modify this object. + * + * @see FluentBackoff + */ + public FluentBackoff withMaxBackoff(Duration maxBackoff) { + checkArgument( + maxBackoff.getMillis() > 0, + "maxBackoff %s must be at least 1 millisecond", + maxBackoff); + return new FluentBackoff( + exponent, initialBackoff, maxBackoff, maxCumulativeBackoff, maxRetries); + } + + /** + * Returns a copy of this {@link FluentBackoff} that limits the total time spent in backoff + * returned across all calls to {@link BackOff#nextBackOffMillis()}. + * + *

              Does not modify this object. + * + * @see FluentBackoff + */ + public FluentBackoff withMaxCumulativeBackoff(Duration maxCumulativeBackoff) { + checkArgument(maxCumulativeBackoff.isLongerThan(Duration.ZERO), + "maxCumulativeBackoff %s must be at least 1 millisecond", maxCumulativeBackoff); + return new FluentBackoff( + exponent, initialBackoff, maxBackoff, maxCumulativeBackoff, maxRetries); + } + + /** + * Returns a copy of this {@link FluentBackoff} that limits the total number of retries, aka + * the total number of calls to {@link BackOff#nextBackOffMillis()} before returning + * {@link BackOff#STOP}. + * + *

              Does not modify this object. + * + * @see FluentBackoff + */ + public FluentBackoff withMaxRetries(int maxRetries) { + checkArgument(maxRetries >= 0, "maxRetries %s cannot be negative", maxRetries); + return new FluentBackoff( + exponent, initialBackoff, maxBackoff, maxCumulativeBackoff, maxRetries); + } + + public String toString() { + return MoreObjects.toStringHelper(FluentBackoff.class) + .add("exponent", exponent) + .add("initialBackoff", initialBackoff) + .add("maxBackoff", maxBackoff) + .add("maxRetries", maxRetries) + .add("maxCumulativeBackoff", maxCumulativeBackoff) + .toString(); + } + + private static class BackoffImpl implements BackOff { + + // Customization of this backoff. + private final FluentBackoff backoffConfig; + // Current state + private Duration currentCumulativeBackoff; + private int currentRetry; + + @Override + public void reset() { + currentRetry = 0; + currentCumulativeBackoff = Duration.ZERO; + } + + @Override + public long nextBackOffMillis() { + // Maximum number of retries reached. + if (currentRetry >= backoffConfig.maxRetries) { + return BackOff.STOP; + } + // Maximum cumulative backoff reached. + if (currentCumulativeBackoff.compareTo(backoffConfig.maxCumulativeBackoff) >= 0) { + return BackOff.STOP; + } + + double currentIntervalMillis = + Math.min( + backoffConfig.initialBackoff.getMillis() + * Math.pow(backoffConfig.exponent, currentRetry), + backoffConfig.maxBackoff.getMillis()); + double randomOffset = + (Math.random() * 2 - 1) * DEFAULT_RANDOMIZATION_FACTOR * currentIntervalMillis; + long nextBackoffMillis = Math.round(currentIntervalMillis + randomOffset); + // Cap to limit on cumulative backoff + Duration remainingCumulative = + backoffConfig.maxCumulativeBackoff.minus(currentCumulativeBackoff); + nextBackoffMillis = Math.min(nextBackoffMillis, remainingCumulative.getMillis()); + + // Update state and return backoff. + currentCumulativeBackoff = currentCumulativeBackoff.plus(nextBackoffMillis); + currentRetry += 1; + return nextBackoffMillis; + } + + private BackoffImpl(FluentBackoff backoffConfig) { + this.backoffConfig = backoffConfig; + this.reset(); + } + + public String toString() { + return MoreObjects.toStringHelper(BackoffImpl.class) + .add("backoffConfig", backoffConfig) + .add("currentRetry", currentRetry) + .add("currentCumulativeBackoff", currentCumulativeBackoff) + .toString(); + } + } + + private FluentBackoff( + double exponent, Duration initialBackoff, Duration maxBackoff, Duration maxCumulativeBackoff, + int maxRetries) { + this.exponent = exponent; + this.initialBackoff = initialBackoff; + this.maxBackoff = maxBackoff; + this.maxRetries = maxRetries; + this.maxCumulativeBackoff = maxCumulativeBackoff; + } +} 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 44a182eeb3d8f..41c372e9bc0c5 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 @@ -66,6 +66,7 @@ import org.apache.beam.sdk.options.GcsOptions; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.util.gcsfs.GcsPath; +import org.joda.time.Duration; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -119,6 +120,9 @@ public GcsUtil create(PipelineOptions options) { */ private static final int MAX_CONCURRENT_BATCHES = 256; + private static final FluentBackoff BACKOFF_FACTORY = + FluentBackoff.DEFAULT.withMaxRetries(3).withInitialBackoff(Duration.millis(200)); + ///////////////////////////////////////////////////////////////////////////// /** Client for the GCS API. */ @@ -177,7 +181,7 @@ public List expand(GcsPath gcsPattern) throws IOException { // the request has strong global consistency. ResilientOperation.retry( ResilientOperation.getGoogleRequestCallable(getObject), - new AttemptBoundedExponentialBackOff(3, 200), + BACKOFF_FACTORY.backoff(), RetryDeterminer.SOCKET_ERRORS, IOException.class); return ImmutableList.of(gcsPattern); @@ -216,7 +220,7 @@ public List expand(GcsPath gcsPattern) throws IOException { try { objects = ResilientOperation.retry( ResilientOperation.getGoogleRequestCallable(listObject), - new AttemptBoundedExponentialBackOff(3, 200), + BACKOFF_FACTORY.backoff(), RetryDeterminer.SOCKET_ERRORS, IOException.class); } catch (Exception e) { @@ -257,7 +261,10 @@ Integer getUploadBufferSizeBytes() { * if the resource does not exist. */ public long fileSize(GcsPath path) throws IOException { - return fileSize(path, new AttemptBoundedExponentialBackOff(4, 200), Sleeper.DEFAULT); + return fileSize( + path, + BACKOFF_FACTORY.backoff(), + Sleeper.DEFAULT); } /** @@ -335,7 +342,10 @@ public WritableByteChannel create(GcsPath path, * be accessible otherwise the permissions exception will be propagated. */ public boolean bucketExists(GcsPath path) throws IOException { - return bucketExists(path, new AttemptBoundedExponentialBackOff(4, 200), Sleeper.DEFAULT); + return bucketExists( + path, + BACKOFF_FACTORY.backoff(), + Sleeper.DEFAULT); } /** diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/IntervalBoundedExponentialBackOff.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/IntervalBoundedExponentialBackOff.java index 519776a550da9..6fac6dca67a69 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/IntervalBoundedExponentialBackOff.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/IntervalBoundedExponentialBackOff.java @@ -49,6 +49,7 @@ * *

              Implementation is not thread-safe. */ +@Deprecated public class IntervalBoundedExponentialBackOff implements BackOff { public static final double DEFAULT_MULTIPLIER = 1.5; public static final double DEFAULT_RANDOMIZATION_FACTOR = 0.5; diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/AttemptAndTimeBoundedExponentialBackOffTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/AttemptAndTimeBoundedExponentialBackOffTest.java deleted file mode 100644 index 59e0fb7dab284..0000000000000 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/AttemptAndTimeBoundedExponentialBackOffTest.java +++ /dev/null @@ -1,212 +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; - -import static org.hamcrest.MatcherAssert.assertThat; -import static org.hamcrest.Matchers.allOf; -import static org.hamcrest.Matchers.greaterThan; -import static org.hamcrest.Matchers.lessThan; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertTrue; - -import com.google.api.client.util.BackOff; -import org.apache.beam.sdk.testing.FastNanoClockAndSleeper; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.ExpectedException; -import org.junit.runner.RunWith; -import org.junit.runners.JUnit4; - -/** Unit tests for {@link AttemptAndTimeBoundedExponentialBackOff}. */ -@RunWith(JUnit4.class) -public class AttemptAndTimeBoundedExponentialBackOffTest { - @Rule public ExpectedException exception = ExpectedException.none(); - @Rule public FastNanoClockAndSleeper fastClock = new FastNanoClockAndSleeper(); - - @Test - public void testUsingInvalidInitialInterval() throws Exception { - exception.expect(IllegalArgumentException.class); - exception.expectMessage("Initial interval must be greater than zero."); - new AttemptAndTimeBoundedExponentialBackOff(10, 0L, 1000L); - } - - @Test - public void testUsingInvalidTimeInterval() throws Exception { - exception.expect(IllegalArgumentException.class); - exception.expectMessage("Maximum total wait time must be greater than zero."); - new AttemptAndTimeBoundedExponentialBackOff(10, 2L, 0L); - } - - @Test - public void testUsingInvalidMaximumNumberOfRetries() throws Exception { - exception.expect(IllegalArgumentException.class); - exception.expectMessage("Maximum number of attempts must be greater than zero."); - new AttemptAndTimeBoundedExponentialBackOff(-1, 10L, 1000L); - } - - @Test - public void testThatFixedNumberOfAttemptsExits() throws Exception { - BackOff backOff = - new AttemptAndTimeBoundedExponentialBackOff( - 3, - 500L, - 1000L, - AttemptAndTimeBoundedExponentialBackOff.ResetPolicy.ALL, - fastClock); - assertThat(backOff.nextBackOffMillis(), allOf(greaterThan(249L), lessThan(751L))); - assertThat(backOff.nextBackOffMillis(), allOf(greaterThan(374L), lessThan(1126L))); - assertEquals(BackOff.STOP, backOff.nextBackOffMillis()); - } - - @Test - public void testThatResettingAllowsReuse() throws Exception { - AttemptBoundedExponentialBackOff backOff = - new AttemptAndTimeBoundedExponentialBackOff( - 3, - 500, - 1000L, - AttemptAndTimeBoundedExponentialBackOff.ResetPolicy.ALL, - fastClock); - assertThat(backOff.nextBackOffMillis(), allOf(greaterThan(249L), lessThan(751L))); - assertThat(backOff.nextBackOffMillis(), allOf(greaterThan(374L), lessThan(1126L))); - assertEquals(BackOff.STOP, backOff.nextBackOffMillis()); - backOff.reset(); - assertThat(backOff.nextBackOffMillis(), allOf(greaterThan(249L), lessThan(751L))); - assertThat(backOff.nextBackOffMillis(), allOf(greaterThan(374L), lessThan(1126L))); - assertEquals(BackOff.STOP, backOff.nextBackOffMillis()); - - backOff = - new AttemptAndTimeBoundedExponentialBackOff( - 30, - 500, - 1000L, - AttemptAndTimeBoundedExponentialBackOff.ResetPolicy.ALL, - fastClock); - assertThat(backOff.nextBackOffMillis(), allOf(greaterThan(249L), lessThan(751L))); - assertThat(backOff.nextBackOffMillis(), allOf(greaterThan(374L), lessThan(1126L))); - fastClock.sleep(2000L); - assertEquals(BackOff.STOP, backOff.nextBackOffMillis()); - backOff.reset(); - assertThat(backOff.nextBackOffMillis(), allOf(greaterThan(249L), lessThan(751L))); - } - - @Test - public void testThatResettingAttemptsAllowsReuse() throws Exception { - AttemptBoundedExponentialBackOff backOff = - new AttemptAndTimeBoundedExponentialBackOff( - 3, - 500, - 1000, - AttemptAndTimeBoundedExponentialBackOff.ResetPolicy.ATTEMPTS, - fastClock); - assertThat(backOff.nextBackOffMillis(), allOf(greaterThan(249L), lessThan(751L))); - assertThat(backOff.nextBackOffMillis(), allOf(greaterThan(374L), lessThan(1126L))); - assertEquals(BackOff.STOP, backOff.nextBackOffMillis()); - backOff.reset(); - assertThat(backOff.nextBackOffMillis(), allOf(greaterThan(249L), lessThan(751L))); - assertThat(backOff.nextBackOffMillis(), allOf(greaterThan(374L), lessThan(1126L))); - assertEquals(BackOff.STOP, backOff.nextBackOffMillis()); - } - - @Test - public void testThatResettingAttemptsDoesNotAllowsReuse() throws Exception { - AttemptBoundedExponentialBackOff backOff = - new AttemptAndTimeBoundedExponentialBackOff( - 30, - 500, - 1000L, - AttemptAndTimeBoundedExponentialBackOff.ResetPolicy.ATTEMPTS, - fastClock); - assertThat(backOff.nextBackOffMillis(), allOf(greaterThan(249L), lessThan(751L))); - assertThat(backOff.nextBackOffMillis(), allOf(greaterThan(374L), lessThan(1126L))); - fastClock.sleep(2000L); - assertEquals(BackOff.STOP, backOff.nextBackOffMillis()); - backOff.reset(); - assertEquals(BackOff.STOP, backOff.nextBackOffMillis()); - } - - @Test - public void testThatResettingTimerAllowsReuse() throws Exception { - AttemptBoundedExponentialBackOff backOff = - new AttemptAndTimeBoundedExponentialBackOff( - 30, - 500, - 1000L, - AttemptAndTimeBoundedExponentialBackOff.ResetPolicy.TIMER, - fastClock); - assertThat(backOff.nextBackOffMillis(), allOf(greaterThan(249L), lessThan(751L))); - assertThat(backOff.nextBackOffMillis(), allOf(greaterThan(374L), lessThan(1126L))); - fastClock.sleep(2000L); - assertEquals(BackOff.STOP, backOff.nextBackOffMillis()); - backOff.reset(); - assertThat(backOff.nextBackOffMillis(), allOf(greaterThan(561L), lessThan(1688L))); - assertThat(backOff.nextBackOffMillis(), allOf(greaterThan(843L), lessThan(2531L))); - } - - @Test - public void testThatResettingTimerDoesNotAllowReuse() throws Exception { - AttemptBoundedExponentialBackOff backOff = - new AttemptAndTimeBoundedExponentialBackOff( - 3, - 500, - 1000L, - AttemptAndTimeBoundedExponentialBackOff.ResetPolicy.TIMER, - fastClock); - assertThat(backOff.nextBackOffMillis(), allOf(greaterThan(249L), lessThan(751L))); - assertThat(backOff.nextBackOffMillis(), allOf(greaterThan(374L), lessThan(1126L))); - assertEquals(BackOff.STOP, backOff.nextBackOffMillis()); - backOff.reset(); - assertEquals(BackOff.STOP, backOff.nextBackOffMillis()); - } - - @Test - public void testTimeBound() throws Exception { - AttemptBoundedExponentialBackOff backOff = - new AttemptAndTimeBoundedExponentialBackOff( - 3, 500L, 5L, AttemptAndTimeBoundedExponentialBackOff.ResetPolicy.ALL, fastClock); - assertEquals(backOff.nextBackOffMillis(), 5L); - } - - @Test - public void testAtMaxAttempts() throws Exception { - AttemptBoundedExponentialBackOff backOff = - new AttemptAndTimeBoundedExponentialBackOff( - 3, - 500L, - 1000L, - AttemptAndTimeBoundedExponentialBackOff.ResetPolicy.ALL, - fastClock); - assertFalse(backOff.atMaxAttempts()); - backOff.nextBackOffMillis(); - assertFalse(backOff.atMaxAttempts()); - backOff.nextBackOffMillis(); - assertTrue(backOff.atMaxAttempts()); - assertEquals(BackOff.STOP, backOff.nextBackOffMillis()); - } - - @Test - public void testAtMaxTime() throws Exception { - AttemptBoundedExponentialBackOff backOff = - new AttemptAndTimeBoundedExponentialBackOff( - 3, 500L, 1L, AttemptAndTimeBoundedExponentialBackOff.ResetPolicy.ALL, fastClock); - fastClock.sleep(2); - assertTrue(backOff.atMaxAttempts()); - assertEquals(BackOff.STOP, backOff.nextBackOffMillis()); - } -} diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/AttemptBoundedExponentialBackOffTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/AttemptBoundedExponentialBackOffTest.java deleted file mode 100644 index 3cfa961865fb3..0000000000000 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/AttemptBoundedExponentialBackOffTest.java +++ /dev/null @@ -1,84 +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; - -import static org.hamcrest.MatcherAssert.assertThat; -import static org.hamcrest.Matchers.allOf; -import static org.hamcrest.Matchers.greaterThan; -import static org.hamcrest.Matchers.lessThan; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertTrue; - -import com.google.api.client.util.BackOff; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.ExpectedException; -import org.junit.runner.RunWith; -import org.junit.runners.JUnit4; - -/** Unit tests for {@link AttemptBoundedExponentialBackOff}. */ -@RunWith(JUnit4.class) -public class AttemptBoundedExponentialBackOffTest { - @Rule public ExpectedException exception = ExpectedException.none(); - - @Test - public void testUsingInvalidInitialInterval() throws Exception { - exception.expect(IllegalArgumentException.class); - exception.expectMessage("Initial interval must be greater than zero."); - new AttemptBoundedExponentialBackOff(10, 0L); - } - - @Test - public void testUsingInvalidMaximumNumberOfRetries() throws Exception { - exception.expect(IllegalArgumentException.class); - exception.expectMessage("Maximum number of attempts must be greater than zero."); - new AttemptBoundedExponentialBackOff(-1, 10L); - } - - @Test - public void testThatFixedNumberOfAttemptsExits() throws Exception { - BackOff backOff = new AttemptBoundedExponentialBackOff(3, 500); - assertThat(backOff.nextBackOffMillis(), allOf(greaterThan(249L), lessThan(751L))); - assertThat(backOff.nextBackOffMillis(), allOf(greaterThan(374L), lessThan(1126L))); - assertEquals(BackOff.STOP, backOff.nextBackOffMillis()); - } - - @Test - public void testThatResettingAllowsReuse() throws Exception { - BackOff backOff = new AttemptBoundedExponentialBackOff(3, 500); - assertThat(backOff.nextBackOffMillis(), allOf(greaterThan(249L), lessThan(751L))); - assertThat(backOff.nextBackOffMillis(), allOf(greaterThan(374L), lessThan(1126L))); - assertEquals(BackOff.STOP, backOff.nextBackOffMillis()); - backOff.reset(); - assertThat(backOff.nextBackOffMillis(), allOf(greaterThan(249L), lessThan(751L))); - assertThat(backOff.nextBackOffMillis(), allOf(greaterThan(374L), lessThan(1126L))); - assertEquals(BackOff.STOP, backOff.nextBackOffMillis()); - } - - @Test - public void testAtMaxAttempts() throws Exception { - AttemptBoundedExponentialBackOff backOff = new AttemptBoundedExponentialBackOff(3, 500); - assertFalse(backOff.atMaxAttempts()); - backOff.nextBackOffMillis(); - assertFalse(backOff.atMaxAttempts()); - backOff.nextBackOffMillis(); - assertTrue(backOff.atMaxAttempts()); - assertEquals(BackOff.STOP, backOff.nextBackOffMillis()); - } -} diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/FluentBackoffTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/FluentBackoffTest.java new file mode 100644 index 0000000000000..20b03cf603a38 --- /dev/null +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/FluentBackoffTest.java @@ -0,0 +1,226 @@ +/* + * 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.allOf; +import static org.hamcrest.Matchers.containsString; +import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.greaterThanOrEqualTo; +import static org.hamcrest.Matchers.lessThan; +import static org.hamcrest.Matchers.lessThanOrEqualTo; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertThat; + +import com.google.api.client.util.BackOff; +import java.io.IOException; +import org.joda.time.Duration; +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 FluentBackoff}. + */ +@RunWith(JUnit4.class) +public class FluentBackoffTest { + + @Rule + public ExpectedException thrown = ExpectedException.none(); + private final FluentBackoff defaultBackoff = FluentBackoff.DEFAULT; + + @Test + public void testInvalidExponent() { + thrown.expect(IllegalArgumentException.class); + thrown.expectMessage("exponent -2.0 must be greater than 0"); + defaultBackoff.withExponent(-2.0); + } + + @Test + public void testInvalidInitialBackoff() { + thrown.expect(IllegalArgumentException.class); + thrown.expectMessage("initialBackoff PT0S must be at least 1 millisecond"); + defaultBackoff.withInitialBackoff(Duration.ZERO); + } + + @Test + public void testInvalidMaxBackoff() { + thrown.expect(IllegalArgumentException.class); + thrown.expectMessage("maxBackoff PT0S must be at least 1 millisecond"); + defaultBackoff.withMaxBackoff(Duration.ZERO); + } + + @Test + public void testInvalidMaxRetries() { + thrown.expect(IllegalArgumentException.class); + thrown.expectMessage("maxRetries -1 cannot be negative"); + defaultBackoff.withMaxRetries(-1); + } + + @Test + public void testInvalidCumulativeBackoff() { + thrown.expect(IllegalArgumentException.class); + thrown.expectMessage("maxCumulativeBackoff PT-0.002S must be at least 1 millisecond"); + defaultBackoff.withMaxCumulativeBackoff(Duration.millis(-2)); + } + + /** + * Tests with bounded interval, custom exponent, and unlimited retries. + */ + @Test + public void testBoundedIntervalWithReset() throws Exception { + BackOff backOff = + FluentBackoff.DEFAULT + .withInitialBackoff(Duration.millis(500)) + .withMaxBackoff(Duration.standardSeconds(1)).backoff(); + assertThat(backOff.nextBackOffMillis(), allOf(greaterThanOrEqualTo(249L), lessThan(751L))); + assertThat(backOff.nextBackOffMillis(), allOf(greaterThanOrEqualTo(374L), lessThan(1126L))); + assertThat(backOff.nextBackOffMillis(), allOf(greaterThanOrEqualTo(500L), + lessThanOrEqualTo(1500L))); + assertThat(backOff.nextBackOffMillis(), allOf(greaterThanOrEqualTo(500L), + lessThanOrEqualTo(1500L))); + assertThat(backOff.nextBackOffMillis(), allOf(greaterThanOrEqualTo(500L), + lessThanOrEqualTo(1500L))); + assertThat(backOff.nextBackOffMillis(), allOf(greaterThanOrEqualTo(500L), + lessThanOrEqualTo(1500L))); + assertThat(backOff.nextBackOffMillis(), allOf(greaterThanOrEqualTo(500L), + lessThanOrEqualTo(1500L))); + + // Reset, should go back to short times. + backOff.reset(); + assertThat(backOff.nextBackOffMillis(), allOf(greaterThanOrEqualTo(249L), lessThan(751L))); + assertThat(backOff.nextBackOffMillis(), allOf(greaterThanOrEqualTo(374L), lessThan(1126L))); + assertThat(backOff.nextBackOffMillis(), allOf(greaterThanOrEqualTo(500L), + lessThanOrEqualTo(1500L))); + assertThat(backOff.nextBackOffMillis(), allOf(greaterThanOrEqualTo(500L), + lessThanOrEqualTo(1500L))); + + } + + /** + * Tests with bounded interval, custom exponent, limited retries, and a reset. + */ + @Test + public void testMaxRetriesWithReset() throws Exception { + BackOff backOff = + FluentBackoff.DEFAULT + .withInitialBackoff(Duration.millis(500)) + .withMaxRetries(1) + .backoff(); + assertThat(backOff.nextBackOffMillis(), allOf(greaterThanOrEqualTo(249L), lessThan(751L))); + assertThat(backOff.nextBackOffMillis(), equalTo(BackOff.STOP)); + assertThat(backOff.nextBackOffMillis(), equalTo(BackOff.STOP)); + assertThat(backOff.nextBackOffMillis(), equalTo(BackOff.STOP)); + assertThat(backOff.nextBackOffMillis(), equalTo(BackOff.STOP)); + + backOff.reset(); + assertThat(backOff.nextBackOffMillis(), allOf(greaterThanOrEqualTo(249L), lessThan(751L))); + assertThat(backOff.nextBackOffMillis(), equalTo(BackOff.STOP)); + } + + private static long countMaximumBackoff(BackOff backOff) throws IOException { + long cumulativeBackoffMillis = 0; + long currentBackoffMillis = backOff.nextBackOffMillis(); + while (currentBackoffMillis != BackOff.STOP) { + cumulativeBackoffMillis += currentBackoffMillis; + currentBackoffMillis = backOff.nextBackOffMillis(); + } + return cumulativeBackoffMillis; + } + + /** + * Tests with bounded interval, custom exponent, limited cumulative time, and a reset. + */ + @Test + public void testBoundedIntervalAndCumTimeWithReset() throws Exception { + BackOff backOff = + FluentBackoff.DEFAULT + .withInitialBackoff(Duration.millis(500)) + .withMaxBackoff(Duration.standardSeconds(1)) + .withMaxCumulativeBackoff(Duration.standardMinutes(1)).backoff(); + + assertThat(countMaximumBackoff(backOff), equalTo(Duration.standardMinutes(1).getMillis())); + + backOff.reset(); + assertThat(countMaximumBackoff(backOff), equalTo(Duration.standardMinutes(1).getMillis())); + // sanity check: should get 0 if we don't reset + assertThat(countMaximumBackoff(backOff), equalTo(0L)); + + backOff.reset(); + assertThat(countMaximumBackoff(backOff), equalTo(Duration.standardMinutes(1).getMillis())); + } + + /** + * Tests with bounded interval, custom exponent, limited cumulative time and retries. + */ + @Test + public void testBoundedIntervalAndCumTimeAndRetriesWithReset() throws Exception { + BackOff backOff = + FluentBackoff.DEFAULT + .withInitialBackoff(Duration.millis(500)) + .withMaxBackoff(Duration.standardSeconds(1)) + .withMaxCumulativeBackoff(Duration.standardMinutes(1)) + .backoff(); + + long cumulativeBackoffMillis = 0; + long currentBackoffMillis = backOff.nextBackOffMillis(); + while (currentBackoffMillis != BackOff.STOP) { + cumulativeBackoffMillis += currentBackoffMillis; + currentBackoffMillis = backOff.nextBackOffMillis(); + } + assertThat(cumulativeBackoffMillis, equalTo(Duration.standardMinutes(1).getMillis())); + } + + @Test + public void testFluentBackoffToString() throws IOException { + FluentBackoff config = FluentBackoff.DEFAULT + .withExponent(3.4) + .withMaxRetries(4) + .withInitialBackoff(Duration.standardSeconds(3)) + .withMaxBackoff(Duration.standardHours(1)) + .withMaxCumulativeBackoff(Duration.standardDays(1)); + + assertEquals( + "FluentBackoff{exponent=3.4, initialBackoff=PT3S, maxBackoff=PT3600S," + + " maxRetries=4, maxCumulativeBackoff=PT86400S}", + config.toString()); + } + @Test + public void testBackoffImplToString() throws IOException { + FluentBackoff config = FluentBackoff.DEFAULT + .withExponent(3.4) + .withMaxRetries(4) + .withInitialBackoff(Duration.standardSeconds(3)) + .withMaxBackoff(Duration.standardHours(1)) + .withMaxCumulativeBackoff(Duration.standardDays(1)); + BackOff backOff = config.backoff(); + + assertEquals( + "BackoffImpl{backoffConfig=" + config.toString() + "," + + " currentRetry=0, currentCumulativeBackoff=PT0S}", + backOff.toString()); + + // backoff once, ignoring result + backOff.nextBackOffMillis(); + + // currentRetry is exact, we can test it. + assertThat(backOff.toString(), containsString("currentRetry=1")); + // currentCumulativeBackoff is not exact; we cannot even check that it's non-zero (randomness). + } +} 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 681b0aaaa0597..9504b4c0c07cf 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 @@ -365,7 +365,7 @@ public void testRetryFileSize() throws IOException { Storage.Objects mockStorageObjects = Mockito.mock(Storage.Objects.class); Storage.Objects.Get mockStorageGet = Mockito.mock(Storage.Objects.Get.class); - BackOff mockBackOff = new AttemptBoundedExponentialBackOff(3, 200); + BackOff mockBackOff = FluentBackoff.DEFAULT.withMaxRetries(2).backoff(); when(mockStorage.objects()).thenReturn(mockStorageObjects); when(mockStorageObjects.get("testbucket", "testobject")).thenReturn(mockStorageGet); @@ -376,7 +376,7 @@ public void testRetryFileSize() throws IOException { assertEquals(1000, gcsUtil.fileSize(GcsPath.fromComponents("testbucket", "testobject"), mockBackOff, new FastNanoClockAndSleeper())); - assertEquals(mockBackOff.nextBackOffMillis(), BackOff.STOP); + assertEquals(BackOff.STOP, mockBackOff.nextBackOffMillis()); } @Test @@ -390,7 +390,7 @@ public void testBucketExists() throws IOException { Storage.Buckets mockStorageObjects = Mockito.mock(Storage.Buckets.class); Storage.Buckets.Get mockStorageGet = Mockito.mock(Storage.Buckets.Get.class); - BackOff mockBackOff = new AttemptBoundedExponentialBackOff(3, 200); + BackOff mockBackOff = FluentBackoff.DEFAULT.backoff(); when(mockStorage.buckets()).thenReturn(mockStorageObjects); when(mockStorageObjects.get("testbucket")).thenReturn(mockStorageGet); @@ -413,7 +413,7 @@ public void testBucketDoesNotExistBecauseOfAccessError() throws IOException { Storage.Buckets mockStorageObjects = Mockito.mock(Storage.Buckets.class); Storage.Buckets.Get mockStorageGet = Mockito.mock(Storage.Buckets.Get.class); - BackOff mockBackOff = new AttemptBoundedExponentialBackOff(3, 200); + BackOff mockBackOff = FluentBackoff.DEFAULT.backoff(); GoogleJsonResponseException expectedException = googleJsonResponseException(HttpStatusCodes.STATUS_CODE_FORBIDDEN, "Waves hand mysteriously", "These aren't the buckets your looking for"); @@ -438,7 +438,7 @@ public void testBucketDoesNotExist() throws IOException { Storage.Buckets mockStorageObjects = Mockito.mock(Storage.Buckets.class); Storage.Buckets.Get mockStorageGet = Mockito.mock(Storage.Buckets.Get.class); - BackOff mockBackOff = new AttemptBoundedExponentialBackOff(3, 200); + BackOff mockBackOff = FluentBackoff.DEFAULT.backoff(); when(mockStorage.buckets()).thenReturn(mockStorageObjects); when(mockStorageObjects.get("testbucket")).thenReturn(mockStorageGet); 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 304dc820ac9da..6dde581ab4d59 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,9 +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.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.model.Job; import com.google.api.services.bigquery.model.JobConfigurationExtract; @@ -69,7 +66,6 @@ import java.util.UUID; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ThreadLocalRandom; -import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicReference; import java.util.regex.Matcher; import java.util.regex.Pattern; @@ -108,7 +104,6 @@ import org.apache.beam.sdk.transforms.windowing.DefaultTrigger; import org.apache.beam.sdk.transforms.windowing.GlobalWindows; import org.apache.beam.sdk.transforms.windowing.Window; -import org.apache.beam.sdk.util.AttemptBoundedExponentialBackOff; import org.apache.beam.sdk.util.FileIOChannelFactory; import org.apache.beam.sdk.util.GcsIOChannelFactory; import org.apache.beam.sdk.util.GcsUtil; @@ -129,6 +124,7 @@ import org.apache.beam.sdk.values.PDone; 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.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -294,7 +290,7 @@ public class BigQueryIO { * *

              If the project id is omitted, the default project id is used. */ - public static TableReference parseTableSpec(String tableSpec) { + static TableReference parseTableSpec(String tableSpec) { Matcher match = TABLE_SPEC.matcher(tableSpec); if (!match.matches()) { throw new IllegalArgumentException( @@ -953,14 +949,14 @@ private void readObject(ObjectInputStream in) throws ClassNotFoundException, IOE * ... */ private abstract static class BigQuerySourceBase extends BoundedSource { - // The maximum number of attempts to verify temp files. - private static final int MAX_FILES_VERIFY_ATTEMPTS = 10; + // The maximum number of retries to verify temp files. + private static final int MAX_FILES_VERIFY_RETRIES = 9; // The maximum number of retries to poll a BigQuery job. protected static final int JOB_POLL_MAX_RETRIES = Integer.MAX_VALUE; // The initial backoff for verifying temp files. - private static final long INITIAL_FILES_VERIFY_BACKOFF_MILLIS = TimeUnit.SECONDS.toMillis(1); + private static final Duration INITIAL_FILES_VERIFY_BACKOFF = Duration.standardSeconds(1); protected final String jobIdToken; protected final String extractDestinationDir; @@ -1055,14 +1051,7 @@ public TableRow apply(GenericRecord input) { }}; List> avroSources = Lists.newArrayList(); - BackOff backoff = new AttemptBoundedExponentialBackOff( - MAX_FILES_VERIFY_ATTEMPTS, INITIAL_FILES_VERIFY_BACKOFF_MILLIS); for (String fileName : files) { - while (BackOffUtils.next(Sleeper.DEFAULT, backoff)) { - if (IOChannelUtils.getFactory(fileName).getSizeBytes(fileName) != -1) { - break; - } - } avroSources.add(new TransformingSource<>( AvroSource.from(fileName), function, getDefaultOutputCoder())); } 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 6aff3b097710a..8b5e8c28aeb24 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 @@ -53,14 +53,13 @@ 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.util.AttemptBoundedExponentialBackOff; -import org.apache.beam.sdk.util.IntervalBoundedExponentialBackOff; +import org.apache.beam.sdk.util.FluentBackoff; import org.apache.beam.sdk.util.Transport; +import org.joda.time.Duration; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -72,14 +71,14 @@ class BigQueryServicesImpl implements BigQueryServices { private static final Logger LOG = LoggerFactory.getLogger(BigQueryServicesImpl.class); - // The maximum number of attempts to execute a BigQuery RPC. - private static final int MAX_RPC_ATTEMPTS = 10; + // The maximum number of retries to execute a BigQuery RPC. + private static final int MAX_RPC_RETRIES = 9; // The initial backoff for executing a BigQuery RPC. - private static final long INITIAL_RPC_BACKOFF_MILLIS = TimeUnit.SECONDS.toMillis(1); + private static final Duration INITIAL_RPC_BACKOFF = Duration.standardSeconds(1); // The initial backoff for polling the status of a BigQuery job. - private static final long INITIAL_JOB_STATUS_POLL_BACKOFF_MILLIS = TimeUnit.SECONDS.toMillis(1); + private static final Duration INITIAL_JOB_STATUS_POLL_BACKOFF = Duration.standardSeconds(1); @Override public JobService getJobService(BigQueryOptions options) { @@ -121,9 +120,9 @@ private JobServiceImpl(BigQueryOptions options) { /** * {@inheritDoc} * - *

              Tries executing the RPC for at most {@code MAX_RPC_ATTEMPTS} times until it succeeds. + *

              Tries executing the RPC for at most {@code MAX_RPC_RETRIES} times until it succeeds. * - * @throws IOException if it exceeds {@code MAX_RPC_ATTEMPTS} attempts. + * @throws IOException if it exceeds {@code MAX_RPC_RETRIES} attempts. */ @Override public void startLoadJob( @@ -139,9 +138,9 @@ public void startLoadJob( /** * {@inheritDoc} * - *

              Tries executing the RPC for at most {@code MAX_RPC_ATTEMPTS} times until it succeeds. + *

              Tries executing the RPC for at most {@code MAX_RPC_RETRIES} times until it succeeds. * - * @throws IOException if it exceeds {@code MAX_RPC_ATTEMPTS} attempts. + * @throws IOException if it exceeds {@code MAX_RPC_RETRIES} attempts. */ @Override public void startExtractJob(JobReference jobRef, JobConfigurationExtract extractConfig) @@ -157,9 +156,9 @@ public void startExtractJob(JobReference jobRef, JobConfigurationExtract extract /** * {@inheritDoc} * - *

              Tries executing the RPC for at most {@code MAX_RPC_ATTEMPTS} times until it succeeds. + *

              Tries executing the RPC for at most {@code MAX_RPC_RETRIES} times until it succeeds. * - * @throws IOException if it exceeds {@code MAX_RPC_ATTEMPTS} attempts. + * @throws IOException if it exceeds {@code MAX_RPC_RETRIES} attempts. */ @Override public void startQueryJob(JobReference jobRef, JobConfigurationQuery queryConfig) @@ -175,9 +174,9 @@ public void startQueryJob(JobReference jobRef, JobConfigurationQuery queryConfig /** * {@inheritDoc} * - *

              Tries executing the RPC for at most {@code MAX_RPC_ATTEMPTS} times until it succeeds. + *

              Tries executing the RPC for at most {@code MAX_RPC_RETRIES} times until it succeeds. * - * @throws IOException if it exceeds {@code MAX_RPC_ATTEMPTS} attempts. + * @throws IOException if it exceeds {@code MAX_RPC_RETRIES} attempts. */ @Override public void startCopyJob(JobReference jobRef, JobConfigurationTableCopy copyConfig) @@ -194,7 +193,8 @@ private static void startJob(Job job, ApiErrorExtractor errorExtractor, Bigquery client) throws IOException, InterruptedException { BackOff backoff = - new AttemptBoundedExponentialBackOff(MAX_RPC_ATTEMPTS, INITIAL_RPC_BACKOFF_MILLIS); + FluentBackoff.DEFAULT + .withMaxRetries(MAX_RPC_RETRIES).withInitialBackoff(INITIAL_RPC_BACKOFF).backoff(); startJob(job, errorExtractor, client, Sleeper.DEFAULT, backoff); } @@ -227,15 +227,17 @@ static void startJob( throw new IOException( String.format( "Unable to insert job: %s, aborting after %d .", - jobRef.getJobId(), MAX_RPC_ATTEMPTS), + jobRef.getJobId(), MAX_RPC_RETRIES), lastException); } @Override public Job pollJob(JobReference jobRef, int maxAttempts) throws InterruptedException { - BackOff backoff = new AttemptBoundedExponentialBackOff( - maxAttempts, INITIAL_JOB_STATUS_POLL_BACKOFF_MILLIS); + BackOff backoff = + FluentBackoff.DEFAULT + .withMaxRetries(maxAttempts).withInitialBackoff(INITIAL_JOB_STATUS_POLL_BACKOFF) + .backoff(); return pollJob(jobRef, Sleeper.DEFAULT, backoff); } @@ -270,12 +272,13 @@ public JobStatistics dryRunQuery(String projectId, String query) .setQuery(query)) .setDryRun(true)); BackOff backoff = - new AttemptBoundedExponentialBackOff(MAX_RPC_ATTEMPTS, INITIAL_RPC_BACKOFF_MILLIS); + FluentBackoff.DEFAULT + .withMaxRetries(MAX_RPC_RETRIES).withInitialBackoff(INITIAL_RPC_BACKOFF).backoff(); return executeWithRetries( client.jobs().insert(projectId, job), String.format( "Unable to dry run query: %s, aborting after %d retries.", - query, MAX_RPC_ATTEMPTS), + query, MAX_RPC_RETRIES), Sleeper.DEFAULT, backoff).getStatistics(); } @@ -289,15 +292,14 @@ static class DatasetServiceImpl implements DatasetService { // The maximum number of rows to upload per InsertAll request. private static final long MAX_ROWS_PER_BATCH = 500; - // The maximum number of times to retry inserting rows into BigQuery. - private static final int MAX_INSERT_ATTEMPTS = 5; - - // The initial backoff after a failure inserting rows into BigQuery. - private static final long INITIAL_INSERT_BACKOFF_INTERVAL_MS = 200L; + private static final FluentBackoff INSERT_BACKOFF_FACTORY = + FluentBackoff.DEFAULT.withInitialBackoff(Duration.millis(200)).withMaxRetries(5); - // Backoff time bounds for rate limit exceeded errors. - private static final long INITIAL_RATE_LIMIT_EXCEEDED_BACKOFF_MS = TimeUnit.SECONDS.toMillis(1); - private static final long MAX_RATE_LIMIT_EXCEEDED_BACKOFF_MS = TimeUnit.MINUTES.toMillis(2); + // A backoff for rate limit exceeded errors. Retries forever. + private static final FluentBackoff DEFAULT_BACKOFF_FACTORY = + FluentBackoff.DEFAULT + .withInitialBackoff(Duration.standardSeconds(1)) + .withMaxBackoff(Duration.standardMinutes(2)); private final ApiErrorExtractor errorExtractor; private final Bigquery client; @@ -335,20 +337,21 @@ private DatasetServiceImpl(BigQueryOptions bqOptions) { /** * {@inheritDoc} * - *

              Tries executing the RPC for at most {@code MAX_RPC_ATTEMPTS} times until it succeeds. + *

              Tries executing the RPC for at most {@code MAX_RPC_RETRIES} times until it succeeds. * - * @throws IOException if it exceeds {@code MAX_RPC_ATTEMPTS} attempts. + * @throws IOException if it exceeds {@code MAX_RPC_RETRIES} attempts. */ @Override public Table getTable(String projectId, String datasetId, String tableId) throws IOException, InterruptedException { BackOff backoff = - new AttemptBoundedExponentialBackOff(MAX_RPC_ATTEMPTS, INITIAL_RPC_BACKOFF_MILLIS); + FluentBackoff.DEFAULT + .withMaxRetries(MAX_RPC_RETRIES).withInitialBackoff(INITIAL_RPC_BACKOFF).backoff(); return executeWithRetries( client.tables().get(projectId, datasetId, tableId), String.format( "Unable to get table: %s, aborting after %d retries.", - tableId, MAX_RPC_ATTEMPTS), + tableId, MAX_RPC_RETRIES), Sleeper.DEFAULT, backoff); } @@ -356,20 +359,21 @@ public Table getTable(String projectId, String datasetId, String tableId) /** * {@inheritDoc} * - *

              Tries executing the RPC for at most {@code MAX_RPC_ATTEMPTS} times until it succeeds. + *

              Tries executing the RPC for at most {@code MAX_RPC_RETRIES} times until it succeeds. * - * @throws IOException if it exceeds {@code MAX_RPC_ATTEMPTS} attempts. + * @throws IOException if it exceeds {@code MAX_RPC_RETRIES} attempts. */ @Override public void deleteTable(String projectId, String datasetId, String tableId) throws IOException, InterruptedException { BackOff backoff = - new AttemptBoundedExponentialBackOff(MAX_RPC_ATTEMPTS, INITIAL_RPC_BACKOFF_MILLIS); + FluentBackoff.DEFAULT + .withMaxRetries(MAX_RPC_RETRIES).withInitialBackoff(INITIAL_RPC_BACKOFF).backoff(); executeWithRetries( client.tables().delete(projectId, datasetId, tableId), String.format( "Unable to delete table: %s, aborting after %d retries.", - tableId, MAX_RPC_ATTEMPTS), + tableId, MAX_RPC_RETRIES), Sleeper.DEFAULT, backoff); } @@ -378,12 +382,13 @@ public void deleteTable(String projectId, String datasetId, String tableId) public boolean isTableEmpty(String projectId, String datasetId, String tableId) throws IOException, InterruptedException { BackOff backoff = - new AttemptBoundedExponentialBackOff(MAX_RPC_ATTEMPTS, INITIAL_RPC_BACKOFF_MILLIS); + FluentBackoff.DEFAULT + .withMaxRetries(MAX_RPC_RETRIES).withInitialBackoff(INITIAL_RPC_BACKOFF).backoff(); TableDataList dataList = executeWithRetries( client.tabledata().list(projectId, datasetId, tableId), String.format( "Unable to list table data: %s, aborting after %d retries.", - tableId, MAX_RPC_ATTEMPTS), + tableId, MAX_RPC_RETRIES), Sleeper.DEFAULT, backoff); return dataList.getRows() == null || dataList.getRows().isEmpty(); @@ -392,20 +397,21 @@ public boolean isTableEmpty(String projectId, String datasetId, String tableId) /** * {@inheritDoc} * - *

              Tries executing the RPC for at most {@code MAX_RPC_ATTEMPTS} times until it succeeds. + *

              Tries executing the RPC for at most {@code MAX_RPC_RETRIES} times until it succeeds. * - * @throws IOException if it exceeds {@code MAX_RPC_ATTEMPTS} attempts. + * @throws IOException if it exceeds {@code MAX_RPC_RETRIES} attempts. */ @Override public Dataset getDataset(String projectId, String datasetId) throws IOException, InterruptedException { BackOff backoff = - new AttemptBoundedExponentialBackOff(MAX_RPC_ATTEMPTS, INITIAL_RPC_BACKOFF_MILLIS); + FluentBackoff.DEFAULT + .withMaxRetries(MAX_RPC_RETRIES).withInitialBackoff(INITIAL_RPC_BACKOFF).backoff(); return executeWithRetries( client.datasets().get(projectId, datasetId), String.format( "Unable to get dataset: %s, aborting after %d retries.", - datasetId, MAX_RPC_ATTEMPTS), + datasetId, MAX_RPC_RETRIES), Sleeper.DEFAULT, backoff); } @@ -413,21 +419,21 @@ public Dataset getDataset(String projectId, String datasetId) /** * {@inheritDoc} * - *

              Tries executing the RPC for at most {@code MAX_RPC_ATTEMPTS} times until it succeeds. + *

              Tries executing the RPC for at most {@code MAX_RPC_RETRIES} times until it succeeds. * - * @throws IOException if it exceeds {@code MAX_RPC_ATTEMPTS} attempts. + * @throws IOException if it exceeds {@code MAX_RPC_RETRIES} attempts. */ @Override public void createDataset( String projectId, String datasetId, String location, String description) throws IOException, InterruptedException { BackOff backoff = - new AttemptBoundedExponentialBackOff(MAX_RPC_ATTEMPTS, INITIAL_RPC_BACKOFF_MILLIS); + FluentBackoff.DEFAULT + .withMaxRetries(MAX_RPC_RETRIES).withInitialBackoff(INITIAL_RPC_BACKOFF).backoff(); createDataset(projectId, datasetId, location, description, Sleeper.DEFAULT, backoff); } - @VisibleForTesting - void createDataset( + private void createDataset( String projectId, String datasetId, String location, @@ -464,27 +470,28 @@ void createDataset( throw new IOException( String.format( "Unable to create dataset: %s, aborting after %d .", - datasetId, MAX_RPC_ATTEMPTS), + datasetId, MAX_RPC_RETRIES), lastException); } /** * {@inheritDoc} * - *

              Tries executing the RPC for at most {@code MAX_RPC_ATTEMPTS} times until it succeeds. + *

              Tries executing the RPC for at most {@code MAX_RPC_RETRIES} times until it succeeds. * - * @throws IOException if it exceeds {@code MAX_RPC_ATTEMPTS} attempts. + * @throws IOException if it exceeds {@code MAX_RPC_RETRIES} attempts. */ @Override public void deleteDataset(String projectId, String datasetId) throws IOException, InterruptedException { BackOff backoff = - new AttemptBoundedExponentialBackOff(MAX_RPC_ATTEMPTS, INITIAL_RPC_BACKOFF_MILLIS); + FluentBackoff.DEFAULT + .withMaxRetries(MAX_RPC_RETRIES).withInitialBackoff(INITIAL_RPC_BACKOFF).backoff(); executeWithRetries( client.datasets().delete(projectId, datasetId), String.format( "Unable to delete table: %s, aborting after %d retries.", - datasetId, MAX_RPC_ATTEMPTS), + datasetId, MAX_RPC_RETRIES), Sleeper.DEFAULT, backoff); } @@ -502,9 +509,7 @@ public long insertAll( + "as many elements as rowList"); } - AttemptBoundedExponentialBackOff backoff = new AttemptBoundedExponentialBackOff( - MAX_INSERT_ATTEMPTS, - INITIAL_INSERT_BACKOFF_INTERVAL_MS); + BackOff backoff = INSERT_BACKOFF_FACTORY.backoff(); long retTotalDataSize = 0; List allErrors = new ArrayList<>(); @@ -547,8 +552,7 @@ public long insertAll( executor.submit(new Callable>() { @Override public List call() throws IOException { - BackOff backoff = new IntervalBoundedExponentialBackOff( - MAX_RATE_LIMIT_EXCEEDED_BACKOFF_MS, INITIAL_RATE_LIMIT_EXCEEDED_BACKOFF_MS); + BackOff backoff = DEFAULT_BACKOFF_FACTORY.backoff(); while (true) { try { return insert.execute().getInsertErrors(); @@ -603,21 +607,24 @@ public List call() throws IOException { throw new RuntimeException(e.getCause()); } - if (!allErrors.isEmpty() && !backoff.atMaxAttempts()) { - try { - Thread.sleep(backoff.nextBackOffMillis()); - } catch (InterruptedException e) { - Thread.currentThread().interrupt(); - throw new IOException( - "Interrupted while waiting before retrying insert of " + retryRows); - } - LOG.info("Retrying failed inserts to BigQuery"); - rowsToPublish = retryRows; - idsToPublish = retryIds; - allErrors.clear(); - } else { + if (allErrors.isEmpty()) { + break; + } + long nextBackoffMillis = backoff.nextBackOffMillis(); + if (nextBackoffMillis == BackOff.STOP) { break; } + try { + Thread.sleep(backoff.nextBackOffMillis()); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new IOException( + "Interrupted while waiting before retrying insert of " + retryRows); + } + LOG.info("Retrying failed inserts to BigQuery"); + rowsToPublish = retryRows; + idsToPublish = retryIds; + allErrors.clear(); } if (!allErrors.isEmpty()) { throw new IOException("Insert failed: " + allErrors); @@ -628,7 +635,7 @@ public List call() throws IOException { } private static class BigQueryJsonReaderImpl implements BigQueryJsonReader { - BigQueryTableRowIterator iterator; + private BigQueryTableRowIterator iterator; private BigQueryJsonReaderImpl(BigQueryTableRowIterator iterator) { this.iterator = iterator; @@ -706,7 +713,6 @@ static T executeWithRetries( /** * Identical to {@link BackOffUtils#next} but without checked IOException. - * @throws InterruptedException */ private static boolean nextBackOff(Sleeper sleeper, BackOff backoff) throws InterruptedException { try { diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryTableRowIterator.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryTableRowIterator.java index 729da97f7f7ec..677c661b8ac85 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryTableRowIterator.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryTableRowIterator.java @@ -58,7 +58,7 @@ import java.util.Random; import java.util.concurrent.TimeUnit; import javax.annotation.Nullable; -import org.apache.beam.sdk.util.AttemptBoundedExponentialBackOff; +import org.apache.beam.sdk.util.FluentBackoff; import org.joda.time.Duration; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -426,7 +426,8 @@ public static T executeWithBackOff(AbstractGoogleClientRequest client, St Object... errorArgs) throws IOException, InterruptedException { Sleeper sleeper = Sleeper.DEFAULT; BackOff backOff = - new AttemptBoundedExponentialBackOff(MAX_RETRIES, INITIAL_BACKOFF_TIME.getMillis()); + FluentBackoff.DEFAULT + .withMaxRetries(MAX_RETRIES).withInitialBackoff(INITIAL_BACKOFF_TIME).backoff(); T result = null; while (true) { 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 6bd03b5082688..45871f1397690 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 @@ -76,12 +76,13 @@ import org.apache.beam.sdk.transforms.Values; import org.apache.beam.sdk.transforms.display.DisplayData; import org.apache.beam.sdk.transforms.display.DisplayData.Builder; -import org.apache.beam.sdk.util.AttemptBoundedExponentialBackOff; +import org.apache.beam.sdk.util.FluentBackoff; import org.apache.beam.sdk.util.RetryHttpRequestInitializer; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PBegin; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PDone; +import org.joda.time.Duration; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -862,16 +863,11 @@ static class DatastoreWriterFn extends DoFn { private final V1DatastoreFactory datastoreFactory; // Current batch of mutations to be written. private final List mutations = new ArrayList<>(); - /** - * Since a bundle is written in batches, we should retry the commit of a batch in order to - * prevent transient errors from causing the bundle to fail. - */ - private static final int MAX_RETRIES = 5; - /** - * Initial backoff time for exponential backoff for retry attempts. - */ - private static final int INITIAL_BACKOFF_MILLIS = 5000; + private static final int MAX_RETRIES = 5; + private static final FluentBackoff BUNDLE_WRITE_BACKOFF = + FluentBackoff.DEFAULT + .withMaxRetries(MAX_RETRIES).withInitialBackoff(Duration.standardSeconds(5)); DatastoreWriterFn(String projectId) { this(projectId, new V1DatastoreFactory()); @@ -906,10 +902,10 @@ public void finishBundle(Context c) throws Exception { /** * Writes a batch of mutations to Cloud Datastore. * - *

              If a commit fails, it will be retried (up to {@link DatastoreWriterFn#MAX_RETRIES} - * times). All mutations in the batch will be committed again, even if the commit was partially - * successful. If the retry limit is exceeded, the last exception from the Cloud Datastore will - * be thrown. + *

              If a commit fails, it will be retried up to {@link #MAX_RETRIES} times. All + * mutations in the batch will be committed again, even if the commit was partially + * successful. If the retry limit is exceeded, the last exception from Cloud Datastore will be + * thrown. * * @throws DatastoreException if the commit fails or IOException or InterruptedException if * backing off between retries fails. @@ -917,7 +913,7 @@ public void finishBundle(Context c) throws Exception { private void flushBatch() throws DatastoreException, IOException, InterruptedException { LOG.debug("Writing batch of {} mutations", mutations.size()); Sleeper sleeper = Sleeper.DEFAULT; - BackOff backoff = new AttemptBoundedExponentialBackOff(MAX_RETRIES, INITIAL_BACKOFF_MILLIS); + BackOff backoff = BUNDLE_WRITE_BACKOFF.backoff(); while (true) { // Batch upsert entities. 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 eb5fbe6e05f5d..16cb004e318aa 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 @@ -56,7 +56,7 @@ import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.testing.ExpectedLogs; import org.apache.beam.sdk.testing.FastNanoClockAndSleeper; -import org.apache.beam.sdk.util.AttemptBoundedExponentialBackOff; +import org.apache.beam.sdk.util.FluentBackoff; import org.apache.beam.sdk.util.RetryHttpRequestInitializer; import org.apache.beam.sdk.util.Transport; import org.junit.Before; @@ -117,9 +117,8 @@ public void testStartLoadJobSucceeds() throws IOException, InterruptedException when(response.getContent()).thenReturn(toStream(testJob)); Sleeper sleeper = new FastNanoClockAndSleeper(); - BackOff backoff = new AttemptBoundedExponentialBackOff( - 5 /* attempts */, 1000 /* initialIntervalMillis */); - JobServiceImpl.startJob(testJob, new ApiErrorExtractor(), bigquery, sleeper, backoff); + JobServiceImpl.startJob( + testJob, new ApiErrorExtractor(), bigquery, sleeper, FluentBackoff.DEFAULT.backoff()); verify(response, times(1)).getStatusCode(); verify(response, times(1)).getContent(); @@ -141,9 +140,8 @@ public void testStartLoadJobSucceedsAlreadyExists() throws IOException, Interrup when(response.getStatusCode()).thenReturn(409); // 409 means already exists Sleeper sleeper = new FastNanoClockAndSleeper(); - BackOff backoff = new AttemptBoundedExponentialBackOff( - 5 /* attempts */, 1000 /* initialIntervalMillis */); - JobServiceImpl.startJob(testJob, new ApiErrorExtractor(), bigquery, sleeper, backoff); + JobServiceImpl.startJob( + testJob, new ApiErrorExtractor(), bigquery, sleeper, FluentBackoff.DEFAULT.backoff()); verify(response, times(1)).getStatusCode(); verify(response, times(1)).getContent(); @@ -169,9 +167,8 @@ public void testStartLoadJobRetry() throws IOException, InterruptedException { .thenReturn(toStream(testJob)); Sleeper sleeper = new FastNanoClockAndSleeper(); - BackOff backoff = new AttemptBoundedExponentialBackOff( - 5 /* attempts */, 1000 /* initialIntervalMillis */); - JobServiceImpl.startJob(testJob, new ApiErrorExtractor(), bigquery, sleeper, backoff); + JobServiceImpl.startJob( + testJob, new ApiErrorExtractor(), bigquery, sleeper, FluentBackoff.DEFAULT.backoff()); verify(response, times(2)).getStatusCode(); verify(response, times(2)).getContent(); diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/V1TestUtil.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/V1TestUtil.java index a596bb3b76868..b680a0e23e759 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/V1TestUtil.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/datastore/V1TestUtil.java @@ -54,8 +54,9 @@ import org.apache.beam.sdk.options.GcpOptions; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.DoFn; -import org.apache.beam.sdk.util.AttemptBoundedExponentialBackOff; +import org.apache.beam.sdk.util.FluentBackoff; import org.apache.beam.sdk.util.RetryHttpRequestInitializer; +import org.joda.time.Duration; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -232,7 +233,7 @@ static class V1TestWriter { // Number of times to retry on update failure private static final int MAX_RETRIES = 5; //Initial backoff time for exponential backoff for retry attempts. - private static final int INITIAL_BACKOFF_MILLIS = 5000; + private static final Duration INITIAL_BACKOFF = Duration.standardSeconds(5); // Returns true if a Datastore key is complete. A key is complete if its last element // has either an id or a name. @@ -279,7 +280,9 @@ void close() throws Exception { private void flushBatch() throws DatastoreException, IOException, InterruptedException { LOG.info("Writing batch of {} entities", entities.size()); Sleeper sleeper = Sleeper.DEFAULT; - BackOff backoff = new AttemptBoundedExponentialBackOff(MAX_RETRIES, INITIAL_BACKOFF_MILLIS); + BackOff backoff = + FluentBackoff.DEFAULT + .withMaxRetries(MAX_RETRIES).withInitialBackoff(INITIAL_BACKOFF).backoff(); while (true) { // Batch mutate entities. From 9ae5cc7310137e928d353eba0104dfd7ae263a96 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Jean-Baptiste=20Onofr=C3=A9?= Date: Fri, 15 Jul 2016 18:44:26 +0200 Subject: [PATCH 105/346] [BEAM-456] Add MongoDbIO --- sdks/java/io/mongodb/pom.xml | 129 ++++ .../apache/beam/sdk/io/mongodb/MongoDbIO.java | 553 ++++++++++++++++++ .../beam/sdk/io/mongodb/package-info.java | 22 + .../beam/sdk/io/mongodb/MongoDbIOTest.java | 209 +++++++ .../beam/sdk/io/mongodb/package-info.java | 22 + sdks/java/io/pom.xml | 1 + 6 files changed, 936 insertions(+) create mode 100644 sdks/java/io/mongodb/pom.xml create mode 100644 sdks/java/io/mongodb/src/main/java/org/apache/beam/sdk/io/mongodb/MongoDbIO.java create mode 100644 sdks/java/io/mongodb/src/main/java/org/apache/beam/sdk/io/mongodb/package-info.java create mode 100644 sdks/java/io/mongodb/src/test/java/org/apache/beam/sdk/io/mongodb/MongoDbIOTest.java create mode 100644 sdks/java/io/mongodb/src/test/java/org/apache/beam/sdk/io/mongodb/package-info.java diff --git a/sdks/java/io/mongodb/pom.xml b/sdks/java/io/mongodb/pom.xml new file mode 100644 index 0000000000000..60f1d1e108ff7 --- /dev/null +++ b/sdks/java/io/mongodb/pom.xml @@ -0,0 +1,129 @@ + + + + + 4.0.0 + + + org.apache.beam + beam-sdks-java-io-parent + 0.3.0-incubating-SNAPSHOT + ../pom.xml + + + beam-sdks-java-io-mongodb + Apache Beam :: SDKs :: Java :: IO :: MongoDB + IO to read and write on MongoDB. + + + + + org.apache.maven.plugins + maven-compiler-plugin + + + org.apache.maven.plugins + maven-source-plugin + + + org.apache.maven.plugins + maven-surefire-plugin + + + org.apache.maven.plugins + maven-jar-plugin + + + org.apache.maven.plugins + maven-checkstyle-plugin + + + org.apache.maven.plugins + maven-javadoc-plugin + + + + + + 3.2.2 + + + + + org.apache.beam + beam-sdks-java-core + + + + org.slf4j + slf4j-api + + + + com.google.guava + guava + + + + com.google.code.findbugs + annotations + + + + org.mongodb + mongo-java-driver + ${mongo-java-driver.version} + + + + + de.flapdoodle.embed + de.flapdoodle.embed.mongo + 1.50.1 + test + + + de.flapdoodle.embed + de.flapdoodle.embed.process + 1.50.1 + test + + + junit + junit + test + + + org.slf4j + slf4j-jdk14 + test + + + org.apache.beam + beam-runners-direct-java + ${project.version} + test + + + org.hamcrest + hamcrest-all + test + + + + \ No newline at end of file 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 new file mode 100644 index 0000000000000..77246143757c4 --- /dev/null +++ b/sdks/java/io/mongodb/src/main/java/org/apache/beam/sdk/io/mongodb/MongoDbIO.java @@ -0,0 +1,553 @@ +/* + * 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.mongodb; + +import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Preconditions; + +import com.mongodb.BasicDBObject; +import com.mongodb.MongoClient; +import com.mongodb.MongoClientURI; +import com.mongodb.client.MongoCollection; +import com.mongodb.client.MongoCursor; +import com.mongodb.client.MongoDatabase; + +import java.util.ArrayList; +import java.util.List; + +import javax.annotation.Nullable; + +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.coders.SerializableCoder; +import org.apache.beam.sdk.io.BoundedSource; +import org.apache.beam.sdk.options.PipelineOptions; +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.display.DisplayData; +import org.apache.beam.sdk.values.PBegin; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.PDone; + +import org.bson.Document; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * IO to read and write data on MongoDB. + *

              + *

              Reading from MongoDB

              + *

              + *

              MongoDbIO source returns a bounded collection of String as {@code PCollection}. + * The String is the JSON form of the MongoDB Document.

              + *

              + *

              To configure the MongoDB source, you have to provide the connection URI, the database name + * and the collection name. The following example illustrates various options for configuring the + * source:

              + *

              + *

              {@code
              + *
              + * pipeline.apply(MongoDbIO.read()
              + *   .withUri("mongodb://localhost:27017")
              + *   .withDatabase("my-database")
              + *   .withCollection("my-collection"))
              + *   // above three are required configuration, returns PCollection
              + *
              + *   // rest of the settings are optional
              + *
              + * }
              + *

              + *

              The source also accepts an optional configuration: {@code withFilter()} allows you to + * define a JSON filter to get subset of data.

              + *

              + *

              Writing to MongoDB

              + *

              + *

              MongoDB sink supports writing of Document (as JSON String) in a MongoDB.

              + *

              + *

              To configure a MongoDB sink, you must specify a connection {@code URI}, a {@code Database} + * name, a {@code Collection} name. For instance:

              + *

              + *

              {@code
              + *
              + * pipeline
              + *   .apply(...)
              + *   .apply(MongoDbIO.write()
              + *     .withUri("mongodb://localhost:27017")
              + *     .withDatabase("my-database")
              + *     .withCollection("my-collection")
              + *     .withNumSplits(30))
              + *
              + * }
              + */ +// TODO instead of JSON String, does it make sense to populate the PCollection with BSON Document or +// DBObject ?? +public class MongoDbIO { + + private static final Logger LOGGER = LoggerFactory.getLogger(MongoDbIO.class); + + /** Read data from MongoDB. */ + public static Read read() { + return new Read(new BoundedMongoDbSource(null, null, null, null, 0)); + } + + /** Write data to MongoDB. */ + public static Write write() { + return new Write(new Write.MongoDbWriter(null, null, null, 1024L)); + } + + private MongoDbIO() { + } + + /** + * A {@link PTransform} to read data from MongoDB. + */ + public static class Read extends PTransform> { + + public Read withUri(String uri) { + return new Read(source.withUri(uri)); + } + + public Read withDatabase(String database) { + return new Read(source.withDatabase(database)); + } + + public Read withCollection(String collection) { + return new Read(source.withCollection(collection)); + } + + public Read withFilter(String filter) { + return new Read(source.withFilter(filter)); + } + + public Read withNumSplits(int numSplits) { + return new Read(source.withNumSplits(numSplits)); + } + + private final BoundedMongoDbSource source; + + private Read(BoundedMongoDbSource source) { + this.source = source; + } + + @Override + public PCollection apply(PBegin input) { + return input.apply(org.apache.beam.sdk.io.Read.from(getSource())); + } + + /** + * Creates a {@link BoundedSource} with the configuration in {@link Read}. + */ + @VisibleForTesting + BoundedSource getSource() { + return source; + } + + @Override + public void validate(PBegin input) { + source.validate(); + } + + @Override + public void populateDisplayData(DisplayData.Builder builder) { + super.populateDisplayData(builder); + + source.populateDisplayData(builder); + } + + } + + private static class BoundedMongoDbSource extends BoundedSource { + + public BoundedMongoDbSource withUri(String uri) { + return new BoundedMongoDbSource(uri, database, collection, filter, numSplits); + } + + public BoundedMongoDbSource withDatabase(String database) { + return new BoundedMongoDbSource(uri, database, collection, filter, numSplits); + } + + public BoundedMongoDbSource withCollection(String collection) { + return new BoundedMongoDbSource(uri, database, collection, filter, numSplits); + } + + public BoundedMongoDbSource withFilter(String filter) { + return new BoundedMongoDbSource(uri, database, collection, filter, numSplits); + } + + public BoundedMongoDbSource withNumSplits(int numSplits) { + return new BoundedMongoDbSource(uri, database, collection, filter, numSplits); + } + + private final String uri; + private final String database; + private final String collection; + @Nullable + private final String filter; + private final int numSplits; + + public BoundedMongoDbSource(String uri, String database, String collection, String filter, + int numSplits) { + this.uri = uri; + this.database = database; + this.collection = collection; + this.filter = filter; + this.numSplits = numSplits; + } + + @Override + public Coder getDefaultOutputCoder() { + return SerializableCoder.of(String.class); + } + + @Override + public void validate() { + Preconditions.checkNotNull(uri, "uri"); + Preconditions.checkNotNull(database, "database"); + Preconditions.checkNotNull(collection, "collection"); + } + + @Override + public void populateDisplayData(DisplayData.Builder builder) { + builder.add(DisplayData.item("uri", uri)); + builder.add(DisplayData.item("database", database)); + builder.add(DisplayData.item("collection", collection)); + builder.addIfNotNull(DisplayData.item("filter", filter)); + builder.add(DisplayData.item("numSplit", numSplits)); + } + + @Override + public boolean producesSortedKeys(PipelineOptions options) { + return false; + } + + @Override + public BoundedReader createReader(PipelineOptions options) { + return new BoundedMongoDbReader(this); + } + + @Override + public long getEstimatedSizeBytes(PipelineOptions pipelineOptions) { + long estimatedSizeBytes = 0L; + + MongoClient mongoClient = new MongoClient(); + MongoDatabase mongoDatabase = mongoClient.getDatabase(database); + MongoCollection mongoCollection = mongoDatabase.getCollection(collection); + + // get the Mongo collStats object + // it gives the size for the entire collection + BasicDBObject stat = new BasicDBObject(); + stat.append("collStats", collection); + Document stats = mongoDatabase.runCommand(stat); + estimatedSizeBytes = Long.valueOf(stats.get("size").toString()); + return estimatedSizeBytes; + } + + @Override + public List> splitIntoBundles(long desiredBundleSizeBytes, + PipelineOptions options) { + MongoClient mongoClient = new MongoClient(); + MongoDatabase mongoDatabase = mongoClient.getDatabase(database); + + List splitKeys = null; + if (numSplits > 0) { + // the user defines his desired number of splits + // calculate the batch size + long estimatedSizeBytes = getEstimatedSizeBytes(options); + desiredBundleSizeBytes = estimatedSizeBytes / numSplits; + } + + // the desired batch size is small, using default chunk size of 1MB + if (desiredBundleSizeBytes < 1024 * 1024) { + desiredBundleSizeBytes = 1 * 1024 * 1024; + } + + // now we have the batch size (provided by user or provided by the runner) + // we use Mongo splitVector command to get the split keys + BasicDBObject splitVectorCommand = new BasicDBObject(); + splitVectorCommand.append("splitVector", database + "." + collection); + 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); + 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"); + sources.add(this); + return sources; + } + + LOGGER.debug("Number of splits is {}", splitKeys.size()); + for (String shardFilter : splitKeysToFilters(splitKeys, filter)) { + sources.add(this.withFilter(shardFilter)); + } + + return sources; + } + + /** + * Transform a list of split keys as a list of filters containing corresponding range. + * + *

              The list of split keys contains BSon Document basically containing for example: + *

                + *
              • _id: 56
              • + *
              • _id: 109
              • + *
              • _id: 256
              • + *
              + *

              + * + * This method will generate a list of range filters performing the following splits: + *
                + *
              • from the beginning of the collection up to _id 56, so basically data with + * _id lower than 56
              • + *
              • from _id 57 up to _id 109
              • + *
              • from _id 110 up to _id 256
              • + *
              • from _id 257 up to the end of the collection, so basically data with _id greater + * than 257
              • + *
              + * + * @param splitKeys The list of split keys. + * @param additionalFilter A custom (user) additional filter to append to the range filters. + * @return A list of filters containing the ranges. + */ + private static List splitKeysToFilters(List splitKeys, String + additionalFilter) { + 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; + 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\")}}", + splitKey); + } else if (i == splitKeys.size() - 1) { + // this is the last split in the list, the filter defines + // the range from the split up to the end + rangeFilter = String.format("{ $and: [ {\"_id\":{$gt:ObjectId(\"%s\")}}", + splitKey); + } else { + // we are between two splits + rangeFilter = String.format("{ $and: [ {\"_id\":{$gt:ObjectId(\"%s\")," + + "$lte:ObjectId(\"%s\")}}", lowestBound, splitKey); + } + if (additionalFilter != null && !additionalFilter.isEmpty()) { + // user provided a filter, we append the user filter to the range filter + rangeFilter = String.format("%s,%s ]}", rangeFilter, additionalFilter); + } else { + // user didn't provide a filter, just cleany close the range filter + rangeFilter = String.format("%s ]}", rangeFilter); + } + + filters.add(rangeFilter); + + lowestBound = splitKey; + } + return filters; + } + } + + private static class BoundedMongoDbReader extends BoundedSource.BoundedReader { + + private final BoundedMongoDbSource source; + + private MongoClient client; + private MongoCursor cursor; + private String current; + + public BoundedMongoDbReader(BoundedMongoDbSource source) { + this.source = source; + } + + @Override + public boolean start() { + client = new MongoClient(new MongoClientURI(source.uri)); + + MongoDatabase mongoDatabase = client.getDatabase(source.database); + + MongoCollection mongoCollection = mongoDatabase.getCollection(source.collection); + + if (source.filter == null) { + cursor = mongoCollection.find().iterator(); + } else { + Document bson = Document.parse(source.filter); + cursor = mongoCollection.find(bson).iterator(); + } + + return advance(); + } + + @Override + public boolean advance() { + if (cursor.hasNext()) { + current = cursor.next().toJson(); + return true; + } else { + return false; + } + } + + @Override + public BoundedSource getCurrentSource() { + return source; + } + + @Override + public String getCurrent() { + return current; + } + + @Override + public void close() { + try { + if (cursor != null) { + cursor.close(); + } + } catch (Exception e) { + LOGGER.warn("Error closing MongoDB cursor", e); + } + try { + client.close(); + } catch (Exception e) { + LOGGER.warn("Error closing MongoDB client", e); + } + } + + } + + /** + * A {@link PTransform} to write to a MongoDB database. + */ + public static class Write extends PTransform, PDone> { + + public Write withUri(String uri) { + return new Write(writer.withUri(uri)); + } + + public Write withDatabase(String database) { + return new Write(writer.withDatabase(database)); + } + + public Write withCollection(String collection) { + return new Write(writer.withCollection(collection)); + } + + public Write withBatchSize(long batchSize) { + return new Write(writer.withBatchSize(batchSize)); + } + + private final MongoDbWriter writer; + + private Write(MongoDbWriter writer) { + this.writer = writer; + } + + @Override + public PDone apply(PCollection input) { + input.apply(ParDo.of(writer)); + return PDone.in(input.getPipeline()); + } + + @Override + public void validate(PCollection input) { + writer.validate(); + } + + private static class MongoDbWriter extends DoFn { + + private final String uri; + private final String database; + private final String collection; + private final long batchSize; + + private MongoClient client; + private List batch; + + public MongoDbWriter(String uri, String database, String collection, long batchSize) { + this.uri = uri; + this.database = database; + this.collection = collection; + this.batchSize = batchSize; + } + + public MongoDbWriter withUri(String uri) { + return new MongoDbWriter(uri, database, collection, batchSize); + } + + public MongoDbWriter withDatabase(String database) { + return new MongoDbWriter(uri, database, collection, batchSize); + } + + public MongoDbWriter withCollection(String collection) { + return new MongoDbWriter(uri, database, collection, batchSize); + } + + public MongoDbWriter withBatchSize(long batchSize) { + return new MongoDbWriter(uri, database, collection, batchSize); + } + + public void validate() { + Preconditions.checkNotNull(uri, "uri"); + Preconditions.checkNotNull(database, "database"); + Preconditions.checkNotNull(collection, "collection"); + Preconditions.checkNotNull(batchSize, "batchSize"); + } + + @Setup + public void createMongoClient() throws Exception { + client = new MongoClient(new MongoClientURI(uri)); + } + + @StartBundle + public void startBundle(Context ctx) throws Exception { + batch = new ArrayList<>(); + } + + @ProcessElement + public void processElement(ProcessContext ctx) throws Exception { + String value = ctx.element(); + + batch.add(Document.parse(ctx.element())); + if (batch.size() >= batchSize) { + finishBundle(ctx); + } + } + + @FinishBundle + public void finishBundle(Context ctx) throws Exception { + MongoDatabase mongoDatabase = client.getDatabase(database); + MongoCollection mongoCollection = mongoDatabase.getCollection(collection); + + mongoCollection.insertMany(batch); + + batch.clear(); + } + + @Teardown + public void closeMongoClient() throws Exception { + client.close(); + client = null; + } + } + } +} diff --git a/sdks/java/io/mongodb/src/main/java/org/apache/beam/sdk/io/mongodb/package-info.java b/sdks/java/io/mongodb/src/main/java/org/apache/beam/sdk/io/mongodb/package-info.java new file mode 100644 index 0000000000000..fd08b5854d3f8 --- /dev/null +++ b/sdks/java/io/mongodb/src/main/java/org/apache/beam/sdk/io/mongodb/package-info.java @@ -0,0 +1,22 @@ +/* + * 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. + */ + +/** + * Transforms for reading and writing from MongoDB. + */ +package org.apache.beam.sdk.io.mongodb; 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 new file mode 100644 index 0000000000000..308e0719ff5f9 --- /dev/null +++ b/sdks/java/io/mongodb/src/test/java/org/apache/beam/sdk/io/mongodb/MongoDbIOTest.java @@ -0,0 +1,209 @@ +/* + * 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.mongodb; + +import static org.junit.Assert.assertEquals; + +import com.mongodb.MongoClient; +import com.mongodb.client.MongoCollection; +import com.mongodb.client.MongoCursor; +import com.mongodb.client.MongoDatabase; + +import de.flapdoodle.embed.mongo.MongodExecutable; +import de.flapdoodle.embed.mongo.MongodStarter; +import de.flapdoodle.embed.mongo.config.IMongodConfig; +import de.flapdoodle.embed.mongo.config.MongoCmdOptionsBuilder; +import de.flapdoodle.embed.mongo.config.MongodConfigBuilder; +import de.flapdoodle.embed.mongo.config.Net; +import de.flapdoodle.embed.mongo.config.Storage; +import de.flapdoodle.embed.mongo.distribution.Version; +import de.flapdoodle.embed.process.io.file.Files; +import de.flapdoodle.embed.process.runtime.Network; + +import java.io.File; +import java.io.Serializable; +import java.util.ArrayList; + +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.Count; +import org.apache.beam.sdk.transforms.Create; +import org.apache.beam.sdk.transforms.MapElements; +import org.apache.beam.sdk.transforms.SerializableFunction; +import org.apache.beam.sdk.transforms.SimpleFunction; +import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.PCollection; + +import org.bson.Document; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Test on the MongoDbIO. + */ +public class MongoDbIOTest implements Serializable { + + private static final Logger LOGGER = LoggerFactory.getLogger(MongoDbIOTest.class); + + private static final String MONGODB_LOCATION = "target/mongodb"; + private static final int PORT = 27017; + private static final String DATABASE = "beam"; + private static final String COLLECTION = "test"; + + private transient MongodExecutable mongodExecutable; + + @Before + public void setup() throws Exception { + LOGGER.info("Starting MongoDB embedded instance"); + try { + Files.forceDelete(new File(MONGODB_LOCATION)); + } catch (Exception e) { + + } + new File(MONGODB_LOCATION).mkdirs(); + IMongodConfig mongodConfig = new MongodConfigBuilder() + .version(Version.Main.PRODUCTION) + .configServer(false) + .replication(new Storage(MONGODB_LOCATION, null, 0)) + .net(new Net("localhost", PORT, Network.localhostIsIPv6())) + .cmdOptions(new MongoCmdOptionsBuilder() + .syncDelay(10) + .useNoPrealloc(true) + .useSmallFiles(true) + .useNoJournal(true) + .build()) + .build(); + mongodExecutable = MongodStarter.getDefaultInstance().prepare(mongodConfig); + mongodExecutable.start(); + + LOGGER.info("Insert test data"); + + MongoClient client = new MongoClient("localhost", PORT); + MongoDatabase database = client.getDatabase(DATABASE); + + MongoCollection collection = database.getCollection(COLLECTION); + + String[] scientists = {"Einstein", "Darwin", "Copernicus", "Pasteur", "Curie", "Faraday", + "Newton", "Bohr", "Galilei", "Maxwell"}; + for (int i = 1; i <= 1000; i++) { + int index = i % scientists.length; + Document document = new Document(); + document.append("_id", i); + document.append("scientist", scientists[index]); + collection.insertOne(document); + } + + } + + @After + public void stop() throws Exception { + LOGGER.info("Stopping MongoDB instance"); + mongodExecutable.stop(); + } + + @Test + @Category(NeedsRunner.class) + public void testFullRead() throws Exception { + TestPipeline pipeline = TestPipeline.create(); + + PCollection output = pipeline.apply( + MongoDbIO.read() + .withUri("mongodb://localhost:" + PORT) + .withDatabase(DATABASE) + .withCollection(COLLECTION)); + + PAssert.thatSingleton(output.apply("Count All", Count.globally())) + .isEqualTo(1000L); + + PAssert.that(output + .apply("Map Scientist", MapElements.via(new SimpleFunction>() { + public KV apply(String input) { + Document bson = Document.parse(input); + return KV.of(bson.getString("scientist"), null); + } + })) + .apply("Count Scientist", Count.perKey()) + ).satisfies(new SerializableFunction>, Void>() { + @Override + public Void apply(Iterable> input) { + for (KV element : input) { + assertEquals(100L, element.getValue().longValue()); + } + return null; + } + }); + + pipeline.run(); + } + + @Test + @Category(NeedsRunner.class) + public void testReadWithFilter() throws Exception { + TestPipeline pipeline = TestPipeline.create(); + + PCollection output = pipeline.apply( + MongoDbIO.read() + .withUri("mongodb://localhost:" + PORT) + .withDatabase(DATABASE) + .withCollection(COLLECTION) + .withFilter("{\"scientist\":\"Einstein\"}")); + + PAssert.thatSingleton(output.apply("Count", Count.globally())) + .isEqualTo(100L); + + pipeline.run(); + } + + @Test + @Category(NeedsRunner.class) + public void testWrite() throws Exception { + TestPipeline pipeline = TestPipeline.create(); + + ArrayList data = new ArrayList<>(); + for (int i = 0; i < 10000; i++) { + data.add(String.format("{\"scientist\":\"Test %s\"}", i)); + } + pipeline.apply(Create.of(data)) + .apply(MongoDbIO.write().withUri("mongodb://localhost:" + PORT).withDatabase("test") + .withCollection("test")); + + pipeline.run(); + + MongoClient client = new MongoClient("localhost", PORT); + MongoDatabase database = client.getDatabase("test"); + MongoCollection collection = database.getCollection("test"); + + MongoCursor cursor = collection.find().iterator(); + + int count = 0; + while (cursor.hasNext()) { + count = count + 1; + cursor.next(); + } + + Assert.assertEquals(10000, count); + + } + +} diff --git a/sdks/java/io/mongodb/src/test/java/org/apache/beam/sdk/io/mongodb/package-info.java b/sdks/java/io/mongodb/src/test/java/org/apache/beam/sdk/io/mongodb/package-info.java new file mode 100644 index 0000000000000..fd08b5854d3f8 --- /dev/null +++ b/sdks/java/io/mongodb/src/test/java/org/apache/beam/sdk/io/mongodb/package-info.java @@ -0,0 +1,22 @@ +/* + * 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. + */ + +/** + * Transforms for reading and writing from MongoDB. + */ +package org.apache.beam.sdk.io.mongodb; diff --git a/sdks/java/io/pom.xml b/sdks/java/io/pom.xml index 6cbd615a1d6e9..c4c32ed0864f7 100644 --- a/sdks/java/io/pom.xml +++ b/sdks/java/io/pom.xml @@ -38,6 +38,7 @@ jms kafka kinesis + mongodb From 5eb44aa01157ca62f1a618d1738eb064ca3a10e4 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Jean-Baptiste=20Onofr=C3=A9?= Date: Thu, 25 Aug 2016 16:19:54 +0200 Subject: [PATCH 106/346] [BEAM-242] Enable checkstyle and fix checkstyle errors in Flink runner --- runners/flink/runner/pom.xml | 2 - .../FlinkPipelineExecutionEnvironment.java | 17 +- .../runners/flink/FlinkPipelineOptions.java | 27 +-- .../runners/flink/FlinkRunnerRegistrar.java | 6 + .../beam/runners/flink/FlinkRunnerResult.java | 17 +- .../beam/runners/flink/TestFlinkRunner.java | 8 +- .../beam/runners/flink/package-info.java | 22 +++ .../FlinkBatchPipelineTranslator.java | 15 +- .../FlinkBatchTranslationContext.java | 10 +- .../translation/FlinkPipelineTranslator.java | 2 +- .../FlinkStreamingTransformTranslators.java | 5 +- .../flink/translation/TranslationMode.java | 8 +- .../translation/functions/package-info.java | 22 +++ .../flink/translation/package-info.java | 22 +++ .../types/CoderTypeSerializer.java | 2 +- .../types/EncodedValueSerializer.java | 162 +++++++++--------- .../flink/translation/types/package-info.java | 22 +++ .../utils/SerializedPipelineOptions.java | 2 +- .../flink/translation/utils/package-info.java | 22 +++ .../wrappers/DataOutputViewWrapper.java | 2 +- .../translation/wrappers/package-info.java | 22 +++ .../wrappers/streaming/DoFnOperator.java | 12 +- .../streaming/SingletonKeyedWorkItem.java | 5 + .../SingletonKeyedWorkItemCoder.java | 14 +- .../streaming/WindowDoFnOperator.java | 2 +- .../streaming/WorkItemKeySelector.java | 3 +- .../streaming/io/UnboundedFlinkSink.java | 13 +- .../streaming/io/UnboundedFlinkSource.java | 29 ++-- .../streaming/io/UnboundedSocketSource.java | 46 +++-- .../wrappers/streaming/io/package-info.java | 22 +++ .../wrappers/streaming/package-info.java | 22 +++ .../runners/flink/PipelineOptionsTest.java | 3 + .../beam/runners/flink/WriteSinkITCase.java | 3 +- .../beam/runners/flink/package-info.java | 22 +++ .../streaming/FlinkStateInternalsTest.java | 3 +- .../flink/streaming/GroupByNullKeyTest.java | 6 + .../streaming/TopWikipediaSessionsITCase.java | 2 +- .../streaming/UnboundedSourceWrapperTest.java | 33 ++-- .../runners/flink/streaming/package-info.java | 22 +++ 39 files changed, 490 insertions(+), 189 deletions(-) create mode 100644 runners/flink/runner/src/main/java/org/apache/beam/runners/flink/package-info.java create mode 100644 runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/package-info.java create mode 100644 runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/package-info.java create mode 100644 runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/package-info.java create mode 100644 runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/utils/package-info.java create mode 100644 runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/package-info.java create mode 100644 runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/package-info.java create mode 100644 runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/package-info.java create mode 100644 runners/flink/runner/src/test/java/org/apache/beam/runners/flink/package-info.java create mode 100644 runners/flink/runner/src/test/java/org/apache/beam/runners/flink/streaming/package-info.java diff --git a/runners/flink/runner/pom.xml b/runners/flink/runner/pom.xml index 08adc607abf64..7c322804e5b94 100644 --- a/runners/flink/runner/pom.xml +++ b/runners/flink/runner/pom.xml @@ -234,12 +234,10 @@ - org.apache.maven.plugins 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 d1977a445adfa..a5d33b40b8f6a 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 @@ -37,14 +37,15 @@ /** * The class that instantiates and manages the execution of a given job. * Depending on if the job is a Streaming or Batch processing one, it creates - * the adequate execution environment ({@link ExecutionEnvironment} or {@link StreamExecutionEnvironment}), - * the necessary {@link FlinkPipelineTranslator} ({@link FlinkBatchPipelineTranslator} or - * {@link FlinkStreamingPipelineTranslator}) to transform the Beam job into a Flink one, and - * executes the (translated) job. + * the adequate execution environment ({@link ExecutionEnvironment} + * or {@link StreamExecutionEnvironment}), the necessary {@link FlinkPipelineTranslator} + * ({@link FlinkBatchPipelineTranslator} or {@link FlinkStreamingPipelineTranslator}) to + * transform the Beam job into a Flink one, and executes the (translated) job. */ public class FlinkPipelineExecutionEnvironment { - private static final Logger LOG = LoggerFactory.getLogger(FlinkPipelineExecutionEnvironment.class); + private static final Logger LOG = + LoggerFactory.getLogger(FlinkPipelineExecutionEnvironment.class); private final FlinkPipelineOptions options; @@ -79,8 +80,8 @@ public class FlinkPipelineExecutionEnvironment { * Depending on if the job is a Streaming or a Batch one, this method creates * the necessary execution environment and pipeline translator, and translates * the {@link org.apache.beam.sdk.values.PCollection} program into - * a {@link org.apache.flink.api.java.DataSet} or {@link org.apache.flink.streaming.api.datastream.DataStream} - * one. + * a {@link org.apache.flink.api.java.DataSet} + * or {@link org.apache.flink.streaming.api.datastream.DataStream} one. * */ public void translate(Pipeline pipeline) { this.flinkBatchEnv = null; @@ -213,7 +214,7 @@ private StreamExecutionEnvironment createStreamExecutionEnvironment() { // If the value is not -1, then the validity checks are applied. // By default, checkpointing is disabled. long checkpointInterval = options.getCheckpointingInterval(); - if(checkpointInterval != -1) { + if (checkpointInterval != -1) { if (checkpointInterval < 1) { throw new IllegalArgumentException("The checkpoint interval must be positive"); } 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 6561fa5ef996c..1fb23ec6dcec8 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 @@ -29,7 +29,8 @@ /** * Options which can be used to configure a Flink PipelineRunner. */ -public interface FlinkPipelineOptions extends PipelineOptions, ApplicationNameOptions, StreamingOptions { +public interface FlinkPipelineOptions + extends PipelineOptions, ApplicationNameOptions, StreamingOptions { /** * List of local files to make available to workers. @@ -38,8 +39,8 @@ public interface FlinkPipelineOptions extends PipelineOptions, ApplicationNameOp *

              * The default value is the list of jars from the main program's classpath. */ - @Description("Jar-Files to send to all workers and put on the classpath. " + - "The default value is all files from the classpath.") + @Description("Jar-Files to send to all workers and put on the classpath. " + + "The default value is all files from the classpath.") @JsonIgnore List getFilesToStage(); void setFilesToStage(List value); @@ -51,9 +52,9 @@ public interface FlinkPipelineOptions extends PipelineOptions, ApplicationNameOp * Cluster in the JVM, "[collection]" will execute the pipeline on Java Collections while * "[auto]" will let the system decide where to execute the pipeline based on the environment. */ - @Description("Address of the Flink Master where the Pipeline should be executed. Can" + - " either be of the form \"host:port\" or one of the special values [local], " + - "[collection] or [auto].") + @Description("Address of the Flink Master where the Pipeline should be executed. Can" + + " either be of the form \"host:port\" or one of the special values [local], " + + "[collection] or [auto].") String getFlinkMaster(); void setFlinkMaster(String value); @@ -62,21 +63,23 @@ public interface FlinkPipelineOptions extends PipelineOptions, ApplicationNameOp Integer getParallelism(); void setParallelism(Integer value); - @Description("The interval between consecutive checkpoints (i.e. snapshots of the current pipeline state used for " + - "fault tolerance).") + @Description("The interval between consecutive checkpoints (i.e. snapshots of the current" + + "pipeline state used for fault tolerance).") @Default.Long(-1L) Long getCheckpointingInterval(); void setCheckpointingInterval(Long interval); - @Description("Sets the number of times that failed tasks are re-executed. " + - "A value of zero effectively disables fault tolerance. A value of -1 indicates " + - "that the system default value (as defined in the configuration) should be used.") + @Description("Sets the number of times that failed tasks are re-executed. " + + "A value of zero effectively disables fault tolerance. A value of -1 indicates " + + "that the system default value (as defined in the configuration) should be used.") @Default.Integer(-1) Integer getNumberOfExecutionRetries(); void setNumberOfExecutionRetries(Integer retries); - @Description("Sets the delay between executions. A value of {@code -1} indicates that the default value should be used.") + @Description("Sets the delay between executions. A value of {@code -1} " + + "indicates that the default value should be used.") @Default.Long(-1L) Long getExecutionRetryDelay(); void setExecutionRetryDelay(Long delay); + } diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkRunnerRegistrar.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkRunnerRegistrar.java index f328279bfba02..0e4b5130be918 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkRunnerRegistrar.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkRunnerRegistrar.java @@ -36,6 +36,9 @@ public class FlinkRunnerRegistrar { private FlinkRunnerRegistrar() { } + /** + * Pipeline runner registrar. + */ @AutoService(PipelineRunnerRegistrar.class) public static class Runner implements PipelineRunnerRegistrar { @Override @@ -46,6 +49,9 @@ public Iterable>> getPipelineRunners() { } } + /** + * Pipeline options registrar. + */ @AutoService(PipelineOptionsRegistrar.class) public static class Options implements PipelineOptionsRegistrar { @Override diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkRunnerResult.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkRunnerResult.java index dd0733a671c98..90bb64d31046a 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkRunnerResult.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/FlinkRunnerResult.java @@ -35,9 +35,9 @@ public class FlinkRunnerResult implements PipelineResult { private final Map aggregators; private final long runtime; public FlinkRunnerResult(Map aggregators, long runtime) { - this.aggregators = (aggregators == null || aggregators.isEmpty()) ? - Collections.emptyMap() : - Collections.unmodifiableMap(aggregators); + this.aggregators = (aggregators == null || aggregators.isEmpty()) + ? Collections.emptyMap() + : Collections.unmodifiableMap(aggregators); this.runtime = runtime; } @@ -47,7 +47,8 @@ public State getState() { } @Override - public AggregatorValues getAggregatorValues(final Aggregator aggregator) throws AggregatorRetrievalException { + public AggregatorValues getAggregatorValues(final Aggregator aggregator) + throws AggregatorRetrievalException { // TODO provide a list of all accumulator step values Object value = aggregators.get(aggregator.getName()); if (value != null) { @@ -65,10 +66,10 @@ public Map getValuesAtSteps() { @Override public String toString() { - return "FlinkRunnerResult{" + - "aggregators=" + aggregators + - ", runtime=" + runtime + - '}'; + return "FlinkRunnerResult{" + + "aggregators=" + aggregators + + ", runtime=" + runtime + + '}'; } @Override diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/TestFlinkRunner.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/TestFlinkRunner.java index dd231d6e6bdd4..67a7d3891d3d1 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/TestFlinkRunner.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/TestFlinkRunner.java @@ -26,6 +26,9 @@ import org.apache.beam.sdk.values.PInput; import org.apache.beam.sdk.values.POutput; +/** + * Test Flink runner. + */ public class TestFlinkRunner extends PipelineRunner { private FlinkRunner delegate; @@ -37,7 +40,8 @@ private TestFlinkRunner(FlinkPipelineOptions options) { } public static TestFlinkRunner fromOptions(PipelineOptions options) { - FlinkPipelineOptions flinkOptions = PipelineOptionsValidator.validate(FlinkPipelineOptions.class, options); + FlinkPipelineOptions flinkOptions = + PipelineOptionsValidator.validate(FlinkPipelineOptions.class, options); return new TestFlinkRunner(flinkOptions); } @@ -50,7 +54,7 @@ public static TestFlinkRunner create(boolean streaming) { @Override public - OutputT apply(PTransform transform, InputT input) { + OutputT apply(PTransform transform, InputT input) { return delegate.apply(transform, input); } diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/package-info.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/package-info.java new file mode 100644 index 0000000000000..57f1e599ee502 --- /dev/null +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/package-info.java @@ -0,0 +1,22 @@ +/* + * 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. + */ + +/** + * Internal implementation of the Beam runner for Apache Flink. + */ +package org.apache.beam.runners.flink; 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 66c48b07e23b1..aa38bfbbf5d2a 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 @@ -91,15 +91,20 @@ public void visitPrimitiveTransform(TransformTreeNode node) { // get the transformation corresponding to the node we are // currently visiting and translate it into its Flink alternative. PTransform transform = node.getTransform(); - BatchTransformTranslator translator = FlinkBatchTransformTranslators.getTranslator(transform); + BatchTransformTranslator translator = + FlinkBatchTransformTranslators.getTranslator(transform); if (translator == null) { LOG.info(node.getTransform().getClass().toString()); - throw new UnsupportedOperationException("The transform " + transform + " is currently not supported."); + throw new UnsupportedOperationException("The transform " + transform + + " is currently not supported."); } applyBatchTransform(transform, node, translator); } - private > void applyBatchTransform(PTransform transform, TransformTreeNode node, BatchTransformTranslator translator) { + private > void applyBatchTransform( + PTransform transform, + TransformTreeNode node, + BatchTransformTranslator translator) { @SuppressWarnings("unchecked") T typedTransform = (T) transform; @@ -116,8 +121,8 @@ public void visitPrimitiveTransform(TransformTreeNode node) { /** * A translator of a {@link PTransform}. */ - public interface BatchTransformTranslator { - void translateNode(Type transform, FlinkBatchTranslationContext context); + public interface BatchTransformTranslator { + void translateNode(TransformT transform, FlinkBatchTranslationContext context); } /** diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkBatchTranslationContext.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkBatchTranslationContext.java index 835648e43ea8f..611f5e6d93217 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkBatchTranslationContext.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkBatchTranslationContext.java @@ -40,7 +40,7 @@ * {@link FlinkBatchTransformTranslators}. */ public class FlinkBatchTranslationContext { - + private final Map> dataSets; private final Map, DataSet> broadcastDataSets; @@ -55,9 +55,9 @@ public class FlinkBatchTranslationContext { private final PipelineOptions options; private AppliedPTransform currentTransform; - + // ------------------------------------------------------------------------ - + public FlinkBatchTranslationContext(ExecutionEnvironment env, PipelineOptions options) { this.env = env; this.options = options; @@ -66,7 +66,7 @@ public FlinkBatchTranslationContext(ExecutionEnvironment env, PipelineOptions op this.danglingDataSets = new HashMap<>(); } - + // ------------------------------------------------------------------------ public Map> getDanglingDataSets() { @@ -80,7 +80,7 @@ public ExecutionEnvironment getExecutionEnvironment() { public PipelineOptions getPipelineOptions() { return options; } - + @SuppressWarnings("unchecked") public DataSet> getInputDataSet(PValue value) { // assume that the DataSet is used as an input if retrieved here diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkPipelineTranslator.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkPipelineTranslator.java index 4db929b0641bd..cba28e48e01b5 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkPipelineTranslator.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/FlinkPipelineTranslator.java @@ -39,7 +39,7 @@ public void translate(Pipeline pipeline) { } /** - * Utility formatting method + * Utility formatting method. * @param n number of spaces to generate * @return String with "|" followed by n spaces */ 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 3719fa8e5d4c7..4b819b78f00af 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,9 +18,10 @@ package org.apache.beam.runners.flink.translation; -import com.google.api.client.util.Maps; 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; @@ -29,6 +30,7 @@ import java.util.List; import java.util.Map; 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.types.CoderTypeInformation; import org.apache.beam.runners.flink.translation.types.FlinkCoder; @@ -63,7 +65,6 @@ import org.apache.beam.sdk.transforms.windowing.WindowFn; import org.apache.beam.sdk.util.AppliedCombineFn; import org.apache.beam.sdk.util.Reshuffle; -import org.apache.beam.runners.core.SystemReduceFn; 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/TranslationMode.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/TranslationMode.java index 71eb6552b7101..57b69aa147533 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/TranslationMode.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/TranslationMode.java @@ -18,14 +18,14 @@ package org.apache.beam.runners.flink.translation; /** - * The translation mode of the Beam Pipeline + * The translation mode of the Beam Pipeline. */ public enum TranslationMode { - /** Uses the batch mode of Flink */ + /** Uses the batch mode of Flink. */ BATCH, - /** Uses the streaming mode of Flink */ + /** Uses the streaming mode of Flink. */ STREAMING -} \ No newline at end of file +} diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/package-info.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/package-info.java new file mode 100644 index 0000000000000..9f1121225b881 --- /dev/null +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/functions/package-info.java @@ -0,0 +1,22 @@ +/* + * 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. + */ + +/** + * Internal implementation of the Beam runner for Apache Flink. + */ +package org.apache.beam.runners.flink.translation.functions; diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/package-info.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/package-info.java new file mode 100644 index 0000000000000..af4b35491ba41 --- /dev/null +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/package-info.java @@ -0,0 +1,22 @@ +/* + * 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. + */ + +/** + * Internal implementation of the Beam runner for Apache Flink. + */ +package org.apache.beam.runners.flink.translation; diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/CoderTypeSerializer.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/CoderTypeSerializer.java index 4eda357f64eb7..e210ed9d7b982 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/CoderTypeSerializer.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/CoderTypeSerializer.java @@ -33,7 +33,7 @@ * Dataflow {@link org.apache.beam.sdk.coders.Coder Coders}. */ public class CoderTypeSerializer extends TypeSerializer { - + private Coder coder; public CoderTypeSerializer(Coder coder) { diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/EncodedValueSerializer.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/EncodedValueSerializer.java index f3e667d575a5f..41db61edbb493 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/EncodedValueSerializer.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/EncodedValueSerializer.java @@ -18,7 +18,9 @@ package org.apache.beam.runners.flink.translation.types; import java.io.IOException; + import org.apache.beam.sdk.coders.Coder; + import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.core.memory.DataInputView; import org.apache.flink.core.memory.DataOutputView; @@ -28,84 +30,84 @@ */ public final class EncodedValueSerializer extends TypeSerializer { - private static final long serialVersionUID = 1L; - - private static final byte[] EMPTY = new byte[0]; - - @Override - public boolean isImmutableType() { - return true; - } - - @Override - public byte[] createInstance() { - return EMPTY; - } - - @Override - public byte[] copy(byte[] from) { - return from; - } - - @Override - public byte[] copy(byte[] from, byte[] reuse) { - return copy(from); - } - - @Override - public int getLength() { - return -1; - } - - - @Override - public void serialize(byte[] record, DataOutputView target) throws IOException { - if (record == null) { - throw new IllegalArgumentException("The record must not be null."); - } - - final int len = record.length; - target.writeInt(len); - target.write(record); - } - - @Override - public byte[] deserialize(DataInputView source) throws IOException { - final int len = source.readInt(); - byte[] result = new byte[len]; - source.readFully(result); - return result; - } - - @Override - public byte[] deserialize(byte[] reuse, DataInputView source) throws IOException { - return deserialize(source); - } - - @Override - public void copy(DataInputView source, DataOutputView target) throws IOException { - final int len = source.readInt(); - target.writeInt(len); - target.write(source, len); - } - - @Override - public boolean canEqual(Object obj) { - return obj instanceof EncodedValueSerializer; - } - - @Override - public int hashCode() { - return this.getClass().hashCode(); - } - - @Override - public boolean equals(Object obj) { - return obj instanceof EncodedValueSerializer; - } - - @Override - public TypeSerializer duplicate() { - return this; - } + private static final long serialVersionUID = 1L; + + private static final byte[] EMPTY = new byte[0]; + + @Override + public boolean isImmutableType() { + return true; + } + + @Override + public byte[] createInstance() { + return EMPTY; + } + + @Override + public byte[] copy(byte[] from) { + return from; + } + + @Override + public byte[] copy(byte[] from, byte[] reuse) { + return copy(from); + } + + @Override + public int getLength() { + return -1; + } + + + @Override + public void serialize(byte[] record, DataOutputView target) throws IOException { + if (record == null) { + throw new IllegalArgumentException("The record must not be null."); + } + + final int len = record.length; + target.writeInt(len); + target.write(record); + } + + @Override + public byte[] deserialize(DataInputView source) throws IOException { + final int len = source.readInt(); + byte[] result = new byte[len]; + source.readFully(result); + return result; + } + + @Override + public byte[] deserialize(byte[] reuse, DataInputView source) throws IOException { + return deserialize(source); + } + + @Override + public void copy(DataInputView source, DataOutputView target) throws IOException { + final int len = source.readInt(); + target.writeInt(len); + target.write(source, len); + } + + @Override + public boolean canEqual(Object obj) { + return obj instanceof EncodedValueSerializer; + } + + @Override + public int hashCode() { + return this.getClass().hashCode(); + } + + @Override + public boolean equals(Object obj) { + return obj instanceof EncodedValueSerializer; + } + + @Override + public TypeSerializer duplicate() { + return this; + } } diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/package-info.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/package-info.java new file mode 100644 index 0000000000000..6fb3182f27c57 --- /dev/null +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/types/package-info.java @@ -0,0 +1,22 @@ +/* + * 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. + */ + +/** + * Internal implementation of the Beam runner for Apache Flink. + */ +package org.apache.beam.runners.flink.translation.types; diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/utils/SerializedPipelineOptions.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/utils/SerializedPipelineOptions.java index 0c6cea8dfd29a..fe2602b3377b9 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/utils/SerializedPipelineOptions.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/utils/SerializedPipelineOptions.java @@ -33,7 +33,7 @@ public class SerializedPipelineOptions implements Serializable { private final byte[] serializedOptions; - /** Lazily initialized copy of deserialized options */ + /** Lazily initialized copy of deserialized options. */ private transient PipelineOptions pipelineOptions; public SerializedPipelineOptions(PipelineOptions options) { diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/utils/package-info.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/utils/package-info.java new file mode 100644 index 0000000000000..5dedd53c77520 --- /dev/null +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/utils/package-info.java @@ -0,0 +1,22 @@ +/* + * 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. + */ + +/** + * Internal implementation of the Beam runner for Apache Flink. + */ +package org.apache.beam.runners.flink.translation.utils; diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/DataOutputViewWrapper.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/DataOutputViewWrapper.java index 2cb9b188ff064..f2d9db222c56e 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/DataOutputViewWrapper.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/DataOutputViewWrapper.java @@ -28,7 +28,7 @@ * {@link java.io.OutputStream}. */ public class DataOutputViewWrapper extends OutputStream { - + private DataOutputView outputView; public DataOutputViewWrapper(DataOutputView outputView) { diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/package-info.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/package-info.java new file mode 100644 index 0000000000000..72f7debb434f9 --- /dev/null +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/package-info.java @@ -0,0 +1,22 @@ +/* + * 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. + */ + +/** + * Internal implementation of the Beam runner for Apache Flink. + */ +package org.apache.beam.runners.flink.translation.wrappers; 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 3b0fccc2b3650..3b917e2f4bdda 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 @@ -27,6 +27,9 @@ import java.util.HashMap; import java.util.List; import java.util.Map; +import org.apache.beam.runners.core.DoFnRunner; +import org.apache.beam.runners.core.DoFnRunners; +import org.apache.beam.runners.core.PushbackSideInputDoFnRunner; import org.apache.beam.runners.core.SideInputHandler; import org.apache.beam.runners.flink.translation.utils.SerializedPipelineOptions; import org.apache.beam.runners.flink.translation.wrappers.SerializableFnAggregatorWrapper; @@ -40,11 +43,8 @@ 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.runners.core.DoFnRunner; -import org.apache.beam.runners.core.DoFnRunners; import org.apache.beam.sdk.util.ExecutionContext; import org.apache.beam.sdk.util.NullSideInputReader; -import org.apache.beam.runners.core.PushbackSideInputDoFnRunner; import org.apache.beam.sdk.util.SideInputReader; import org.apache.beam.sdk.util.TimerInternals; import org.apache.beam.sdk.util.WindowedValue; @@ -79,8 +79,8 @@ * * @param the input type of the {@link OldDoFn} * @param the output type of the {@link OldDoFn} - * @param the output type of the operator, this can be different from the fn output type when we have - * side outputs + * @param the output type of the operator, this can be different from the fn output + * type when we have side outputs */ public class DoFnOperator extends AbstractStreamOperator @@ -166,7 +166,7 @@ public void open() throws Exception { currentInputWatermark = Long.MIN_VALUE; currentOutputWatermark = currentInputWatermark; - Aggregator.AggregatorFactory aggregatorFactory = new Aggregator.AggregatorFactory() { + Aggregator.AggregatorFactory aggregatorFactory = new Aggregator.AggregatorFactory() { @Override public Aggregator createAggregatorForDoFn( Class fnClass, 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 5751aac781239..6d2582b5da5b9 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 @@ -22,6 +22,11 @@ import org.apache.beam.sdk.util.TimerInternals; import org.apache.beam.sdk.util.WindowedValue; +/** + * Singleton keyed word item. + * @param + * @param + */ public class SingletonKeyedWorkItem implements KeyedWorkItem { final K key; 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 5e583e9013c27..37454a31c4bfe 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 @@ -35,7 +35,13 @@ import org.apache.beam.sdk.util.PropertyNames; import org.apache.beam.sdk.util.WindowedValue; -public class SingletonKeyedWorkItemCoder extends StandardCoder> { +/** + * Singleton keyed word iteam coder. + * @param + * @param + */ +public class SingletonKeyedWorkItemCoder + extends StandardCoder> { /** * Create a new {@link KeyedWorkItemCoder} with the provided key coder, element coder, and window * coder. @@ -68,7 +74,7 @@ private SingletonKeyedWorkItemCoder( this.keyCoder = keyCoder; this.elemCoder = elemCoder; this.windowCoder = windowCoder; - valueCoder= WindowedValue.FullWindowedValueCoder.of(elemCoder, windowCoder); + valueCoder = WindowedValue.FullWindowedValueCoder.of(elemCoder, windowCoder); } public Coder getKeyCoder() { @@ -80,7 +86,9 @@ public Coder getElementCoder() { } @Override - public void encode(SingletonKeyedWorkItem value, OutputStream outStream, Context context) + public void encode(SingletonKeyedWorkItem value, + OutputStream outStream, + Context context) throws CoderException, IOException { Context nestedContext = context.nested(); keyCoder.encode(value.key(), outStream, nestedContext); 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 b8931161045e7..29ae6ae63f8a3 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 @@ -34,6 +34,7 @@ import java.util.Set; import javax.annotation.Nullable; import org.apache.beam.runners.core.GroupAlsoByWindowViaWindowSetDoFn; +import org.apache.beam.runners.core.SystemReduceFn; import org.apache.beam.runners.flink.translation.wrappers.DataInputViewWrapper; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.options.PipelineOptions; @@ -43,7 +44,6 @@ 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.runners.core.SystemReduceFn; 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/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 51d9e0c9399cb..7829163e6a261 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 @@ -33,7 +33,8 @@ * that all key comparisons/hashing happen on the encoded form. */ public class WorkItemKeySelector - implements KeySelector>, ByteBuffer>, ResultTypeQueryable { + implements KeySelector>, ByteBuffer>, + ResultTypeQueryable { private final Coder keyCoder; diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/UnboundedFlinkSink.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/UnboundedFlinkSink.java index 2117e9d193533..5b01796fe1f97 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/UnboundedFlinkSink.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/UnboundedFlinkSink.java @@ -62,7 +62,8 @@ public void initialize(PipelineOptions options) throws Exception { } @Override - public void finalize(Iterable writerResults, PipelineOptions options) throws Exception { + public void finalize(Iterable writerResults, PipelineOptions options) + throws Exception { } @@ -70,12 +71,14 @@ public void finalize(Iterable writerResults, PipelineOptions options) th public Coder getWriterResultCoder() { return new Coder() { @Override - public void encode(Object value, OutputStream outStream, Context context) throws CoderException, IOException { + public void encode(Object value, OutputStream outStream, Context context) + throws CoderException, IOException { } @Override - public Object decode(InputStream inStream, Context context) throws CoderException, IOException { + public Object decode(InputStream inStream, Context context) + throws CoderException, IOException { return null; } @@ -110,7 +113,9 @@ public boolean isRegisterByteSizeObserverCheap(Object value, Context context) { } @Override - public void registerByteSizeObserver(Object value, ElementByteSizeObserver observer, Context context) throws Exception { + public void registerByteSizeObserver(Object value, + ElementByteSizeObserver observer, + Context context) throws Exception { } diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/UnboundedFlinkSource.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/UnboundedFlinkSource.java index c6e0825648f46..ac20c34ff204f 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/UnboundedFlinkSource.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/UnboundedFlinkSource.java @@ -36,17 +36,19 @@ public class UnboundedFlinkSource extends UnboundedSource flinkSource; - /** Coder set during translation */ + /** Coder set during translation. */ private Coder coder; - /** Timestamp / watermark assigner for source; defaults to ingestion time */ - private AssignerWithPeriodicWatermarks flinkTimestampAssigner = new IngestionTimeExtractor(); + /** Timestamp / watermark assigner for source; defaults to ingestion time. */ + private AssignerWithPeriodicWatermarks flinkTimestampAssigner = + new IngestionTimeExtractor(); public UnboundedFlinkSource(SourceFunction source) { flinkSource = checkNotNull(source); } - public UnboundedFlinkSource(SourceFunction source, AssignerWithPeriodicWatermarks timestampAssigner) { + public UnboundedFlinkSource(SourceFunction source, + AssignerWithPeriodicWatermarks timestampAssigner) { flinkSource = checkNotNull(source); flinkTimestampAssigner = checkNotNull(timestampAssigner); } @@ -60,19 +62,25 @@ public AssignerWithPeriodicWatermarks getFlinkTimestampAssigner() { } @Override - public List> generateInitialSplits(int desiredNumSplits, PipelineOptions options) throws Exception { - throw new RuntimeException("Flink Sources are supported only when running with the FlinkRunner."); + public List> generateInitialSplits( + int desiredNumSplits, + PipelineOptions options) throws Exception { + throw new RuntimeException("Flink Sources are supported only when " + + "running with the FlinkRunner."); } @Override - public UnboundedReader createReader(PipelineOptions options, @Nullable CheckpointMark checkpointMark) { - throw new RuntimeException("Flink Sources are supported only when running with the FlinkRunner."); + public UnboundedReader createReader(PipelineOptions options, + @Nullable CheckpointMark checkpointMark) { + throw new RuntimeException("Flink Sources are supported only when " + + "running with the FlinkRunner."); } @Nullable @Override public Coder getCheckpointMarkCoder() { - throw new RuntimeException("Flink Sources are supported only when running with the FlinkRunner."); + throw new RuntimeException("Flink Sources are supported only when " + + "running with the FlinkRunner."); } @@ -100,7 +108,8 @@ public void setFlinkTimestampAssigner(AssignerWithPeriodicWatermarks flinkTim * @param The type that the source function produces. * @return The wrapped source function. */ - public static UnboundedSource of(SourceFunction flinkSource) { + public static UnboundedSource of( + SourceFunction flinkSource) { return new UnboundedFlinkSource<>(flinkSource); } diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/UnboundedSocketSource.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/UnboundedSocketSource.java index 8d37fe70dfb8b..96b51389deaf2 100644 --- a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/UnboundedSocketSource.java +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/UnboundedSocketSource.java @@ -38,9 +38,11 @@ import org.slf4j.LoggerFactory; /** - * An example unbounded Beam source that reads input from a socket. This is used mainly for testing and debugging. + * An example unbounded Beam source that reads input from a socket. + * This is used mainly for testing and debugging. * */ -public class UnboundedSocketSource extends UnboundedSource { +public class UnboundedSocketSource + extends UnboundedSource { private static final Coder DEFAULT_SOCKET_CODER = StringUtf8Coder.of(); @@ -60,7 +62,11 @@ public UnboundedSocketSource(String hostname, int port, char delimiter, long max this(hostname, port, delimiter, maxNumRetries, DEFAULT_CONNECTION_RETRY_SLEEP); } - public UnboundedSocketSource(String hostname, int port, char delimiter, long maxNumRetries, long delayBetweenRetries) { + public UnboundedSocketSource(String hostname, + int port, + char delimiter, + long maxNumRetries, + long delayBetweenRetries) { this.hostname = hostname; this.port = port; this.delimiter = delimiter; @@ -89,12 +95,15 @@ public long getDelayBetweenRetries() { } @Override - public List> generateInitialSplits(int desiredNumSplits, PipelineOptions options) throws Exception { - return Collections.>singletonList(this); + public List> generateInitialSplits( + int desiredNumSplits, + PipelineOptions options) throws Exception { + return Collections.>singletonList(this); } @Override - public UnboundedReader createReader(PipelineOptions options, @Nullable C checkpointMark) { + public UnboundedReader createReader(PipelineOptions options, + @Nullable CheckpointMarkT checkpointMark) { return new UnboundedSocketReader(this); } @@ -109,7 +118,8 @@ public Coder getCheckpointMarkCoder() { @Override public void validate() { checkArgument(port > 0 && port < 65536, "port is out of range"); - checkArgument(maxNumRetries >= -1, "maxNumRetries must be zero or larger (num retries), or -1 (infinite retries)"); + checkArgument(maxNumRetries >= -1, "maxNumRetries must be zero or larger (num retries), " + + "or -1 (infinite retries)"); checkArgument(delayBetweenRetries >= 0, "delayBetweenRetries must be zero or positive"); } @@ -118,7 +128,11 @@ public Coder getDefaultOutputCoder() { return DEFAULT_SOCKET_CODER; } - public static class UnboundedSocketReader extends UnboundedSource.UnboundedReader implements Serializable { + /** + * Unbounded socket reader. + */ + public static class UnboundedSocketReader extends UnboundedSource.UnboundedReader + implements Serializable { private static final long serialVersionUID = 7526472295622776147L; private static final Logger LOG = LoggerFactory.getLogger(UnboundedSocketReader.class); @@ -138,7 +152,8 @@ public UnboundedSocketReader(UnboundedSocketSource source) { private void openConnection() throws IOException { this.socket = new Socket(); - this.socket.connect(new InetSocketAddress(this.source.getHostname(), this.source.getPort()), CONNECTION_TIMEOUT_TIME); + this.socket.connect(new InetSocketAddress(this.source.getHostname(), this.source.getPort()), + CONNECTION_TIMEOUT_TIME); this.reader = new BufferedReader(new InputStreamReader(this.socket.getInputStream())); this.isRunning = true; } @@ -149,11 +164,14 @@ public boolean start() throws IOException { while (!isRunning) { try { openConnection(); - LOG.info("Connected to server socket " + this.source.getHostname() + ':' + this.source.getPort()); + LOG.info("Connected to server socket " + this.source.getHostname() + ':' + + this.source.getPort()); return advance(); } catch (IOException e) { - LOG.info("Lost connection to server socket " + this.source.getHostname() + ':' + this.source.getPort() + ". Retrying in " + this.source.getDelayBetweenRetries() + " msecs..."); + LOG.info("Lost connection to server socket " + this.source.getHostname() + ':' + + this.source.getPort() + ". Retrying in " + + this.source.getDelayBetweenRetries() + " msecs..."); if (this.source.getMaxNumRetries() == -1 || attempt++ < this.source.getMaxNumRetries()) { try { @@ -167,7 +185,8 @@ public boolean start() throws IOException { } } } - LOG.error("Unable to connect to host " + this.source.getHostname() + " : " + this.source.getPort()); + LOG.error("Unable to connect to host " + this.source.getHostname() + + " : " + this.source.getPort()); return false; } @@ -211,7 +230,8 @@ public void close() throws IOException { this.reader.close(); this.socket.close(); this.isRunning = false; - LOG.info("Closed connection to server socket at " + this.source.getHostname() + ":" + this.source.getPort() + "."); + LOG.info("Closed connection to server socket at " + this.source.getHostname() + ":" + + this.source.getPort() + "."); } @Override diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/package-info.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/package-info.java new file mode 100644 index 0000000000000..b431ce745bde5 --- /dev/null +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/package-info.java @@ -0,0 +1,22 @@ +/* + * 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. + */ + +/** + * Internal implementation of the Beam runner for Apache Flink. + */ +package org.apache.beam.runners.flink.translation.wrappers.streaming.io; diff --git a/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/package-info.java b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/package-info.java new file mode 100644 index 0000000000000..0674871307e65 --- /dev/null +++ b/runners/flink/runner/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/package-info.java @@ -0,0 +1,22 @@ +/* + * 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. + */ + +/** + * Internal implementation of the Beam runner for Apache Flink. + */ +package org.apache.beam.runners.flink.translation.wrappers.streaming; 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 32339dce5165f..3c30fed923bac 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 @@ -52,6 +52,9 @@ */ public class PipelineOptionsTest { + /** + * Pipeline options. + */ public interface MyOptions extends FlinkPipelineOptions { @Description("Bla bla bla") @Default.String("Hello") diff --git a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/WriteSinkITCase.java b/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/WriteSinkITCase.java index 09881463f6c23..37eedb220bac2 100644 --- a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/WriteSinkITCase.java +++ b/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/WriteSinkITCase.java @@ -118,7 +118,8 @@ public void initialize(PipelineOptions options) throws Exception { } @Override - public void finalize(Iterable writerResults, PipelineOptions options) throws Exception { + public void finalize(Iterable writerResults, PipelineOptions options) + throws Exception { } diff --git a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/package-info.java b/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/package-info.java new file mode 100644 index 0000000000000..57f1e599ee502 --- /dev/null +++ b/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/package-info.java @@ -0,0 +1,22 @@ +/* + * 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. + */ + +/** + * Internal implementation of the Beam runner for Apache Flink. + */ +package org.apache.beam.runners.flink; diff --git a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/streaming/FlinkStateInternalsTest.java b/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/streaming/FlinkStateInternalsTest.java index 711ae0001f795..628212a4e5f1c 100644 --- a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/streaming/FlinkStateInternalsTest.java +++ b/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/streaming/FlinkStateInternalsTest.java @@ -56,7 +56,8 @@ import org.junit.runners.JUnit4; /** - * Tests for {@link FlinkStateInternals}. This is based on the tests for {@code InMemoryStateInternals}. + * Tests for {@link FlinkStateInternals}. This is based on the tests for + * {@code InMemoryStateInternals}. */ @RunWith(JUnit4.class) public class FlinkStateInternalsTest { 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 ab98c27733b3c..c6381ee4002e2 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 @@ -36,6 +36,9 @@ import org.joda.time.Duration; import org.joda.time.Instant; +/** + * Test for GroupByNullKey. + */ public class GroupByNullKeyTest extends StreamingProgramTestBase implements Serializable { @@ -58,6 +61,9 @@ protected void postSubmit() throws Exception { compareResultsByLinesInMemory(Joiner.on('\n').join(EXPECTED_RESULT), resultPath); } + /** + * DoFn extracting user and timestamp. + */ public static class ExtractUserAndTimestamp extends OldDoFn, String> { private static final long serialVersionUID = 0; 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 64f978fa0f234..9410481f054e9 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 @@ -38,7 +38,7 @@ /** - * Session window test + * Session window test. */ public class TopWikipediaSessionsITCase extends StreamingProgramTestBase implements Serializable { protected String resultPath; diff --git a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/streaming/UnboundedSourceWrapperTest.java b/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/streaming/UnboundedSourceWrapperTest.java index a70ad49b9f73a..73124a9eed72a 100644 --- a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/streaming/UnboundedSourceWrapperTest.java +++ b/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/streaming/UnboundedSourceWrapperTest.java @@ -56,14 +56,14 @@ public class UnboundedSourceWrapperTest { */ @Test public void testWithOneReader() throws Exception { - final int NUM_ELEMENTS = 20; + final int numElements = 20; final Object checkpointLock = new Object(); PipelineOptions options = PipelineOptionsFactory.create(); // this source will emit exactly NUM_ELEMENTS across all parallel readers, // afterwards it will stall. We check whether we also receive NUM_ELEMENTS // elements later. - TestCountingSource source = new TestCountingSource(NUM_ELEMENTS); + TestCountingSource source = new TestCountingSource(numElements); UnboundedSourceWrapper, TestCountingSource.CounterMark> flinkWrapper = new UnboundedSourceWrapper<>(options, source, 1); @@ -92,7 +92,7 @@ public void collect( StreamRecord>> windowedValueStreamRecord) { count++; - if (count >= NUM_ELEMENTS) { + if (count >= numElements) { throw new SuccessException(); } } @@ -116,14 +116,14 @@ public void close() { */ @Test public void testWithMultipleReaders() throws Exception { - final int NUM_ELEMENTS = 20; + final int numElements = 20; final Object checkpointLock = new Object(); PipelineOptions options = PipelineOptionsFactory.create(); // this source will emit exactly NUM_ELEMENTS across all parallel readers, // afterwards it will stall. We check whether we also receive NUM_ELEMENTS // elements later. - TestCountingSource source = new TestCountingSource(NUM_ELEMENTS); + TestCountingSource source = new TestCountingSource(numElements); UnboundedSourceWrapper, TestCountingSource.CounterMark> flinkWrapper = new UnboundedSourceWrapper<>(options, source, 4); @@ -149,10 +149,10 @@ public void emitWatermark(Watermark watermark) { @Override public void collect( - StreamRecord>> windowedValueStreamRecord) { + StreamRecord>> windowedValueStreamRecord) { count++; - if (count >= NUM_ELEMENTS) { + if (count >= numElements) { throw new SuccessException(); } } @@ -177,14 +177,14 @@ public void close() { */ @Test public void testRestore() throws Exception { - final int NUM_ELEMENTS = 20; + final int numElements = 20; final Object checkpointLock = new Object(); PipelineOptions options = PipelineOptionsFactory.create(); // this source will emit exactly NUM_ELEMENTS across all parallel readers, // afterwards it will stall. We check whether we also receive NUM_ELEMENTS // elements later. - TestCountingSource source = new TestCountingSource(NUM_ELEMENTS); + TestCountingSource source = new TestCountingSource(numElements); UnboundedSourceWrapper, TestCountingSource.CounterMark> flinkWrapper = new UnboundedSourceWrapper<>(options, source, 1); @@ -213,11 +213,11 @@ public void emitWatermark(Watermark watermark) { @Override public void collect( - StreamRecord>> windowedValueStreamRecord) { + StreamRecord>> windowedValueStreamRecord) { emittedElements.add(windowedValueStreamRecord.getValue().getValue()); count++; - if (count >= NUM_ELEMENTS / 2) { + if (count >= numElements / 2) { throw new SuccessException(); } } @@ -238,7 +238,7 @@ public void close() { byte[] snapshot = flinkWrapper.snapshotState(0, 0); // create a completely new source but restore from the snapshot - TestCountingSource restoredSource = new TestCountingSource(NUM_ELEMENTS); + TestCountingSource restoredSource = new TestCountingSource(numElements); UnboundedSourceWrapper< KV, TestCountingSource.CounterMark> restoredFlinkWrapper = new UnboundedSourceWrapper<>(options, restoredSource, 1); @@ -271,10 +271,10 @@ public void emitWatermark(Watermark watermark) { @Override public void collect( - StreamRecord>> windowedValueStreamRecord) { + StreamRecord>> windowedValueStreamRecord) { emittedElements.add(windowedValueStreamRecord.getValue().getValue()); count++; - if (count >= NUM_ELEMENTS / 2) { + if (count >= numElements / 2) { throw new SuccessException(); } } @@ -292,7 +292,7 @@ public void close() { assertTrue("Did not successfully read second batch of elements.", readSecondBatchOfElements); // verify that we saw all NUM_ELEMENTS elements - assertTrue(emittedElements.size() == NUM_ELEMENTS); + assertTrue(emittedElements.size() == numElements); } @SuppressWarnings("unchecked") @@ -310,7 +310,8 @@ private static void setupSourceOperator(StreamSource operator) { when(mockTask.getConfiguration()).thenReturn(cfg); when(mockTask.getEnvironment()).thenReturn(env); when(mockTask.getExecutionConfig()).thenReturn(executionConfig); - when(mockTask.getAccumulatorMap()).thenReturn(Collections.>emptyMap()); + when(mockTask.getAccumulatorMap()) + .thenReturn(Collections.>emptyMap()); operator.setup(mockTask, cfg, (Output< StreamRecord>) mock(Output.class)); } diff --git a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/streaming/package-info.java b/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/streaming/package-info.java new file mode 100644 index 0000000000000..08a1e03ec243e --- /dev/null +++ b/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/streaming/package-info.java @@ -0,0 +1,22 @@ +/* + * 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. + */ + +/** + * Internal implementation of the Beam runner for Apache Flink. + */ +package org.apache.beam.runners.flink.streaming; From 958f3fecb0dcfae48b237f1a2f84c5d24c868752 Mon Sep 17 00:00:00 2001 From: Dan Halperin Date: Sun, 4 Sep 2016 14:54:42 -0700 Subject: [PATCH 107/346] BigQuery: limit max job polling time to 1 minute Before the backoff would grow unboundedly, so we could in principle wait 1.5x to 2x the actual job time. For long running jobs this is hours. Now, we just back off at most 1 minute between checking the job state. Note there should be no danger of QPS overload here because we should have very few concurrent outstanding jobs --- .../beam/sdk/io/gcp/bigquery/BigQueryServicesImpl.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/BigQueryServicesImpl.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryServicesImpl.java index 8b5e8c28aeb24..20dadff23f9d6 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 @@ -232,11 +232,12 @@ static void startJob( } @Override - public Job pollJob(JobReference jobRef, int maxAttempts) - throws InterruptedException { + public Job pollJob(JobReference jobRef, int maxAttempts) throws InterruptedException { BackOff backoff = FluentBackoff.DEFAULT - .withMaxRetries(maxAttempts).withInitialBackoff(INITIAL_JOB_STATUS_POLL_BACKOFF) + .withMaxRetries(maxAttempts) + .withInitialBackoff(INITIAL_JOB_STATUS_POLL_BACKOFF) + .withMaxBackoff(Duration.standardMinutes(1)) .backoff(); return pollJob(jobRef, Sleeper.DEFAULT, backoff); } From c8052b6e1b65e37341b41e827f95b31e0df0be99 Mon Sep 17 00:00:00 2001 From: Thomas Groh Date: Fri, 2 Sep 2016 10:43:43 -0700 Subject: [PATCH 108/346] Be more accepting in UnboundedReadDeduplicatorTest Don't depend on all the threads failing. Instead, assert that at most one success was encountered, and we saw at most numThreads - 1 failures. --- .../runners/direct/UnboundedReadDeduplicatorTest.java | 11 +++++++---- 1 file changed, 7 insertions(+), 4 deletions(-) diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/UnboundedReadDeduplicatorTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/UnboundedReadDeduplicatorTest.java index 7d2a95cd167c7..0aa2c493a821c 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/UnboundedReadDeduplicatorTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/UnboundedReadDeduplicatorTest.java @@ -19,6 +19,7 @@ import static org.hamcrest.Matchers.equalTo; import static org.hamcrest.Matchers.is; +import static org.hamcrest.Matchers.lessThan; import static org.junit.Assert.assertThat; import java.util.concurrent.CountDownLatch; @@ -60,18 +61,18 @@ public void cachedIdDeduplicatorMultithreaded() throws InterruptedException { byte[] id = new byte[] {-1, 2, 4, 22}; UnboundedReadDeduplicator dedupper = CachedIdDeduplicator.create(); final CountDownLatch startSignal = new CountDownLatch(1); - int numThreads = 1000; + int numThreads = 50; final CountDownLatch readyLatch = new CountDownLatch(numThreads); final CountDownLatch finishLine = new CountDownLatch(numThreads); ExecutorService executor = Executors.newCachedThreadPool(); AtomicInteger successCount = new AtomicInteger(); - AtomicInteger failureCount = new AtomicInteger(); + AtomicInteger noOutputCount = new AtomicInteger(); for (int i = 0; i < numThreads; i++) { executor.submit(new TryOutputIdRunnable(dedupper, id, successCount, - failureCount, + noOutputCount, readyLatch, startSignal, finishLine)); @@ -82,8 +83,10 @@ public void cachedIdDeduplicatorMultithreaded() throws InterruptedException { finishLine.await(10L, TimeUnit.SECONDS); executor.shutdownNow(); + // The first thread to run will succeed, and no others will assertThat(successCount.get(), equalTo(1)); - assertThat(failureCount.get(), equalTo(numThreads - 1)); + // The threads may not all complete; all of the threads that do not succeed must not output + assertThat(noOutputCount.get(), lessThan(numThreads)); } private static class TryOutputIdRunnable implements Runnable { From 8f68085bf4f66425d7b5b4835c844d523deb3e5e Mon Sep 17 00:00:00 2001 From: Thomas Groh Date: Tue, 6 Sep 2016 11:13:07 -0700 Subject: [PATCH 109/346] Remove timeout in JAXBCoderTest The infrastructure should kill the test if the await does not return. --- .../org/apache/beam/sdk/coders/JAXBCoderTest.java | 13 +++++-------- 1 file changed, 5 insertions(+), 8 deletions(-) diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/JAXBCoderTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/JAXBCoderTest.java index 129be61ae7c1e..36190f9523c64 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/JAXBCoderTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/coders/JAXBCoderTest.java @@ -18,7 +18,6 @@ package org.apache.beam.sdk.coders; import static org.junit.Assert.assertEquals; -import static org.junit.Assert.fail; import com.google.common.collect.ImmutableList; import java.io.IOException; @@ -28,7 +27,6 @@ import java.util.concurrent.CountDownLatch; import java.util.concurrent.Executor; import java.util.concurrent.Executors; -import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicReference; import javax.xml.bind.annotation.XmlRootElement; import org.apache.beam.sdk.testing.CoderProperties; @@ -119,7 +117,7 @@ public void testEncodeDecodeNested() throws Exception { @Test public void testEncodeDecodeMultithreaded() throws Throwable { final JAXBCoder coder = JAXBCoder.of(TestType.class); - int numThreads = 1000; + int numThreads = 100; final CountDownLatch ready = new CountDownLatch(numThreads); final CountDownLatch start = new CountDownLatch(1); @@ -155,11 +153,10 @@ public void run() { ready.await(); start.countDown(); - if (!done.await(10L, TimeUnit.SECONDS)) { - fail("Should be able to clone " + numThreads + " elements in 10 seconds"); - } - if (thrown.get() != null) { - throw thrown.get(); + done.await(); + Throwable actuallyThrown = thrown.get(); + if (actuallyThrown != null) { + throw actuallyThrown; } } From 50c1c88160c013dcc36ba6488b43405ad2463842 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Jean-Baptiste=20Onofr=C3=A9?= Date: Tue, 6 Sep 2016 07:26:45 +0200 Subject: [PATCH 110/346] [BEAM-242] Enable and fix checkstyle in Flink runner examples --- runners/flink/examples/pom.xml | 2 -- .../runners/flink/examples/WordCount.java | 9 +++++ .../runners/flink/examples/package-info.java | 22 ++++++++++++ .../examples/streaming/AutoComplete.java | 5 +-- .../examples/streaming/JoinExamples.java | 3 +- .../examples/streaming/KafkaIOExamples.java | 34 +++++++++---------- .../KafkaWindowedWordCountExample.java | 27 ++++++++++++--- .../examples/streaming/WindowedWordCount.java | 19 +++++++---- .../examples/streaming/package-info.java | 22 ++++++++++++ 9 files changed, 110 insertions(+), 33 deletions(-) create mode 100644 runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/package-info.java create mode 100644 runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/streaming/package-info.java diff --git a/runners/flink/examples/pom.xml b/runners/flink/examples/pom.xml index 9f705db88dbeb..b8489fc23871a 100644 --- a/runners/flink/examples/pom.xml +++ b/runners/flink/examples/pom.xml @@ -109,12 +109,10 @@ - org.apache.maven.plugins 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 ab9297f72d8ef..9cce757980769 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 @@ -36,8 +36,14 @@ import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; +/** + * Wordcount pipeline. + */ public class WordCount { + /** + * Function to extract words. + */ public static class ExtractWordsFn extends DoFn { private final Aggregator emptyLines = createAggregator("emptyLines", new Sum.SumLongFn()); @@ -60,6 +66,9 @@ public void processElement(ProcessContext c) { } } + /** + * PTransform counting words. + */ public static class CountWords extends PTransform, PCollection>> { @Override diff --git a/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/package-info.java b/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/package-info.java new file mode 100644 index 0000000000000..b0ecb56466051 --- /dev/null +++ b/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/package-info.java @@ -0,0 +1,22 @@ +/* + * 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. + */ + +/** + * Flink Beam runner exemple. + */ +package org.apache.beam.runners.flink.examples; 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 9b5e31d08bbe0..4636e3f3149f2 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 @@ -96,7 +96,8 @@ public PCollection>> apply(PCollection> elem = c.element(); - str.append(elem.getKey() +" @ "+ window +" -> "); + str.append(elem.getKey() + " @ " + window + " -> "); for (CompletionCandidate cand: elem.getValue()) { str.append(cand.toString() + " "); } diff --git a/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/streaming/JoinExamples.java b/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/streaming/JoinExamples.java index bf5dfc453e19a..96638aab1db69 100644 --- a/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/streaming/JoinExamples.java +++ b/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/streaming/JoinExamples.java @@ -132,7 +132,8 @@ public static void main(String[] args) throws Exception { options.setExecutionRetryDelay(3000L); options.setRunner(FlinkRunner.class); - WindowFn windowFn = FixedWindows.of(Duration.standardSeconds(options.getWindowSize())); + WindowFn windowFn = FixedWindows.of( + Duration.standardSeconds(options.getWindowSize())); Pipeline p = Pipeline.create(options); diff --git a/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/streaming/KafkaIOExamples.java b/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/streaming/KafkaIOExamples.java index 27faefe6d7457..f0bf188e32285 100644 --- a/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/streaming/KafkaIOExamples.java +++ b/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/streaming/KafkaIOExamples.java @@ -56,15 +56,15 @@ public class KafkaIOExamples { private static final String KAFKA_AVRO_TOPIC = "output"; // Default kafka topic to read from private static final String KAFKA_BROKER = "localhost:9092"; // Default kafka broker to contact private static final String GROUP_ID = "myGroup"; // Default groupId - private static final String ZOOKEEPER = "localhost:2181"; // Default zookeeper to connect to for Kafka + private static final String ZOOKEEPER = "localhost:2181"; // Default zookeeper to connect (Kafka) /** - * Read/Write String data to Kafka + * Read/Write String data to Kafka. */ public static class KafkaString { /** - * Read String data from Kafka + * Read String data from Kafka. */ public static class ReadStringFromKafka { @@ -88,7 +88,7 @@ public static void main(String[] args) { } /** - * Write String data to Kafka + * Write String data to Kafka. */ public static class WriteStringToKafka { @@ -113,12 +113,12 @@ public static void main(String[] args) { } /** - * Read/Write Avro data to Kafka + * Read/Write Avro data to Kafka. */ public static class KafkaAvro { /** - * Read Avro data from Kafka + * Read Avro data from Kafka. */ public static class ReadAvroFromKafka { @@ -142,7 +142,7 @@ public static void main(String[] args) { } /** - * Write Avro data to Kafka + * Write Avro data to Kafka. */ public static class WriteAvroToKafka { @@ -169,7 +169,7 @@ public static void main(String[] args) { } /** - * Serialiation/Deserialiation schema for Avro types + * Serialiation/Deserialiation schema for Avro types. * @param */ static class AvroSerializationDeserializationSchema @@ -217,7 +217,7 @@ public TypeInformation getProducedType() { } /** - * Custom type for Avro serialization + * Custom type for Avro serialization. */ static class MyType implements Serializable { @@ -233,10 +233,10 @@ public MyType() {} @Override public String toString() { - return "MyType{" + - "word='" + word + '\'' + - ", count=" + count + - '}'; + return "MyType{" + + "word='" + word + '\'' + + ", count=" + count + + '}'; } } } @@ -244,7 +244,7 @@ public String toString() { // -------------- Utilities -------------- /** - * Custom options for the Pipeline + * Custom options for the Pipeline. */ public interface KafkaOptions extends FlinkPipelineOptions { @Description("The Kafka topic to read from") @@ -279,7 +279,7 @@ public interface KafkaOptions extends FlinkPipelineOptions { } /** - * Initializes some options for the Flink runner + * Initializes some options for the Flink runner. * @param args The command line args * @return the pipeline */ @@ -298,7 +298,7 @@ private static Pipeline initializePipeline(String[] args) { } /** - * Gets KafkaOptions from the Pipeline + * Gets KafkaOptions from the Pipeline. * @param p the pipeline * @return KafkaOptions */ @@ -322,7 +322,7 @@ private static Properties getKafkaProps(KafkaOptions options) { } /** - * Print contents to stdout + * Print contents to stdout. * @param type of the input */ private static class PrintFn extends DoFn { diff --git a/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/streaming/KafkaWindowedWordCountExample.java b/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/streaming/KafkaWindowedWordCountExample.java index 365fb7b486726..42c42f3241980 100644 --- a/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/streaming/KafkaWindowedWordCountExample.java +++ b/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/streaming/KafkaWindowedWordCountExample.java @@ -40,6 +40,9 @@ import org.apache.flink.streaming.util.serialization.SimpleStringSchema; import org.joda.time.Duration; +/** + * Wordcount example using Kafka topic. + */ public class KafkaWindowedWordCountExample { static final String KAFKA_TOPIC = "test"; // Default kafka topic to read from @@ -47,6 +50,9 @@ public class KafkaWindowedWordCountExample { static final String GROUP_ID = "myGroup"; // Default groupId static final String ZOOKEEPER = "localhost:2181"; // Default zookeeper to connect to for Kafka + /** + * Function to extract words. + */ public static class ExtractWordsFn extends DoFn { private final Aggregator emptyLines = createAggregator("emptyLines", new Sum.SumLongFn()); @@ -69,16 +75,24 @@ public void processElement(ProcessContext c) { } } + /** + * Function to format KV as String. + */ public static class FormatAsStringFn extends DoFn, String> { @ProcessElement public void processElement(ProcessContext c) { - String row = c.element().getKey() + " - " + c.element().getValue() + " @ " + c.timestamp().toString(); + String row = c.element().getKey() + " - " + c.element().getValue() + " @ " + + c.timestamp().toString(); System.out.println(row); c.output(row); } } - public interface KafkaStreamingWordCountOptions extends WindowedWordCount.StreamingWordCountOptions { + /** + * Pipeline options. + */ + public interface KafkaStreamingWordCountOptions + extends WindowedWordCount.StreamingWordCountOptions { @Description("The Kafka topic to read from") @Default.String(KAFKA_TOPIC) String getKafkaTopic(); @@ -107,7 +121,8 @@ public interface KafkaStreamingWordCountOptions extends WindowedWordCount.Stream public static void main(String[] args) { PipelineOptionsFactory.register(KafkaStreamingWordCountOptions.class); - KafkaStreamingWordCountOptions options = PipelineOptionsFactory.fromArgs(args).as(KafkaStreamingWordCountOptions.class); + KafkaStreamingWordCountOptions options = PipelineOptionsFactory.fromArgs(args) + .as(KafkaStreamingWordCountOptions.class); options.setJobName("KafkaExample - WindowSize: " + options.getWindowSize() + " seconds"); options.setStreaming(true); options.setCheckpointingInterval(1000L); @@ -115,7 +130,8 @@ public static void main(String[] args) { options.setExecutionRetryDelay(3000L); options.setRunner(FlinkRunner.class); - System.out.println(options.getKafkaTopic() +" "+ options.getZookeeper() +" "+ options.getBroker() +" "+ options.getGroup() ); + System.out.println(options.getKafkaTopic() + " " + options.getZookeeper() + " " + + options.getBroker() + " " + options.getGroup()); Pipeline pipeline = Pipeline.create(options); Properties p = new Properties(); @@ -132,7 +148,8 @@ public static void main(String[] args) { PCollection words = pipeline .apply("StreamingWordCount", Read.from(UnboundedFlinkSource.of(kafkaConsumer))) .apply(ParDo.of(new ExtractWordsFn())) - .apply(Window.into(FixedWindows.of(Duration.standardSeconds(options.getWindowSize()))) + .apply(Window.into(FixedWindows.of( + Duration.standardSeconds(options.getWindowSize()))) .triggering(AfterWatermark.pastEndOfWindow()).withAllowedLateness(Duration.ZERO) .discardingFiredPanes()); diff --git a/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/streaming/WindowedWordCount.java b/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/streaming/WindowedWordCount.java index f3361c5716973..0e250b868b725 100644 --- a/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/streaming/WindowedWordCount.java +++ b/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/streaming/WindowedWordCount.java @@ -60,7 +60,8 @@ public class WindowedWordCount { static class FormatAsStringFn extends DoFn, String> { @ProcessElement public void processElement(ProcessContext c) { - String row = c.element().getKey() + " - " + c.element().getValue() + " @ " + c.timestamp().toString(); + String row = c.element().getKey() + " - " + c.element().getValue() + " @ " + + c.timestamp().toString(); c.output(row); } } @@ -87,7 +88,11 @@ public void processElement(ProcessContext c) { } } - public interface StreamingWordCountOptions extends org.apache.beam.runners.flink.examples.WordCount.Options { + /** + * Pipeline options. + */ + public interface StreamingWordCountOptions + extends org.apache.beam.runners.flink.examples.WordCount.Options { @Description("Sliding window duration, in seconds") @Default.Long(WINDOW_SIZE) Long getWindowSize(); @@ -102,7 +107,8 @@ public interface StreamingWordCountOptions extends org.apache.beam.runners.flink } public static void main(String[] args) throws IOException { - StreamingWordCountOptions options = PipelineOptionsFactory.fromArgs(args).withValidation().as(StreamingWordCountOptions.class); + StreamingWordCountOptions options = PipelineOptionsFactory.fromArgs(args).withValidation() + .as(StreamingWordCountOptions.class); options.setStreaming(true); options.setWindowSize(10L); options.setSlide(5L); @@ -111,8 +117,8 @@ public static void main(String[] args) throws IOException { options.setExecutionRetryDelay(3000L); options.setRunner(FlinkRunner.class); - LOG.info("Windpwed WordCount with Sliding Windows of " + options.getWindowSize() + - " sec. and a slide of " + options.getSlide()); + LOG.info("Windpwed WordCount with Sliding Windows of " + options.getWindowSize() + + " sec. and a slide of " + options.getSlide()); Pipeline pipeline = Pipeline.create(options); @@ -120,7 +126,8 @@ public static void main(String[] args) throws IOException { .apply("StreamingWordCount", Read.from(new UnboundedSocketSource<>("localhost", 9999, '\n', 3))) .apply(ParDo.of(new ExtractWordsFn())) - .apply(Window.into(SlidingWindows.of(Duration.standardSeconds(options.getWindowSize())) + .apply(Window.into(SlidingWindows.of( + Duration.standardSeconds(options.getWindowSize())) .every(Duration.standardSeconds(options.getSlide()))) .triggering(AfterWatermark.pastEndOfWindow()).withAllowedLateness(Duration.ZERO) .discardingFiredPanes()); diff --git a/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/streaming/package-info.java b/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/streaming/package-info.java new file mode 100644 index 0000000000000..58f41b6a8bbde --- /dev/null +++ b/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/streaming/package-info.java @@ -0,0 +1,22 @@ +/* + * 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. + */ + +/** + * Flink Beam runner exemple. + */ +package org.apache.beam.runners.flink.examples.streaming; From b23559578f2a7acda477b4ebccccb6a6f7c9b03e Mon Sep 17 00:00:00 2001 From: Stas Levin Date: Mon, 5 Sep 2016 20:26:12 +0300 Subject: [PATCH 111/346] Add header/footer support to TextIO.Write --- .../java/org/apache/beam/sdk/io/TextIO.java | 114 +++++++++++++++--- .../org/apache/beam/sdk/io/TextIOTest.java | 60 ++++++++- 2 files changed, 152 insertions(+), 22 deletions(-) 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 242470b587926..c754a0ba0c4f2 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 @@ -448,7 +448,15 @@ public static Bound withoutValidation() { return new Bound<>(DEFAULT_TEXT_CODER).withoutValidation(); } - // TODO: appendingNewlines, header, footer, etc. + public static Bound withHeader(String header) { + return new Bound<>(DEFAULT_TEXT_CODER).withHeader(header); + } + + public static Bound withFooter(String footer) { + return new Bound<>(DEFAULT_TEXT_CODER).withFooter(footer); + } + + // TODO: appendingNewlines, etc. /** * A PTransform that writes a bounded PCollection to a text file (or @@ -465,6 +473,12 @@ public static class Bound extends PTransform, PDone> { /** The suffix of each file written, combined with prefix and shardTemplate. */ private final String filenameSuffix; + /** An optional header to add to each file. */ + private final String header; + + /** An optional footer to add to each file. */ + private final String footer; + /** The Coder to use to decode each line. */ private final Coder coder; @@ -478,12 +492,15 @@ public static class Bound extends PTransform, PDone> { private final boolean validate; Bound(Coder coder) { - this(null, null, "", coder, 0, DEFAULT_SHARD_TEMPLATE, true); + this(null, null, "", null, null, coder, 0, DEFAULT_SHARD_TEMPLATE, true); } - private Bound(String name, String filenamePrefix, String filenameSuffix, Coder coder, - int numShards, String shardTemplate, boolean validate) { + private Bound(String name, String filenamePrefix, String filenameSuffix, String header, + String footer, Coder coder, int numShards, String shardTemplate, + boolean validate) { super(name); + this.header = header; + this.footer = footer; this.coder = coder; this.filenamePrefix = filenamePrefix; this.filenameSuffix = filenameSuffix; @@ -502,7 +519,7 @@ private Bound(String name, String filenamePrefix, String filenameSuffix, Coder to(String filenamePrefix) { validateOutputComponent(filenamePrefix); - return new Bound<>(name, filenamePrefix, filenameSuffix, coder, numShards, + return new Bound<>(name, filenamePrefix, filenameSuffix, header, footer, coder, numShards, shardTemplate, validate); } @@ -516,7 +533,7 @@ public Bound to(String filenamePrefix) { */ public Bound withSuffix(String nameExtension) { validateOutputComponent(nameExtension); - return new Bound<>(name, filenamePrefix, nameExtension, coder, numShards, + return new Bound<>(name, filenamePrefix, nameExtension, header, footer, coder, numShards, shardTemplate, validate); } @@ -536,7 +553,7 @@ public Bound withSuffix(String nameExtension) { */ public Bound withNumShards(int numShards) { checkArgument(numShards >= 0); - return new Bound<>(name, filenamePrefix, filenameSuffix, coder, numShards, + return new Bound<>(name, filenamePrefix, filenameSuffix, header, footer, coder, numShards, shardTemplate, validate); } @@ -549,7 +566,7 @@ public Bound withNumShards(int numShards) { * @see ShardNameTemplate */ public Bound withShardNameTemplate(String shardTemplate) { - return new Bound<>(name, filenamePrefix, filenameSuffix, coder, numShards, + return new Bound<>(name, filenamePrefix, filenameSuffix, header, footer, coder, numShards, shardTemplate, validate); } @@ -567,7 +584,8 @@ public Bound withShardNameTemplate(String shardTemplate) { *

              Does not modify this object. */ public Bound withoutSharding() { - return new Bound<>(name, filenamePrefix, filenameSuffix, coder, 1, "", validate); + return new Bound<>(name, filenamePrefix, filenameSuffix, header, footer, coder, 1, "", + validate); } /** @@ -579,7 +597,7 @@ public Bound withoutSharding() { * @param the type of the elements of the input {@link PCollection} */ public Bound withCoder(Coder coder) { - return new Bound<>(name, filenamePrefix, filenameSuffix, coder, numShards, + return new Bound<>(name, filenamePrefix, filenameSuffix, header, footer, coder, numShards, shardTemplate, validate); } @@ -594,7 +612,17 @@ public Bound withCoder(Coder coder) { *

              Does not modify this object. */ public Bound withoutValidation() { - return new Bound<>(name, filenamePrefix, filenameSuffix, coder, numShards, + return new Bound<>(name, filenamePrefix, filenameSuffix, header, footer, coder, numShards, + shardTemplate, false); + } + + public Bound withHeader(String header) { + return new Bound<>(name, filenamePrefix, filenameSuffix, header, footer, coder, numShards, + shardTemplate, false); + } + + public Bound withFooter(String footer) { + return new Bound<>(name, filenamePrefix, filenameSuffix, header, footer, coder, numShards, shardTemplate, false); } @@ -607,7 +635,8 @@ public PDone apply(PCollection input) { org.apache.beam.sdk.io.Write.Bound write = org.apache.beam.sdk.io.Write.to( - new TextSink<>(filenamePrefix, filenameSuffix, shardTemplate, coder)); + new TextSink<>(filenamePrefix, filenameSuffix, header, footer, shardTemplate, + coder)); if (getNumShards() > 0) { write = write.withNumShards(getNumShards()); } @@ -629,7 +658,11 @@ public void populateDisplayData(DisplayData.Builder builder) { .addIfNotDefault(DisplayData.item("validation", validate) .withLabel("Validation Enabled"), true) .addIfNotDefault(DisplayData.item("numShards", numShards) - .withLabel("Maximum Output Shards"), 0); + .withLabel("Maximum Output Shards"), 0) + .addIfNotNull(DisplayData.item("fileHeader", header) + .withLabel("Output file header")) + .addIfNotNull(DisplayData.item("fileFooter", footer) + .withLabel("Output file footer")); } /** @@ -664,6 +697,14 @@ public Coder getCoder() { return coder; } + public String getHeader() { + return header; + } + + public String getFooter() { + return footer; + } + public boolean needsValidation() { return validate; } @@ -946,17 +987,22 @@ private boolean tryToEnsureNumberOfBytesInBuffer(int minCapacity) throws IOExcep @VisibleForTesting static class TextSink extends FileBasedSink { private final Coder coder; + private final String header; + private final String footer; @VisibleForTesting TextSink( - String baseOutputFilename, String extension, String fileNameTemplate, Coder coder) { + String baseOutputFilename, String extension, String header, String footer, + String fileNameTemplate, Coder coder) { super(baseOutputFilename, extension, fileNameTemplate); this.coder = coder; + this.header = header; + this.footer = footer; } @Override public FileBasedSink.FileBasedWriteOperation createWriteOperation(PipelineOptions options) { - return new TextWriteOperation<>(this, coder); + return new TextWriteOperation<>(this, coder, header, footer); } /** @@ -965,15 +1011,19 @@ public FileBasedSink.FileBasedWriteOperation createWriteOperation(PipelineOpt */ private static class TextWriteOperation extends FileBasedWriteOperation { private final Coder coder; + private final String header; + private final String footer; - private TextWriteOperation(TextSink sink, Coder coder) { + private TextWriteOperation(TextSink sink, Coder coder, String header, String footer) { super(sink); this.coder = coder; + this.header = header; + this.footer = footer; } @Override public FileBasedWriter createWriter(PipelineOptions options) throws Exception { - return new TextWriter<>(this, coder); + return new TextWriter<>(this, coder, header, footer); } } @@ -984,19 +1034,49 @@ public FileBasedWriter createWriter(PipelineOptions options) throws Exception private static class TextWriter extends FileBasedWriter { private static final byte[] NEWLINE = "\n".getBytes(StandardCharsets.UTF_8); private final Coder coder; + private final String header; + private final String footer; private OutputStream out; public TextWriter(FileBasedWriteOperation writeOperation, Coder coder) { + this(writeOperation, coder, null, null); + } + + public TextWriter(FileBasedWriteOperation writeOperation, Coder coder, String header) { + this(writeOperation, coder, header, null); + } + + public TextWriter(FileBasedWriteOperation writeOperation, Coder coder, String header, + String footer) { super(writeOperation); + this.header = header; + this.footer = footer; this.mimeType = MimeTypes.TEXT; this.coder = coder; } + private void writeLine(String line) throws IOException { + if (line != null) { + out.write(line.getBytes(StandardCharsets.UTF_8)); + out.write(NEWLINE); + } + } + @Override protected void prepareWrite(WritableByteChannel channel) throws Exception { out = Channels.newOutputStream(channel); } + @Override + protected void writeHeader() throws Exception { + writeLine(header); + } + + @Override + protected void writeFooter() throws Exception { + writeLine(footer); + } + @Override public void write(T value) throws Exception { coder.encode(value, out, Context.OUTER); 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 8f94766063022..2ab2683556971 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 @@ -34,6 +34,8 @@ import static org.junit.Assert.assertTrue; import com.google.common.collect.ImmutableList; +import com.google.common.collect.Lists; + import java.io.BufferedReader; import java.io.File; import java.io.FileOutputStream; @@ -48,6 +50,7 @@ import java.nio.file.StandardOpenOption; import java.util.ArrayList; import java.util.Arrays; +import java.util.LinkedList; import java.util.List; import java.util.Set; import java.util.zip.GZIPOutputStream; @@ -101,6 +104,9 @@ @SuppressWarnings("unchecked") public class TextIOTest { + private static final String MY_HEADER = "myHeader"; + private static final String MY_FOOTER = "myFooter"; + @Rule public TemporaryFolder tmpFolder = new TemporaryFolder(); @Rule @@ -216,10 +222,19 @@ public void testPrimitiveReadDisplayData() { } void runTestWrite(T[] elems, Coder coder) throws Exception { - runTestWrite(elems, coder, 1); + runTestWrite(elems, null, null, coder, 1); } void runTestWrite(T[] elems, Coder coder, int numShards) throws Exception { + runTestWrite(elems, null, null, coder, numShards); + } + + void runTestWrite(T[] elems, Coder coder, String header, String footer) throws Exception { + runTestWrite(elems, header, footer, coder, 1); + } + + void runTestWrite(T[] elems, String header, String footer, Coder coder, int numShards) + throws Exception { String outputName = "file.txt"; String baseFilename = tmpFolder.newFile(outputName).getPath(); @@ -235,6 +250,8 @@ void runTestWrite(T[] elems, Coder coder, int numShards) throws Exception } else { write = TextIO.Write.to(baseFilename).withCoder(coder); } + write = write.withHeader(header).withFooter(footer); + if (numShards == 1) { write = write.withoutSharding(); } else if (numShards > 0) { @@ -244,11 +261,14 @@ void runTestWrite(T[] elems, Coder coder, int numShards) throws Exception p.run(); - assertOutputFiles(elems, coder, numShards, tmpFolder, outputName, write.getShardNameTemplate()); + assertOutputFiles(elems, header, footer, coder, numShards, tmpFolder, outputName, + write.getShardNameTemplate()); } public static void assertOutputFiles( T[] elems, + String header, + String footer, Coder coder, int numShards, TemporaryFolder rootLocation, @@ -284,15 +304,23 @@ public static void assertOutputFiles( } } - String[] expected = new String[elems.length]; + LinkedList expected = Lists.newLinkedList(); + for (int i = 0; i < elems.length; i++) { T elem = elems[i]; byte[] encodedElem = CoderUtils.encodeToByteArray(coder, elem); String line = new String(encodedElem); - expected[i] = line; + expected.add(line); + } + + if (header != null) { + expected.addFirst(header); + } + if (footer != null) { + expected.addLast(footer); } - assertThat(actual, containsInAnyOrder(expected)); + assertThat(actual, containsInAnyOrder(expected.toArray())); } @Test @@ -331,6 +359,24 @@ public void testShardedWrite() throws Exception { runTestWrite(LINES_ARRAY, StringUtf8Coder.of(), 5); } + @Test + @Category(NeedsRunner.class) + public void testWriteWithHeader() throws Exception { + runTestWrite(LINES_ARRAY, StringUtf8Coder.of(), MY_HEADER, null); + } + + @Test + @Category(NeedsRunner.class) + public void testWriteWithFooter() throws Exception { + runTestWrite(LINES_ARRAY, StringUtf8Coder.of(), null, MY_FOOTER); + } + + @Test + @Category(NeedsRunner.class) + public void testWriteWithHeaderAndFooter() throws Exception { + runTestWrite(LINES_ARRAY, StringUtf8Coder.of(), MY_HEADER, MY_FOOTER); + } + @Test public void testWriteDisplayData() { TextIO.Write.Bound write = TextIO.Write @@ -338,12 +384,16 @@ public void testWriteDisplayData() { .withSuffix("bar") .withShardNameTemplate("-SS-of-NN-") .withNumShards(100) + .withFooter("myFooter") + .withHeader("myHeader") .withoutValidation(); DisplayData displayData = DisplayData.from(write); assertThat(displayData, hasDisplayItem("filePrefix", "foo")); assertThat(displayData, hasDisplayItem("fileSuffix", "bar")); + assertThat(displayData, hasDisplayItem("fileHeader", "myHeader")); + assertThat(displayData, hasDisplayItem("fileFooter", "myFooter")); assertThat(displayData, hasDisplayItem("shardNameTemplate", "-SS-of-NN-")); assertThat(displayData, hasDisplayItem("numShards", 100)); assertThat(displayData, hasDisplayItem("validation", false)); From 1b420dbdc08c3fde53dd5ce0a56260576cee3076 Mon Sep 17 00:00:00 2001 From: Stas Levin Date: Tue, 6 Sep 2016 23:22:11 +0300 Subject: [PATCH 112/346] Revised according to comments following a code review. --- .../java/org/apache/beam/sdk/io/TextIO.java | 72 ++++++++++-------- .../org/apache/beam/sdk/io/TextIOTest.java | 73 ++++++++++++++----- 2 files changed, 98 insertions(+), 47 deletions(-) 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 c754a0ba0c4f2..eefa867d2ae5c 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 @@ -22,6 +22,7 @@ import com.google.common.annotations.VisibleForTesting; import com.google.protobuf.ByteString; + import java.io.IOException; import java.io.OutputStream; import java.nio.ByteBuffer; @@ -35,12 +36,14 @@ import javax.annotation.Nullable; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.Coder.Context; +import org.apache.beam.sdk.coders.CoderException; import org.apache.beam.sdk.coders.StringUtf8Coder; import org.apache.beam.sdk.coders.VoidCoder; import org.apache.beam.sdk.io.Read.Bounded; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.display.DisplayData; +import org.apache.beam.sdk.util.CoderUtils; import org.apache.beam.sdk.util.IOChannelUtils; import org.apache.beam.sdk.util.MimeTypes; import org.apache.beam.sdk.values.PBegin; @@ -474,10 +477,10 @@ public static class Bound extends PTransform, PDone> { private final String filenameSuffix; /** An optional header to add to each file. */ - private final String header; + private final T header; /** An optional footer to add to each file. */ - private final String footer; + private final T footer; /** The Coder to use to decode each line. */ private final Coder coder; @@ -495,8 +498,8 @@ public static class Bound extends PTransform, PDone> { this(null, null, "", null, null, coder, 0, DEFAULT_SHARD_TEMPLATE, true); } - private Bound(String name, String filenamePrefix, String filenameSuffix, String header, - String footer, Coder coder, int numShards, String shardTemplate, + private Bound(String name, String filenamePrefix, String filenameSuffix, T header, + T footer, Coder coder, int numShards, String shardTemplate, boolean validate) { super(name); this.header = header; @@ -509,6 +512,14 @@ private Bound(String name, String filenamePrefix, String filenameSuffix, String this.validate = validate; } + private String asString(T obj, Coder coder) { + try { + return obj == null ? "" : new String(CoderUtils.encodeToByteArray(coder, obj)); + } catch (CoderException e) { + throw new RuntimeException(e); + } + } + /** * Returns a transform for writing to text files that's like this one but * that writes to the file(s) with the given filename prefix. @@ -594,9 +605,8 @@ public Bound withoutSharding() { * the elements of the input {@link PCollection PCollection} into an * output text line. Does not modify this object. * - * @param the type of the elements of the input {@link PCollection} */ - public Bound withCoder(Coder coder) { + public Bound withCoder(Coder coder) { return new Bound<>(name, filenamePrefix, filenameSuffix, header, footer, coder, numShards, shardTemplate, validate); } @@ -616,12 +626,12 @@ public Bound withoutValidation() { shardTemplate, false); } - public Bound withHeader(String header) { + public Bound withHeader(T header) { return new Bound<>(name, filenamePrefix, filenameSuffix, header, footer, coder, numShards, shardTemplate, false); } - public Bound withFooter(String footer) { + public Bound withFooter(T footer) { return new Bound<>(name, filenamePrefix, filenameSuffix, header, footer, coder, numShards, shardTemplate, false); } @@ -659,10 +669,10 @@ public void populateDisplayData(DisplayData.Builder builder) { .withLabel("Validation Enabled"), true) .addIfNotDefault(DisplayData.item("numShards", numShards) .withLabel("Maximum Output Shards"), 0) - .addIfNotNull(DisplayData.item("fileHeader", header) - .withLabel("Output file header")) - .addIfNotNull(DisplayData.item("fileFooter", footer) - .withLabel("Output file footer")); + .addIfNotNull(DisplayData.item("fileHeader", asString(header, coder)) + .withLabel("File Header")) + .addIfNotNull(DisplayData.item("fileFooter", asString(footer, coder)) + .withLabel("File Footer")); } /** @@ -697,11 +707,11 @@ public Coder getCoder() { return coder; } - public String getHeader() { + public T getHeader() { return header; } - public String getFooter() { + public T getFooter() { return footer; } @@ -987,17 +997,21 @@ private boolean tryToEnsureNumberOfBytesInBuffer(int minCapacity) throws IOExcep @VisibleForTesting static class TextSink extends FileBasedSink { private final Coder coder; - private final String header; - private final String footer; + private final byte[] header; + private final byte[] footer; @VisibleForTesting TextSink( - String baseOutputFilename, String extension, String header, String footer, + String baseOutputFilename, String extension, T header, T footer, String fileNameTemplate, Coder coder) { super(baseOutputFilename, extension, fileNameTemplate); this.coder = coder; - this.header = header; - this.footer = footer; + try { + this.header = header == null ? null : CoderUtils.encodeToByteArray(coder, header); + this.footer = footer == null ? null : CoderUtils.encodeToByteArray(coder, footer); + } catch (CoderException e) { + throw new RuntimeException(e); + } } @Override @@ -1011,10 +1025,10 @@ public FileBasedSink.FileBasedWriteOperation createWriteOperation(PipelineOpt */ private static class TextWriteOperation extends FileBasedWriteOperation { private final Coder coder; - private final String header; - private final String footer; + private final byte[] header; + private final byte[] footer; - private TextWriteOperation(TextSink sink, Coder coder, String header, String footer) { + private TextWriteOperation(TextSink sink, Coder coder, byte[] header, byte[] footer) { super(sink); this.coder = coder; this.header = header; @@ -1034,20 +1048,20 @@ public FileBasedWriter createWriter(PipelineOptions options) throws Exception private static class TextWriter extends FileBasedWriter { private static final byte[] NEWLINE = "\n".getBytes(StandardCharsets.UTF_8); private final Coder coder; - private final String header; - private final String footer; + private final byte[] header; + private final byte[] footer; private OutputStream out; public TextWriter(FileBasedWriteOperation writeOperation, Coder coder) { this(writeOperation, coder, null, null); } - public TextWriter(FileBasedWriteOperation writeOperation, Coder coder, String header) { + public TextWriter(FileBasedWriteOperation writeOperation, Coder coder, byte[] header) { this(writeOperation, coder, header, null); } - public TextWriter(FileBasedWriteOperation writeOperation, Coder coder, String header, - String footer) { + public TextWriter(FileBasedWriteOperation writeOperation, Coder coder, byte[] header, + byte[] footer) { super(writeOperation); this.header = header; this.footer = footer; @@ -1055,9 +1069,9 @@ public TextWriter(FileBasedWriteOperation writeOperation, Coder coder, Str this.coder = coder; } - private void writeLine(String line) throws IOException { + private void writeLine(byte[] line) throws IOException { if (line != null) { - out.write(line.getBytes(StandardCharsets.UTF_8)); + out.write(line); out.write(NEWLINE); } } 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 2ab2683556971..7028761bb5e5d 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 @@ -33,7 +33,11 @@ import static org.junit.Assert.assertThat; import static org.junit.Assert.assertTrue; +import com.google.common.base.Function; +import com.google.common.base.Predicate; +import com.google.common.collect.FluentIterable; import com.google.common.collect.ImmutableList; +import com.google.common.collect.Iterables; import com.google.common.collect.Lists; import java.io.BufferedReader; @@ -229,11 +233,11 @@ void runTestWrite(T[] elems, Coder coder, int numShards) throws Exception runTestWrite(elems, null, null, coder, numShards); } - void runTestWrite(T[] elems, Coder coder, String header, String footer) throws Exception { + void runTestWrite(T[] elems, Coder coder, T header, T footer) throws Exception { runTestWrite(elems, header, footer, coder, 1); } - void runTestWrite(T[] elems, String header, String footer, Coder coder, int numShards) + void runTestWrite(T[] elems, T header, T footer, Coder coder, int numShards) throws Exception { String outputName = "file.txt"; String baseFilename = tmpFolder.newFile(outputName).getPath(); @@ -248,7 +252,7 @@ void runTestWrite(T[] elems, String header, String footer, Coder coder, i // T==String write = (TextIO.Write.Bound) writeStrings; } else { - write = TextIO.Write.to(baseFilename).withCoder(coder); + write = TextIO.Write.withCoder(coder).to(baseFilename); } write = write.withHeader(header).withFooter(footer); @@ -267,9 +271,9 @@ void runTestWrite(T[] elems, String header, String footer, Coder coder, i public static void assertOutputFiles( T[] elems, - String header, - String footer, - Coder coder, + final T header, + final T footer, + final Coder coder, int numShards, TemporaryFolder rootLocation, String outputName, @@ -291,36 +295,69 @@ public static void assertOutputFiles( } } - List actual = new ArrayList<>(); + List> actual = new ArrayList<>(); + for (File tmpFile : expectedFiles) { try (BufferedReader reader = new BufferedReader(new FileReader(tmpFile))) { + List currentFile = Lists.newArrayList(); for (;;) { String line = reader.readLine(); if (line == null) { break; } - actual.add(line); + currentFile.add(line); } + actual.add(currentFile); } } - LinkedList expected = Lists.newLinkedList(); + LinkedList expectedElements = Lists.newLinkedList(); for (int i = 0; i < elems.length; i++) { T elem = elems[i]; byte[] encodedElem = CoderUtils.encodeToByteArray(coder, elem); String line = new String(encodedElem); - expected.add(line); + expectedElements.add(line); } - if (header != null) { - expected.addFirst(header); - } - if (footer != null) { - expected.addLast(footer); - } - - assertThat(actual, containsInAnyOrder(expected.toArray())); + final String headerString = + header == null ? null : new String(CoderUtils.encodeToByteArray(coder, header)); + + final String footerString = + footer == null ? null : new String(CoderUtils.encodeToByteArray(coder, footer)); + + ArrayList actualElements = + Lists.newArrayList( + Iterables.concat( + FluentIterable + .from(actual) + .transform(new Function, List>() { + @Nullable + @Override + public List apply(List lines) { + ArrayList newLines = Lists.newArrayList(lines); + if (headerString != null) { + newLines.remove(0); + } + if (footerString != null) { + int last = newLines.size() - 1; + newLines.remove(last); + } + return newLines; + } + }) + .toList())); + + assertThat(actualElements, containsInAnyOrder(expectedElements.toArray())); + + assertTrue(Iterables.all(actual, new Predicate>() { + @Override + public boolean apply(@Nullable List fileLines) { + int last = fileLines.size() - 1; + return (headerString == null || fileLines.get(0).equals(headerString)) + && (footerString == null || fileLines.get(last).equals(footerString)); + } + })); } @Test From 092a1870fc84067ae0e19a736a37160a9a55c653 Mon Sep 17 00:00:00 2001 From: Stas Levin Date: Wed, 7 Sep 2016 09:57:17 +0300 Subject: [PATCH 113/346] Reverted header and footer to be of type String. --- .../java/org/apache/beam/sdk/io/TextIO.java | 68 ++++++++----------- .../org/apache/beam/sdk/io/TextIOTest.java | 68 ++++++++++--------- 2 files changed, 63 insertions(+), 73 deletions(-) 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 eefa867d2ae5c..0895123aba372 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 @@ -22,7 +22,6 @@ import com.google.common.annotations.VisibleForTesting; import com.google.protobuf.ByteString; - import java.io.IOException; import java.io.OutputStream; import java.nio.ByteBuffer; @@ -36,14 +35,12 @@ import javax.annotation.Nullable; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.Coder.Context; -import org.apache.beam.sdk.coders.CoderException; import org.apache.beam.sdk.coders.StringUtf8Coder; import org.apache.beam.sdk.coders.VoidCoder; import org.apache.beam.sdk.io.Read.Bounded; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.display.DisplayData; -import org.apache.beam.sdk.util.CoderUtils; import org.apache.beam.sdk.util.IOChannelUtils; import org.apache.beam.sdk.util.MimeTypes; import org.apache.beam.sdk.values.PBegin; @@ -477,10 +474,10 @@ public static class Bound extends PTransform, PDone> { private final String filenameSuffix; /** An optional header to add to each file. */ - private final T header; + private final String header; /** An optional footer to add to each file. */ - private final T footer; + private final String footer; /** The Coder to use to decode each line. */ private final Coder coder; @@ -498,8 +495,8 @@ public static class Bound extends PTransform, PDone> { this(null, null, "", null, null, coder, 0, DEFAULT_SHARD_TEMPLATE, true); } - private Bound(String name, String filenamePrefix, String filenameSuffix, T header, - T footer, Coder coder, int numShards, String shardTemplate, + private Bound(String name, String filenamePrefix, String filenameSuffix, String header, + String footer, Coder coder, int numShards, String shardTemplate, boolean validate) { super(name); this.header = header; @@ -512,14 +509,6 @@ private Bound(String name, String filenamePrefix, String filenameSuffix, T heade this.validate = validate; } - private String asString(T obj, Coder coder) { - try { - return obj == null ? "" : new String(CoderUtils.encodeToByteArray(coder, obj)); - } catch (CoderException e) { - throw new RuntimeException(e); - } - } - /** * Returns a transform for writing to text files that's like this one but * that writes to the file(s) with the given filename prefix. @@ -605,8 +594,9 @@ public Bound withoutSharding() { * the elements of the input {@link PCollection PCollection} into an * output text line. Does not modify this object. * + * @param the type of the elements of the input {@link PCollection} */ - public Bound withCoder(Coder coder) { + public Bound withCoder(Coder coder) { return new Bound<>(name, filenamePrefix, filenameSuffix, header, footer, coder, numShards, shardTemplate, validate); } @@ -626,12 +616,12 @@ public Bound withoutValidation() { shardTemplate, false); } - public Bound withHeader(T header) { + public Bound withHeader(String header) { return new Bound<>(name, filenamePrefix, filenameSuffix, header, footer, coder, numShards, shardTemplate, false); } - public Bound withFooter(T footer) { + public Bound withFooter(String footer) { return new Bound<>(name, filenamePrefix, filenameSuffix, header, footer, coder, numShards, shardTemplate, false); } @@ -669,9 +659,9 @@ public void populateDisplayData(DisplayData.Builder builder) { .withLabel("Validation Enabled"), true) .addIfNotDefault(DisplayData.item("numShards", numShards) .withLabel("Maximum Output Shards"), 0) - .addIfNotNull(DisplayData.item("fileHeader", asString(header, coder)) + .addIfNotNull(DisplayData.item("fileHeader", header) .withLabel("File Header")) - .addIfNotNull(DisplayData.item("fileFooter", asString(footer, coder)) + .addIfNotNull(DisplayData.item("fileFooter", footer) .withLabel("File Footer")); } @@ -707,11 +697,11 @@ public Coder getCoder() { return coder; } - public T getHeader() { + public String getHeader() { return header; } - public T getFooter() { + public String getFooter() { return footer; } @@ -997,21 +987,17 @@ private boolean tryToEnsureNumberOfBytesInBuffer(int minCapacity) throws IOExcep @VisibleForTesting static class TextSink extends FileBasedSink { private final Coder coder; - private final byte[] header; - private final byte[] footer; + private final String header; + private final String footer; @VisibleForTesting TextSink( - String baseOutputFilename, String extension, T header, T footer, + String baseOutputFilename, String extension, String header, String footer, String fileNameTemplate, Coder coder) { super(baseOutputFilename, extension, fileNameTemplate); this.coder = coder; - try { - this.header = header == null ? null : CoderUtils.encodeToByteArray(coder, header); - this.footer = footer == null ? null : CoderUtils.encodeToByteArray(coder, footer); - } catch (CoderException e) { - throw new RuntimeException(e); - } + this.header = header; + this.footer = footer; } @Override @@ -1025,10 +1011,10 @@ public FileBasedSink.FileBasedWriteOperation createWriteOperation(PipelineOpt */ private static class TextWriteOperation extends FileBasedWriteOperation { private final Coder coder; - private final byte[] header; - private final byte[] footer; + private final String header; + private final String footer; - private TextWriteOperation(TextSink sink, Coder coder, byte[] header, byte[] footer) { + private TextWriteOperation(TextSink sink, Coder coder, String header, String footer) { super(sink); this.coder = coder; this.header = header; @@ -1048,20 +1034,20 @@ public FileBasedWriter createWriter(PipelineOptions options) throws Exception private static class TextWriter extends FileBasedWriter { private static final byte[] NEWLINE = "\n".getBytes(StandardCharsets.UTF_8); private final Coder coder; - private final byte[] header; - private final byte[] footer; + private final String header; + private final String footer; private OutputStream out; public TextWriter(FileBasedWriteOperation writeOperation, Coder coder) { this(writeOperation, coder, null, null); } - public TextWriter(FileBasedWriteOperation writeOperation, Coder coder, byte[] header) { + public TextWriter(FileBasedWriteOperation writeOperation, Coder coder, String header) { this(writeOperation, coder, header, null); } - public TextWriter(FileBasedWriteOperation writeOperation, Coder coder, byte[] header, - byte[] footer) { + public TextWriter(FileBasedWriteOperation writeOperation, Coder coder, String header, + String footer) { super(writeOperation); this.header = header; this.footer = footer; @@ -1069,9 +1055,9 @@ public TextWriter(FileBasedWriteOperation writeOperation, Coder coder, byt this.coder = coder; } - private void writeLine(byte[] line) throws IOException { + private void writeLine(String line) throws IOException { if (line != null) { - out.write(line); + out.write(line.getBytes(StandardCharsets.UTF_8)); out.write(NEWLINE); } } 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 7028761bb5e5d..c60b735b0129a 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 @@ -233,11 +233,11 @@ void runTestWrite(T[] elems, Coder coder, int numShards) throws Exception runTestWrite(elems, null, null, coder, numShards); } - void runTestWrite(T[] elems, Coder coder, T header, T footer) throws Exception { + void runTestWrite(T[] elems, Coder coder, String header, String footer) throws Exception { runTestWrite(elems, header, footer, coder, 1); } - void runTestWrite(T[] elems, T header, T footer, Coder coder, int numShards) + void runTestWrite(T[] elems, String header, String footer, Coder coder, int numShards) throws Exception { String outputName = "file.txt"; String baseFilename = tmpFolder.newFile(outputName).getPath(); @@ -252,7 +252,7 @@ void runTestWrite(T[] elems, T header, T footer, Coder coder, int numShar // T==String write = (TextIO.Write.Bound) writeStrings; } else { - write = TextIO.Write.withCoder(coder).to(baseFilename); + write = TextIO.Write.to(baseFilename).withCoder(coder); } write = write.withHeader(header).withFooter(footer); @@ -271,9 +271,9 @@ void runTestWrite(T[] elems, T header, T footer, Coder coder, int numShar public static void assertOutputFiles( T[] elems, - final T header, - final T footer, - final Coder coder, + final String header, + final String footer, + Coder coder, int numShards, TemporaryFolder rootLocation, String outputName, @@ -320,44 +320,48 @@ public static void assertOutputFiles( expectedElements.add(line); } - final String headerString = - header == null ? null : new String(CoderUtils.encodeToByteArray(coder, header)); - - final String footerString = - footer == null ? null : new String(CoderUtils.encodeToByteArray(coder, footer)); - ArrayList actualElements = Lists.newArrayList( Iterables.concat( FluentIterable .from(actual) - .transform(new Function, List>() { - @Nullable - @Override - public List apply(List lines) { - ArrayList newLines = Lists.newArrayList(lines); - if (headerString != null) { - newLines.remove(0); - } - if (footerString != null) { - int last = newLines.size() - 1; - newLines.remove(last); - } - return newLines; - } - }) + .transform(removeHeaderAndFooter(header, footer)) .toList())); assertThat(actualElements, containsInAnyOrder(expectedElements.toArray())); - assertTrue(Iterables.all(actual, new Predicate>() { + assertTrue(Iterables.all(actual, haveProperHeaderAndFooter(header, footer))); + } + + private static Function, List> removeHeaderAndFooter(final String header, + final String footer) { + return new Function, List>() { + @Nullable + @Override + public List apply(List lines) { + ArrayList newLines = Lists.newArrayList(lines); + if (header != null) { + newLines.remove(0); + } + if (footer != null) { + int last = newLines.size() - 1; + newLines.remove(last); + } + return newLines; + } + }; + } + + private static Predicate> haveProperHeaderAndFooter(final String header, + final String footer) { + return new Predicate>() { @Override - public boolean apply(@Nullable List fileLines) { + public boolean apply(List fileLines) { int last = fileLines.size() - 1; - return (headerString == null || fileLines.get(0).equals(headerString)) - && (footerString == null || fileLines.get(last).equals(footerString)); + return (header == null || fileLines.get(0).equals(header)) + && (footer == null || fileLines.get(last).equals(footer)); } - })); + }; } @Test From 5084580f0f11f97df00c1c972056acaae713c41f Mon Sep 17 00:00:00 2001 From: Stas Levin Date: Wed, 7 Sep 2016 18:33:48 +0300 Subject: [PATCH 114/346] Added javadoc to TextIO#withHeader and TextIO#withFooter. --- .../src/main/java/org/apache/beam/sdk/io/TextIO.java | 12 ++++++++++++ 1 file changed, 12 insertions(+) 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 0895123aba372..e75aaa9f2f5e4 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 @@ -448,10 +448,22 @@ public static Bound withoutValidation() { return new Bound<>(DEFAULT_TEXT_CODER).withoutValidation(); } + /** + * Returns a transform for writing to text files that adds a header string to the files + * it writes. + * + * @param header the string to be added as file header + */ public static Bound withHeader(String header) { return new Bound<>(DEFAULT_TEXT_CODER).withHeader(header); } + /** + * Returns a transform for writing to text files that adds a footer string to the files + * it writes. + * + * @param footer the string to be added as file footer + */ public static Bound withFooter(String footer) { return new Bound<>(DEFAULT_TEXT_CODER).withFooter(footer); } From e5db1c78bbc910db31f7a324ea2f3f48b4a0de3e Mon Sep 17 00:00:00 2001 From: Stas Levin Date: Wed, 7 Sep 2016 18:38:28 +0300 Subject: [PATCH 115/346] Added even more javadoc to TextIO#withHeader and TextIO#withFooter. --- .../core/src/main/java/org/apache/beam/sdk/io/TextIO.java | 4 ++++ 1 file changed, 4 insertions(+) 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 e75aaa9f2f5e4..f09ab6c5654a4 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 @@ -452,6 +452,8 @@ public static Bound withoutValidation() { * Returns a transform for writing to text files that adds a header string to the files * it writes. * + *

              A null value will clear any previously configured header.

              + * * @param header the string to be added as file header */ public static Bound withHeader(String header) { @@ -462,6 +464,8 @@ public static Bound withHeader(String header) { * Returns a transform for writing to text files that adds a footer string to the files * it writes. * + *

              A null value will clear any previously configured footer.

              + * * @param footer the string to be added as file footer */ public static Bound withFooter(String footer) { From 34c731fd23b5984c1389783d2b3bfd561650c216 Mon Sep 17 00:00:00 2001 From: Stas Levin Date: Thu, 8 Sep 2016 00:02:52 +0300 Subject: [PATCH 116/346] Added even more javadoc to TextIO#withHeader and TextIO#withFooter (2). --- .../main/java/org/apache/beam/sdk/io/TextIO.java | 16 ++++++++++++++++ 1 file changed, 16 insertions(+) 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 f09ab6c5654a4..e4fcc327bd0d5 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 @@ -632,11 +632,27 @@ public Bound withoutValidation() { shardTemplate, false); } + /** + * Returns a transform for writing to text files that adds a header string to the files + * it writes. + * + *

              A null value will clear any previously configured header.

              + * + * @param header the string to be added as file header + */ public Bound withHeader(String header) { return new Bound<>(name, filenamePrefix, filenameSuffix, header, footer, coder, numShards, shardTemplate, false); } + /** + * Returns a transform for writing to text files that adds a footer string to the files + * it writes. + * + *

              A null value will clear any previously configured footer.

              + * + * @param footer the string to be added as file footer + */ public Bound withFooter(String footer) { return new Bound<>(name, filenamePrefix, filenameSuffix, header, footer, coder, numShards, shardTemplate, false); From 6cd48c4256539aa45e535729a7a1042f7ccb66fd Mon Sep 17 00:00:00 2001 From: Luke Cwik Date: Wed, 7 Sep 2016 15:38:16 -0700 Subject: [PATCH 117/346] !fixup Minor javadoc clean-up --- .../src/main/java/org/apache/beam/sdk/io/TextIO.java | 12 ++++++++---- 1 file changed, 8 insertions(+), 4 deletions(-) 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 e4fcc327bd0d5..c0761b19ed88c 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 @@ -452,7 +452,7 @@ public static Bound withoutValidation() { * Returns a transform for writing to text files that adds a header string to the files * it writes. * - *

              A null value will clear any previously configured header.

              + *

              A {@code null} value will clear any previously configured header. * * @param header the string to be added as file header */ @@ -464,7 +464,7 @@ public static Bound withHeader(String header) { * Returns a transform for writing to text files that adds a footer string to the files * it writes. * - *

              A null value will clear any previously configured footer.

              + *

              A {@code null} value will clear any previously configured footer. * * @param footer the string to be added as file footer */ @@ -636,7 +636,9 @@ public Bound withoutValidation() { * Returns a transform for writing to text files that adds a header string to the files * it writes. * - *

              A null value will clear any previously configured header.

              + *

              A {@code null} value will clear any previously configured header. + * + *

              Does not modify this object. * * @param header the string to be added as file header */ @@ -649,7 +651,9 @@ public Bound withHeader(String header) { * Returns a transform for writing to text files that adds a footer string to the files * it writes. * - *

              A null value will clear any previously configured footer.

              + *

              A {@code null} value will clear any previously configured footer. + * + *

              Does not modify this object. * * @param footer the string to be added as file footer */ From 59ae94c59931732d5cf78c5431147d580f9ff747 Mon Sep 17 00:00:00 2001 From: manuzhang Date: Mon, 12 Sep 2016 11:45:15 +0800 Subject: [PATCH 118/346] fix import order --- runners/gearpump/pom.xml | 7 ++--- .../gearpump/GearpumpPipelineOptions.java | 8 +++--- .../gearpump/GearpumpPipelineResult.java | 4 ++- .../gearpump/GearpumpPipelineRunner.java | 28 +++++++++---------- .../GearpumpPipelineRunnerRegistrar.java | 9 +++--- .../gearpump/GearpumpPipelineTranslator.java | 7 ++--- .../gearpump/examples/StreamingWordCount.java | 15 +++------- .../examples/UnboundedTextSource.java | 15 +++++----- .../translators/GroupByKeyTranslator.java | 14 ++++++---- .../ParDoBoundMultiTranslator.java | 16 +++++------ .../translators/TransformTranslator.java | 3 +- .../translators/TranslationContext.java | 6 ++-- .../translators/functions/DoFnFunction.java | 14 +++++----- .../translators/io/BoundedSourceWrapper.java | 4 +-- .../translators/io/GearpumpSource.java | 12 ++++---- .../io/UnboundedSourceWrapper.java | 4 +-- .../gearpump/translators/io/ValuesSource.java | 12 ++++---- .../translators/utils/GearpumpDoFnRunner.java | 28 +++++++++---------- .../utils/NoOpSideInputReader.java | 8 +++--- .../translators/utils/NoOpStepContext.java | 6 ++-- .../java/org/apache/beam/sdk/Pipeline.java | 2 +- .../beam/sdk/runners/PipelineRunner.java | 1 + .../beam/sdk/transforms/DoFnAdapters.java | 5 ++++ 23 files changed, 114 insertions(+), 114 deletions(-) diff --git a/runners/gearpump/pom.xml b/runners/gearpump/pom.xml index cc99a7a38489d..296de6b52ff61 100644 --- a/runners/gearpump/pom.xml +++ b/runners/gearpump/pom.xml @@ -122,7 +122,6 @@ org.apache.gearpump gearpump-daemon_2.11 ${gearpump.version} - provided org.apache.gearpump @@ -185,10 +184,6 @@ com.fasterxml.jackson.core jackson-annotations - - com.google.http-client - google-http-client - com.google.guava guava @@ -225,6 +220,7 @@ auto-service 1.0-rc2 + @@ -287,6 +283,7 @@ org.apache.maven.plugins maven-checkstyle-plugin + diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpPipelineOptions.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpPipelineOptions.java index 5b6ee960745d8..e02cbbc01a8d4 100644 --- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpPipelineOptions.java +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpPipelineOptions.java @@ -18,17 +18,17 @@ package org.apache.beam.runners.gearpump; +import com.fasterxml.jackson.annotation.JsonIgnore; + +import java.util.Map; + import org.apache.beam.sdk.options.Default; import org.apache.beam.sdk.options.Description; import org.apache.beam.sdk.options.PipelineOptions; -import com.fasterxml.jackson.annotation.JsonIgnore; - import org.apache.gearpump.cluster.client.ClientContext; import org.apache.gearpump.cluster.embedded.EmbeddedCluster; -import java.util.Map; - /** * Options that configure the Gearpump pipeline. */ diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpPipelineResult.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpPipelineResult.java index 6184bc39a347e..2011a4b68544f 100644 --- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpPipelineResult.java +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpPipelineResult.java @@ -17,14 +17,16 @@ */ package org.apache.beam.runners.gearpump; +import java.io.IOException; + 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.transforms.Aggregator; + import org.joda.time.Duration; -import java.io.IOException; /** * Result of executing a {@link Pipeline} with Gearpump. diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpPipelineRunner.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpPipelineRunner.java index 4182ee4785df6..ad7bb3e06a719 100644 --- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpPipelineRunner.java +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpPipelineRunner.java @@ -17,6 +17,13 @@ */ package org.apache.beam.runners.gearpump; +import com.typesafe.config.Config; +import com.typesafe.config.ConfigValueFactory; + +import java.util.HashMap; +import java.util.Map; + +import org.apache.beam.runners.core.AssignWindows; import org.apache.beam.runners.gearpump.translators.TranslationContext; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.options.PipelineOptions; @@ -30,25 +37,18 @@ 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.AssignWindows; import org.apache.beam.sdk.util.WindowingStrategy; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionList; import org.apache.beam.sdk.values.PInput; import org.apache.beam.sdk.values.POutput; -import com.typesafe.config.Config; -import com.typesafe.config.ConfigValueFactory; - import org.apache.gearpump.cluster.ClusterConfig; import org.apache.gearpump.cluster.UserConfig; import org.apache.gearpump.cluster.client.ClientContext; import org.apache.gearpump.cluster.embedded.EmbeddedCluster; import org.apache.gearpump.streaming.dsl.javaapi.JavaStreamApp; -import java.util.HashMap; -import java.util.Map; - /** * A {@link PipelineRunner} that executes the operations in the * pipeline by first translating them to Gearpump Stream DSL @@ -79,16 +79,16 @@ public OutputT apply( PTransform transform, InputT input) { if (Window.Bound.class.equals(transform.getClass())) { return (OutputT) super.apply( - new AssignWindowsAndSetStrategy((Window.Bound) transform), input); + new AssignWindowsAndSetStrategy((Window.Bound) transform), input); } else if (Flatten.FlattenPCollectionList.class.equals(transform.getClass()) - && ((PCollectionList) input).size() == 0) { - return (OutputT) Pipeline.applyTransform(input, Create.of()); + && ((PCollectionList) input).size() == 0) { + return (OutputT) Pipeline.applyTransform(input.getPipeline().begin(), Create.of()); } else if (Create.Values.class.equals(transform.getClass())) { return (OutputT) PCollection - .createPrimitiveOutputInternal( - input.getPipeline(), - WindowingStrategy.globalDefault(), - PCollection.IsBounded.BOUNDED); + .createPrimitiveOutputInternal( + input.getPipeline(), + WindowingStrategy.globalDefault(), + PCollection.IsBounded.BOUNDED); } else { return super.apply(transform, input); } diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpPipelineRunnerRegistrar.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpPipelineRunnerRegistrar.java index 2b9e89e6a4c42..ca173d10883e3 100644 --- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpPipelineRunnerRegistrar.java +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpPipelineRunnerRegistrar.java @@ -18,14 +18,14 @@ package org.apache.beam.runners.gearpump; +import com.google.auto.service.AutoService; +import com.google.common.collect.ImmutableList; + import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.options.PipelineOptionsRegistrar; import org.apache.beam.sdk.runners.PipelineRunner; import org.apache.beam.sdk.runners.PipelineRunnerRegistrar; -import com.google.auto.service.AutoService; -import com.google.common.collect.ImmutableList; - /** * Contains the {@link PipelineRunnerRegistrar} and {@link PipelineOptionsRegistrar} for the * {@link GearpumpPipelineRunner}. @@ -44,8 +44,7 @@ public static class Runner implements PipelineRunnerRegistrar { @Override public Iterable>> getPipelineRunners() { - return ImmutableList.>>of( - TestGearpumpRunner.class); + return ImmutableList.>>of(TestGearpumpRunner.class); } } diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpPipelineTranslator.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpPipelineTranslator.java index 59f0df7e8d3e5..5045ae46e20b7 100644 --- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpPipelineTranslator.java +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpPipelineTranslator.java @@ -18,6 +18,8 @@ package org.apache.beam.runners.gearpump; +import java.util.HashMap; +import java.util.Map; import org.apache.beam.runners.gearpump.translators.CreateValuesTranslator; import org.apache.beam.runners.gearpump.translators.FlattenPCollectionTranslator; import org.apache.beam.runners.gearpump.translators.GroupByKeyTranslator; @@ -41,9 +43,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.util.HashMap; -import java.util.Map; - /** * {@link GearpumpPipelineTranslator} knows how to translate {@link Pipeline} objects * into Gearpump {@link Graph}. @@ -109,7 +108,7 @@ public void visitPrimitiveTransform(TransformTreeNode node) { @Override public void visitValue(PValue value, TransformTreeNode producer) { - LOG.debug("visiting value {}", value); + LOG.info("visiting value {}", value); } /** diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/examples/StreamingWordCount.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/examples/StreamingWordCount.java index 5f35c6b155a69..ba50de79dda68 100644 --- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/examples/StreamingWordCount.java +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/examples/StreamingWordCount.java @@ -23,11 +23,9 @@ import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.io.Read; import org.apache.beam.sdk.options.PipelineOptionsFactory; -import org.apache.beam.sdk.transforms.Aggregator; import org.apache.beam.sdk.transforms.Count; import org.apache.beam.sdk.transforms.OldDoFn; import org.apache.beam.sdk.transforms.ParDo; -import org.apache.beam.sdk.transforms.Sum; import org.apache.beam.sdk.transforms.windowing.FixedWindows; import org.apache.beam.sdk.transforms.windowing.Window; import org.apache.beam.sdk.values.KV; @@ -45,15 +43,9 @@ public class StreamingWordCount { static class ExtractWordsFn extends OldDoFn { - private final Aggregator emptyLines = - createAggregator("emptyLines", new Sum.SumLongFn()); @Override 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']+"); @@ -81,11 +73,12 @@ public void processElement(ProcessContext c) { public static void main(String[] args) { - GearpumpPipelineOptions options = PipelineOptionsFactory.fromArgs(args).withValidation() - .as(GearpumpPipelineOptions.class); - options.setApplicationName("StreamingWordCount"); + GearpumpPipelineOptions options = PipelineOptionsFactory + .fromArgs(args).as(GearpumpPipelineOptions.class); options.setRunner(GearpumpPipelineRunner.class); + options.setApplicationName("StreamingWordCount"); options.setParallelism(1); + Pipeline p = Pipeline.create(options); PCollection> wordCounts = diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/examples/UnboundedTextSource.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/examples/UnboundedTextSource.java index caf066c9710a5..b01443273287f 100644 --- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/examples/UnboundedTextSource.java +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/examples/UnboundedTextSource.java @@ -18,13 +18,6 @@ package org.apache.beam.runners.gearpump.examples; -import org.apache.beam.sdk.coders.Coder; -import org.apache.beam.sdk.coders.StringUtf8Coder; -import org.apache.beam.sdk.io.UnboundedSource; -import org.apache.beam.sdk.options.PipelineOptions; - -import org.joda.time.Instant; - import java.io.IOException; import java.io.Serializable; import java.util.Collections; @@ -33,6 +26,14 @@ import javax.annotation.Nullable; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.coders.StringUtf8Coder; +import org.apache.beam.sdk.io.UnboundedSource; +import org.apache.beam.sdk.options.PipelineOptions; + +import org.joda.time.Instant; + + /** * unbounded source that reads from text. */ diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/GroupByKeyTranslator.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/GroupByKeyTranslator.java index f36b908f842d9..43e3336b913a5 100644 --- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/GroupByKeyTranslator.java +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/GroupByKeyTranslator.java @@ -18,23 +18,25 @@ package org.apache.beam.runners.gearpump.translators; +import com.google.common.collect.Iterables; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.Iterator; +import java.util.List; + import org.apache.beam.sdk.transforms.GroupByKey; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; -import com.google.common.collect.Iterables; - import org.apache.gearpump.streaming.dsl.javaapi.JavaStream; import org.apache.gearpump.streaming.javaapi.dsl.functions.FlatMapFunction; import org.apache.gearpump.streaming.javaapi.dsl.functions.GroupByFunction; import org.apache.gearpump.streaming.javaapi.dsl.functions.MapFunction; import org.apache.gearpump.streaming.javaapi.dsl.functions.ReduceFunction; -import java.util.ArrayList; -import java.util.Collections; -import java.util.Iterator; -import java.util.List; + /** * {@link GroupByKey} is translated to Gearpump groupBy function. diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/ParDoBoundMultiTranslator.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/ParDoBoundMultiTranslator.java index d5ed0d2b06c11..2b49684c991dd 100644 --- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/ParDoBoundMultiTranslator.java +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/ParDoBoundMultiTranslator.java @@ -18,6 +18,14 @@ package org.apache.beam.runners.gearpump.translators; +import com.google.common.collect.Lists; + +import java.util.Iterator; +import java.util.List; +import java.util.Map; + +import org.apache.beam.runners.core.DoFnRunner; +import org.apache.beam.runners.core.DoFnRunners; import org.apache.beam.runners.gearpump.GearpumpPipelineOptions; import org.apache.beam.runners.gearpump.translators.utils.GearpumpDoFnRunner; import org.apache.beam.runners.gearpump.translators.utils.NoOpSideInputReader; @@ -25,8 +33,6 @@ 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.util.DoFnRunner; -import org.apache.beam.sdk.util.DoFnRunners; import org.apache.beam.sdk.util.SideInputReader; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.WindowingStrategy; @@ -35,17 +41,11 @@ import org.apache.beam.sdk.values.TupleTag; import org.apache.beam.sdk.values.TupleTagList; -import com.google.common.collect.Lists; - import org.apache.gearpump.streaming.dsl.javaapi.JavaStream; import org.apache.gearpump.streaming.javaapi.dsl.functions.FilterFunction; import org.apache.gearpump.streaming.javaapi.dsl.functions.FlatMapFunction; import org.apache.gearpump.streaming.javaapi.dsl.functions.MapFunction; -import java.util.Iterator; -import java.util.List; -import java.util.Map; - /** * {@link ParDo.BoundMulti} is translated to Gearpump flatMap function * with {@link DoFn} wrapped in {@link DoFnMultiFunction}. The outputs are diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/TransformTranslator.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/TransformTranslator.java index 1ed6d5daa28e1..c8587d3914298 100644 --- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/TransformTranslator.java +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/TransformTranslator.java @@ -18,11 +18,10 @@ package org.apache.beam.runners.gearpump.translators; +import java.io.Serializable; import org.apache.beam.sdk.transforms.PTransform; -import java.io.Serializable; - /** * translates {@link PTransform} to Gearpump functions. */ diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/TranslationContext.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/TranslationContext.java index b9b2c7aeeb913..d3bc75d39e7b5 100644 --- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/TranslationContext.java +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/TranslationContext.java @@ -20,6 +20,9 @@ import static com.google.common.base.Preconditions.checkArgument; +import java.util.HashMap; +import java.util.Map; + import org.apache.beam.runners.gearpump.GearpumpPipelineOptions; import org.apache.beam.sdk.runners.TransformTreeNode; import org.apache.beam.sdk.transforms.AppliedPTransform; @@ -33,9 +36,6 @@ import org.apache.gearpump.streaming.dsl.javaapi.JavaStreamApp; import org.apache.gearpump.streaming.source.DataSource; -import java.util.HashMap; -import java.util.Map; - /** * Maintains context data for {@link TransformTranslator}s. */ diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/functions/DoFnFunction.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/functions/DoFnFunction.java index b1ebd2a0d1799..8d163569f00db 100644 --- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/functions/DoFnFunction.java +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/functions/DoFnFunction.java @@ -18,26 +18,26 @@ package org.apache.beam.runners.gearpump.translators.functions; +import com.google.common.collect.Lists; + +import java.util.Iterator; +import java.util.List; + +import org.apache.beam.runners.core.DoFnRunner; +import org.apache.beam.runners.core.DoFnRunners; import org.apache.beam.runners.gearpump.GearpumpPipelineOptions; import org.apache.beam.runners.gearpump.translators.utils.GearpumpDoFnRunner; import org.apache.beam.runners.gearpump.translators.utils.NoOpStepContext; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.OldDoFn; -import org.apache.beam.sdk.util.DoFnRunner; -import org.apache.beam.sdk.util.DoFnRunners; import org.apache.beam.sdk.util.SideInputReader; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.WindowingStrategy; import org.apache.beam.sdk.values.TupleTag; import org.apache.beam.sdk.values.TupleTagList; -import com.google.api.client.util.Lists; - import org.apache.gearpump.streaming.javaapi.dsl.functions.FlatMapFunction; -import java.util.Iterator; -import java.util.List; - /** * Gearpump {@link FlatMapFunction} wrapper over Beam {@link DoFn}. */ diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/io/BoundedSourceWrapper.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/io/BoundedSourceWrapper.java index f25d113e5c9d1..f8891017688a3 100644 --- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/io/BoundedSourceWrapper.java +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/io/BoundedSourceWrapper.java @@ -18,12 +18,12 @@ package org.apache.beam.runners.gearpump.translators.io; +import java.io.IOException; + import org.apache.beam.sdk.io.BoundedSource; import org.apache.beam.sdk.io.Source; import org.apache.beam.sdk.options.PipelineOptions; -import java.io.IOException; - /** * wrapper over BoundedSource for Gearpump DataSource API. */ diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/io/GearpumpSource.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/io/GearpumpSource.java index 892ccc3b39e91..8f2beb20195fa 100644 --- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/io/GearpumpSource.java +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/io/GearpumpSource.java @@ -18,23 +18,23 @@ package org.apache.beam.runners.gearpump.translators.io; +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.ObjectMapper; + +import java.io.IOException; + import org.apache.beam.sdk.io.Source; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.windowing.GlobalWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.util.WindowedValue; -import com.fasterxml.jackson.core.JsonProcessingException; -import com.fasterxml.jackson.databind.ObjectMapper; - import org.apache.gearpump.Message; import org.apache.gearpump.streaming.source.DataSource; import org.apache.gearpump.streaming.task.TaskContext; import org.joda.time.Instant; -import java.io.IOException; - /** * common methods for {@link BoundedSourceWrapper} and {@link UnboundedSourceWrapper}. */ @@ -61,6 +61,7 @@ public void open(TaskContext context, long startTime) { PipelineOptions options = new ObjectMapper() .readValue(serializedOptions, PipelineOptions.class); this.reader = createReader(options); + this.available = reader.start(); } catch (Exception e) { throw new RuntimeException(e); } finally { @@ -97,4 +98,5 @@ public void close() { throw new RuntimeException(e); } } + } diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/io/UnboundedSourceWrapper.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/io/UnboundedSourceWrapper.java index b39f29f4219a5..dfdecb2a21b41 100644 --- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/io/UnboundedSourceWrapper.java +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/io/UnboundedSourceWrapper.java @@ -18,12 +18,12 @@ package org.apache.beam.runners.gearpump.translators.io; +import java.io.IOException; + import org.apache.beam.sdk.io.Source; import org.apache.beam.sdk.io.UnboundedSource; import org.apache.beam.sdk.options.PipelineOptions; -import java.io.IOException; - /** * wrapper over UnboundedSource for Gearpump DataSource API. */ diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/io/ValuesSource.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/io/ValuesSource.java index 24055f7e4d0ad..9359e35ac05cd 100644 --- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/io/ValuesSource.java +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/io/ValuesSource.java @@ -18,12 +18,6 @@ package org.apache.beam.runners.gearpump.translators.io; -import org.apache.beam.sdk.coders.Coder; -import org.apache.beam.sdk.io.UnboundedSource; -import org.apache.beam.sdk.options.PipelineOptions; - -import org.joda.time.Instant; - import java.io.ByteArrayInputStream; import java.io.ByteArrayOutputStream; import java.io.IOException; @@ -36,6 +30,12 @@ import javax.annotation.Nullable; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.io.UnboundedSource; +import org.apache.beam.sdk.options.PipelineOptions; + +import org.joda.time.Instant; + /** * unbounded source that reads from a Java {@link Iterable}. */ diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/utils/GearpumpDoFnRunner.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/utils/GearpumpDoFnRunner.java index be0d0256290d3..e2055759b6047 100644 --- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/utils/GearpumpDoFnRunner.java +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/utils/GearpumpDoFnRunner.java @@ -18,6 +18,20 @@ package org.apache.beam.runners.gearpump.translators.utils; +import com.google.common.base.Preconditions; +import com.google.common.collect.Iterables; +import com.google.common.collect.Sets; + +import java.io.IOException; +import java.io.Serializable; +import java.util.Collection; +import java.util.Iterator; +import java.util.List; +import java.util.Set; + +import org.apache.beam.runners.core.DoFnRunner; +import org.apache.beam.runners.core.DoFnRunners; +import org.apache.beam.runners.core.SimpleDoFnRunner; import org.apache.beam.runners.gearpump.GearpumpPipelineOptions; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.IterableCoder; @@ -31,11 +45,8 @@ 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.DoFnRunner; -import org.apache.beam.sdk.util.DoFnRunners; import org.apache.beam.sdk.util.ExecutionContext; import org.apache.beam.sdk.util.SideInputReader; -import org.apache.beam.sdk.util.SimpleDoFnRunner; import org.apache.beam.sdk.util.SystemDoFnInternal; import org.apache.beam.sdk.util.TimerInternals; import org.apache.beam.sdk.util.UserCodeException; @@ -46,19 +57,8 @@ import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TupleTag; -import com.google.common.base.Preconditions; -import com.google.common.collect.Iterables; -import com.google.common.collect.Sets; - import org.joda.time.Instant; -import java.io.IOException; -import java.io.Serializable; -import java.util.Collection; -import java.util.Iterator; -import java.util.List; -import java.util.Set; - /** * a serializable {@link SimpleDoFnRunner}. */ diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/utils/NoOpSideInputReader.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/utils/NoOpSideInputReader.java index 600ebfb225d09..d1a91988d962b 100644 --- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/utils/NoOpSideInputReader.java +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/utils/NoOpSideInputReader.java @@ -18,14 +18,14 @@ package org.apache.beam.runners.gearpump.translators.utils; -import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.SideInputReader; -import org.apache.beam.sdk.values.PCollectionView; - import java.io.Serializable; import javax.annotation.Nullable; +import org.apache.beam.sdk.transforms.windowing.BoundedWindow; +import org.apache.beam.sdk.util.SideInputReader; +import org.apache.beam.sdk.values.PCollectionView; + /** * no-op side input reader. */ diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/utils/NoOpStepContext.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/utils/NoOpStepContext.java index ce0935a38081d..45f146b7d9650 100644 --- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/utils/NoOpStepContext.java +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/utils/NoOpStepContext.java @@ -18,6 +18,9 @@ package org.apache.beam.runners.gearpump.translators.utils; +import java.io.IOException; +import java.io.Serializable; + import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.util.ExecutionContext; @@ -26,9 +29,6 @@ import org.apache.beam.sdk.util.state.StateInternals; import org.apache.beam.sdk.values.TupleTag; -import java.io.IOException; -import java.io.Serializable; - /** * serializable {@link ExecutionContext.StepContext} that basically does nothing. */ 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 53f46f6f06aba..e95304dc81e2c 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 @@ -134,7 +134,7 @@ public PipelineExecutionException(Throwable cause) { */ public static Pipeline create(PipelineOptions options) { Pipeline pipeline = new Pipeline(PipelineRunner.fromOptions(options), options); - LOG.debug("Creating {}", pipeline); + LOG.info("Creating {}", pipeline); return pipeline; } 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 ede1507853a14..1ec4103fe6d2b 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 @@ -57,6 +57,7 @@ public static PipelineRunner fromOptions(PipelineOptio .fromFactoryMethod("fromOptions") .withArg(PipelineOptions.class, options) .build(); + System.out.println("runner: " + result.getClass().getName()); return result; } 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 4803d77f57f10..642971f8c4516 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,6 +18,7 @@ package org.apache.beam.sdk.transforms; import java.io.IOException; + import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.display.DisplayData; import org.apache.beam.sdk.transforms.reflect.DoFnInvoker; @@ -31,6 +32,8 @@ import org.apache.beam.sdk.values.TypeDescriptor; import org.joda.time.Duration; import org.joda.time.Instant; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * Utility class containing adapters for running a {@link DoFn} as an {@link OldDoFn}. @@ -72,6 +75,8 @@ public static OldDoFn toOldDoFn(DoFn extends OldDoFn { private final DoFn fn; private transient DoFnInvoker invoker; + private static final Logger LOG = + LoggerFactory.getLogger(SimpleDoFnAdapter.class); SimpleDoFnAdapter(DoFn fn) { super(fn.aggregators); From 272fe9f6eb6c721a1d4a39cc816e3de9dc705373 Mon Sep 17 00:00:00 2001 From: manuzhang Date: Wed, 14 Sep 2016 11:20:27 +0800 Subject: [PATCH 119/346] [BEAM-79] fix integration-test failure --- runners/gearpump/pom.xml | 55 ++++++++++++++++++++-------------------- 1 file changed, 27 insertions(+), 28 deletions(-) diff --git a/runners/gearpump/pom.xml b/runners/gearpump/pom.xml index 296de6b52ff61..6576ba63b59ad 100644 --- a/runners/gearpump/pom.xml +++ b/runners/gearpump/pom.xml @@ -32,6 +32,14 @@ Apache Beam :: Runners :: Gearpump jar + + + apache-repo + apache maven repo + https://repository.apache.org/content/repositories/releases + + + UTF-8 UTF-8 @@ -47,22 +55,24 @@ org.apache.maven.plugins maven-surefire-plugin - 2.19.1 runnable-on-service-tests + integration-test + + test + org.apache.beam.sdk.testing.RunnableOnService none true org.apache.beam:beam-sdks-java-core - org.apache.beam:beam-runners-java-core - org.apache.beam.sdk.io.BigQueryIOTest, + org.apache.beam.sdk.io.gcp.bigquery.BigQueryIOTest, org.apache.beam.sdk.io.CountingInputTest, org.apache.beam.sdk.io.CountingSourceTest, org.apache.beam.sdk.testing.PAssertTest, @@ -188,6 +198,20 @@ com.google.guava guava + + org.apache.gearpump + gearpump-shaded-metrics-graphite_2.11 + ${gearpump.version} + assembly + test + + + org.apache.gearpump + gearpump-shaded-guava_2.11 + ${gearpump.version} + assembly + test + junit junit @@ -218,9 +242,7 @@ com.google.auto.service auto-service - 1.0-rc2 - @@ -245,29 +267,6 @@ maven-compiler-plugin - - - maven-failsafe-plugin - - - - integration-test - verify - - - - - 1 - -Dlog4j.configuration=log4j-test.properties -XX:-UseGCOverheadLimit - - - - - - org.apache.maven.plugins - maven-surefire-plugin - - org.apache.maven.plugins From 8f013cb76fb85421da00eb8df0074dac0a8233fa Mon Sep 17 00:00:00 2001 From: manuzhang Date: Wed, 26 Oct 2016 11:22:56 +0800 Subject: [PATCH 120/346] post-merge fix --- .../beam/runners/core/SimpleDoFnRunner.java | 586 ++++++++++++++++++ .../beam/runners/direct/DirectRunner.java | 22 - .../runners/direct/KeyedResourcePool.java | 47 -- .../direct/LockedKeyedResourcePool.java | 95 --- .../direct/ParDoSingleEvaluatorFactory.java | 2 +- .../direct/TransformEvaluatorRegistry.java | 37 -- .../direct/LockedKeyedResourcePoolTest.java | 163 ----- .../examples/streaming}/package-info.java | 4 +- .../beam/runners/flink/package-info.java | 22 - runners/gearpump/pom.xml | 2 +- .../gearpump/GearpumpPipelineResult.java | 6 + .../gearpump/GearpumpPipelineRunner.java | 2 - .../translators/TransformTranslator.java | 12 +- .../io/UnboundedSourceWrapper.java | 45 ++ .../translators/utils/GearpumpDoFnRunner.java | 16 +- .../beam/runners/dataflow/util/DoFnInfo.java | 1 - .../spark/translation/DoFnFunction.java | 2 - .../java/org/apache/beam/sdk/Pipeline.java | 2 +- .../java/org/apache/beam/sdk/io/TextIO.java | 15 +- .../beam/sdk/runners/PipelineRunner.java | 1 - .../apache/beam/sdk/testing/TestPipeline.java | 2 - .../org/apache/beam/sdk/transforms/DoFn.java | 14 - .../org/apache/beam/sdk/transforms/ParDo.java | 12 - .../beam/sdk/transforms/LatestFnTests.java | 233 ------- 24 files changed, 663 insertions(+), 680 deletions(-) create mode 100644 runners/core-java/src/main/java/org/apache/beam/runners/core/SimpleDoFnRunner.java delete mode 100644 runners/direct-java/src/main/java/org/apache/beam/runners/direct/KeyedResourcePool.java delete mode 100644 runners/direct-java/src/main/java/org/apache/beam/runners/direct/LockedKeyedResourcePool.java delete mode 100644 runners/direct-java/src/test/java/org/apache/beam/runners/direct/LockedKeyedResourcePoolTest.java rename {sdks/java/io/mongodb/src/test/java/org/apache/beam/sdk/io/mongodb => runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/streaming}/package-info.java (89%) delete mode 100644 runners/flink/runner/src/test/java/org/apache/beam/runners/flink/package-info.java rename sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/package-info.java => runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/TransformTranslator.java (70%) create mode 100644 runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/io/UnboundedSourceWrapper.java delete mode 100644 sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/LatestFnTests.java 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 new file mode 100644 index 0000000000000..dec9905a0d602 --- /dev/null +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/SimpleDoFnRunner.java @@ -0,0 +1,586 @@ +/* + * 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.checkNotNull; + +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; +import org.apache.beam.sdk.transforms.Combine.CombineFn; +import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.transforms.DoFn.InputProvider; +import org.apache.beam.sdk.transforms.DoFn.OutputReceiver; +import org.apache.beam.sdk.transforms.reflect.DoFnInvoker; +import org.apache.beam.sdk.transforms.reflect.DoFnInvokers; +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.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.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.StateInternals; +import org.apache.beam.sdk.values.PCollectionView; +import org.apache.beam.sdk.values.TupleTag; +import org.joda.time.Instant; +import org.joda.time.format.PeriodFormat; + +/** + * Runs a {@link DoFn} by constructing the appropriate contexts and passing them in. + * + * @param the type of the {@link DoFn} (main) input elements + * @param the type of the {@link DoFn} (main) output elements + */ +public class SimpleDoFnRunner implements DoFnRunner { + + /** The {@link DoFn} being run. */ + private final DoFn fn; + + /** The {@link DoFnInvoker} being run. */ + private final DoFnInvoker invoker; + + /** The context used for running the {@link DoFn}. */ + private final DoFnContext context; + + private final OutputManager outputManager; + + private final TupleTag mainOutputTag; + + private final boolean observesWindow; + + public SimpleDoFnRunner( + PipelineOptions options, + DoFn fn, + SideInputReader sideInputReader, + OutputManager outputManager, + TupleTag mainOutputTag, + List> sideOutputTags, + StepContext stepContext, + AggregatorFactory aggregatorFactory, + WindowingStrategy windowingStrategy) { + this.fn = fn; + this.observesWindow = + DoFnSignatures.INSTANCE.getSignature(fn.getClass()).processElement().observesWindow(); + this.invoker = DoFnInvokers.INSTANCE.newByteBuddyInvoker(fn); + this.outputManager = outputManager; + this.mainOutputTag = mainOutputTag; + this.context = + new DoFnContext<>( + options, + fn, + sideInputReader, + outputManager, + mainOutputTag, + sideOutputTags, + stepContext, + aggregatorFactory, + windowingStrategy == null ? null : windowingStrategy.getWindowFn()); + } + + @Override + public void startBundle() { + // This can contain user code. Wrap it in case it throws an exception. + try { + invoker.invokeStartBundle(context); + } catch (Throwable t) { + // Exception in user code. + throw wrapUserCodeException(t); + } + } + + @Override + public void processElement(WindowedValue compressedElem) { + if (observesWindow) { + for (WindowedValue elem : compressedElem.explodeWindows()) { + invokeProcessElement(elem); + } + } else { + invokeProcessElement(compressedElem); + } + } + + private void invokeProcessElement(WindowedValue elem) { + final DoFn.ProcessContext processContext = createProcessContext(elem); + + // Note that if the element must be exploded into all its windows, that has to be done outside + // of this runner. + final DoFn.ExtraContextFactory extraContextFactory = + createExtraContextFactory(elem); + + // This can contain user code. Wrap it in case it throws an exception. + try { + invoker.invokeProcessElement(processContext, extraContextFactory); + } catch (Exception ex) { + throw wrapUserCodeException(ex); + } + } + + @Override + public void finishBundle() { + // This can contain user code. Wrap it in case it throws an exception. + try { + invoker.invokeFinishBundle(context); + } catch (Throwable t) { + // Exception in user code. + throw wrapUserCodeException(t); + } + } + + /** Returns a new {@link DoFn.ProcessContext} for the given element. */ + private DoFn.ProcessContext createProcessContext(WindowedValue elem) { + return new DoFnProcessContext(fn, context, elem); + } + + private DoFn.ExtraContextFactory createExtraContextFactory( + WindowedValue elem) { + return new DoFnExtraContextFactory(elem.getWindows(), elem.getPane()); + } + + private RuntimeException wrapUserCodeException(Throwable t) { + throw UserCodeException.wrapIf(!isSystemDoFn(), t); + } + + private boolean isSystemDoFn() { + return invoker.getClass().isAnnotationPresent(SystemDoFnInternal.class); + } + + /** + * A concrete implementation of {@code DoFn.Context} used for running a {@link DoFn}. + * + * @param the type of the {@link DoFn} (main) input elements + * @param the type of the {@link DoFn} (main) output elements + */ + private static class DoFnContext extends DoFn.Context { + private static final int MAX_SIDE_OUTPUTS = 1000; + + final PipelineOptions options; + final DoFn fn; + final SideInputReader sideInputReader; + final OutputManager outputManager; + final TupleTag mainOutputTag; + final StepContext stepContext; + final AggregatorFactory aggregatorFactory; + final WindowFn windowFn; + + /** + * The set of known output tags, some of which may be undeclared, so we can throw an exception + * when it exceeds {@link #MAX_SIDE_OUTPUTS}. + */ + private Set> outputTags; + + public DoFnContext( + PipelineOptions options, + DoFn fn, + SideInputReader sideInputReader, + OutputManager outputManager, + TupleTag mainOutputTag, + List> sideOutputTags, + StepContext stepContext, + AggregatorFactory aggregatorFactory, + WindowFn windowFn) { + fn.super(); + this.options = options; + this.fn = fn; + this.sideInputReader = sideInputReader; + this.outputManager = outputManager; + this.mainOutputTag = mainOutputTag; + this.outputTags = Sets.newHashSet(); + + outputTags.add(mainOutputTag); + for (TupleTag sideOutputTag : sideOutputTags) { + outputTags.add(sideOutputTag); + } + + this.stepContext = stepContext; + this.aggregatorFactory = aggregatorFactory; + this.windowFn = windowFn; + super.setupDelegateAggregators(); + } + + ////////////////////////////////////////////////////////////////////////////// + + @Override + public PipelineOptions getPipelineOptions() { + return options; + } + + WindowedValue makeWindowedValue( + T output, Instant timestamp, Collection windows, PaneInfo pane) { + final Instant inputTimestamp = timestamp; + + if (timestamp == null) { + timestamp = BoundedWindow.TIMESTAMP_MIN_VALUE; + } + + if (windows == null) { + try { + // The windowFn can never succeed at accessing the element, so its type does not + // matter here + @SuppressWarnings("unchecked") + WindowFn objectWindowFn = (WindowFn) windowFn; + windows = + objectWindowFn.assignWindows( + objectWindowFn.new AssignContext() { + @Override + public Object element() { + throw new UnsupportedOperationException( + "WindowFn attempted to access input element when none was available"); + } + + @Override + public Instant timestamp() { + if (inputTimestamp == null) { + throw new UnsupportedOperationException( + "WindowFn attempted to access input timestamp when none was available"); + } + return inputTimestamp; + } + + @Override + public W window() { + throw new UnsupportedOperationException( + "WindowFn attempted to access input windows when none were available"); + } + }); + } catch (Exception e) { + throw UserCodeException.wrap(e); + } + } + + return WindowedValue.of(output, timestamp, windows, pane); + } + + public T sideInput(PCollectionView view, BoundedWindow mainInputWindow) { + if (!sideInputReader.contains(view)) { + throw new IllegalArgumentException("calling sideInput() with unknown view"); + } + BoundedWindow sideInputWindow = + view.getWindowingStrategyInternal().getWindowFn().getSideInputWindow(mainInputWindow); + return sideInputReader.get(view, sideInputWindow); + } + + void outputWindowedValue( + OutputT output, + Instant timestamp, + Collection windows, + PaneInfo pane) { + outputWindowedValue(makeWindowedValue(output, timestamp, windows, pane)); + } + + void outputWindowedValue(WindowedValue windowedElem) { + outputManager.output(mainOutputTag, windowedElem); + if (stepContext != null) { + stepContext.noteOutput(windowedElem); + } + } + + private void sideOutputWindowedValue( + TupleTag tag, + T output, + Instant timestamp, + Collection windows, + PaneInfo pane) { + sideOutputWindowedValue(tag, makeWindowedValue(output, timestamp, windows, pane)); + } + + private void sideOutputWindowedValue(TupleTag tag, WindowedValue windowedElem) { + if (!outputTags.contains(tag)) { + // This tag wasn't declared nor was it seen before during this execution. + // Thus, this must be a new, undeclared and unconsumed output. + // To prevent likely user errors, enforce the limit on the number of side + // outputs. + if (outputTags.size() >= MAX_SIDE_OUTPUTS) { + throw new IllegalArgumentException( + "the number of side outputs has exceeded a limit of " + MAX_SIDE_OUTPUTS); + } + outputTags.add(tag); + } + + outputManager.output(tag, windowedElem); + if (stepContext != null) { + stepContext.noteSideOutput(tag, windowedElem); + } + } + + // Following implementations of output, outputWithTimestamp, and sideOutput + // are only accessible in DoFn.startBundle and DoFn.finishBundle, and will be shadowed by + // ProcessContext's versions in DoFn.processElement. + @Override + public void output(OutputT output) { + outputWindowedValue(output, null, null, PaneInfo.NO_FIRING); + } + + @Override + public void outputWithTimestamp(OutputT output, Instant timestamp) { + outputWindowedValue(output, timestamp, null, PaneInfo.NO_FIRING); + } + + @Override + public void sideOutput(TupleTag tag, T output) { + checkNotNull(tag, "TupleTag passed to sideOutput cannot be null"); + sideOutputWindowedValue(tag, output, null, null, PaneInfo.NO_FIRING); + } + + @Override + public void sideOutputWithTimestamp(TupleTag tag, T output, Instant timestamp) { + checkNotNull(tag, "TupleTag passed to sideOutputWithTimestamp cannot be null"); + sideOutputWindowedValue(tag, output, timestamp, null, PaneInfo.NO_FIRING); + } + + @Override + protected Aggregator createAggregator( + String name, CombineFn combiner) { + checkNotNull(combiner, "Combiner passed to createAggregator cannot be null"); + return aggregatorFactory.createAggregatorForDoFn(fn.getClass(), stepContext, name, combiner); + } + } + + /** + * A concrete implementation of {@link DoFn.ProcessContext} used for running a {@link DoFn} over a + * single element. + * + * @param the type of the {@link DoFn} (main) input elements + * @param the type of the {@link DoFn} (main) output elements + */ + private static class DoFnProcessContext + extends DoFn.ProcessContext { + + final DoFn fn; + final DoFnContext context; + final WindowedValue windowedValue; + + public DoFnProcessContext( + DoFn fn, + DoFnContext context, + WindowedValue windowedValue) { + fn.super(); + this.fn = fn; + this.context = context; + this.windowedValue = windowedValue; + } + + @Override + public PipelineOptions getPipelineOptions() { + return context.getPipelineOptions(); + } + + @Override + public InputT element() { + return windowedValue.getValue(); + } + + @Override + public T sideInput(PCollectionView view) { + checkNotNull(view, "View passed to sideInput cannot be null"); + Iterator windowIter = windows().iterator(); + BoundedWindow window; + if (!windowIter.hasNext()) { + if (context.windowFn instanceof GlobalWindows) { + // TODO: Remove this once GroupByKeyOnly no longer outputs elements + // without windows + window = GlobalWindow.INSTANCE; + } else { + throw new IllegalStateException( + "sideInput called when main input element is not in any windows"); + } + } else { + window = windowIter.next(); + if (windowIter.hasNext()) { + throw new IllegalStateException( + "sideInput called when main input element is in multiple windows"); + } + } + return context.sideInput(view, window); + } + + @Override + public PaneInfo pane() { + return windowedValue.getPane(); + } + + @Override + public void output(OutputT output) { + context.outputWindowedValue(windowedValue.withValue(output)); + } + + @Override + public void outputWithTimestamp(OutputT output, Instant timestamp) { + checkTimestamp(timestamp); + context.outputWindowedValue( + output, timestamp, windowedValue.getWindows(), windowedValue.getPane()); + } + + void outputWindowedValue( + OutputT output, + Instant timestamp, + Collection windows, + PaneInfo pane) { + context.outputWindowedValue(output, timestamp, windows, pane); + } + + @Override + public void sideOutput(TupleTag tag, T output) { + checkNotNull(tag, "Tag passed to sideOutput cannot be null"); + context.sideOutputWindowedValue(tag, windowedValue.withValue(output)); + } + + @Override + public void sideOutputWithTimestamp(TupleTag tag, T output, Instant timestamp) { + checkNotNull(tag, "Tag passed to sideOutputWithTimestamp cannot be null"); + checkTimestamp(timestamp); + context.sideOutputWindowedValue( + tag, output, timestamp, windowedValue.getWindows(), windowedValue.getPane()); + } + + @Override + public Instant timestamp() { + return windowedValue.getTimestamp(); + } + + public Collection windows() { + return windowedValue.getWindows(); + } + + private void checkTimestamp(Instant timestamp) { + if (timestamp.isBefore(windowedValue.getTimestamp().minus(fn.getAllowedTimestampSkew()))) { + throw new IllegalArgumentException( + String.format( + "Cannot output with timestamp %s. Output timestamps must be no earlier than the " + + "timestamp of the current input (%s) minus the allowed skew (%s). See the " + + "DoFn#getAllowedTimestampSkew() Javadoc for details on changing the allowed " + + "skew.", + timestamp, + windowedValue.getTimestamp(), + PeriodFormat.getDefault().print(fn.getAllowedTimestampSkew().toPeriod()))); + } + } + + @Override + protected + Aggregator createAggregator( + String name, CombineFn combiner) { + return context.createAggregator(name, combiner); + } + } + + private class DoFnExtraContextFactory + implements DoFn.ExtraContextFactory { + + /** The windows of the current element. */ + private final Collection windows; + + /** The pane of the current element. */ + private final PaneInfo pane; + + public DoFnExtraContextFactory(Collection windows, PaneInfo pane) { + this.windows = windows; + this.pane = pane; + } + + @Override + public BoundedWindow window() { + return Iterables.getOnlyElement(windows); + } + + @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 WindowingInternals windowingInternals() { + return new WindowingInternals() { + @Override + public Collection windows() { + return windows; + } + + @Override + public PaneInfo pane() { + return pane; + } + + @Override + 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(); + } + + @Override + public void outputWindowedValue( + OutputT output, + Instant timestamp, + Collection windows, + PaneInfo pane) {} + + @Override + public T sideInput(PCollectionView view, BoundedWindow mainInputWindow) { + return context.sideInput(view, mainInputWindow); + } + }; + } + } +} 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 f87f1c1c278d9..e02c8a603933e 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 @@ -456,28 +456,6 @@ public ExecutorService get() { } - /** - * A {@link Supplier} that creates a {@link NanosOffsetClock}. - */ - private static class NanosOffsetClockSupplier implements Supplier { - @Override - public Clock get() { - return NanosOffsetClock.create(); - } - } - - /** - * A {@link Supplier} that creates a {@link ExecutorService} based on - * {@link Executors#newFixedThreadPool(int)}. - */ - private static class FixedThreadPoolSupplier implements Supplier { - @Override - public ExecutorService get() { - return Executors.newFixedThreadPool(Runtime.getRuntime().availableProcessors()); - } - } - - /** * A {@link Supplier} that creates a {@link NanosOffsetClock}. */ diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/KeyedResourcePool.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/KeyedResourcePool.java deleted file mode 100644 index b976b696925d1..0000000000000 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/KeyedResourcePool.java +++ /dev/null @@ -1,47 +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.direct; - -import com.google.common.base.Optional; -import java.util.concurrent.Callable; -import java.util.concurrent.ExecutionException; - -/** - * A pool of resources associated with specific keys. Implementations enforce specific use patterns, - * such as limiting the the number of outstanding elements available per key. - */ -interface KeyedResourcePool { - /** - * Tries to acquire a value for the provided key, loading it via the provided loader if necessary. - * - *

              If the returned {@link Optional} contains a value, the caller obtains ownership of that - * value. The value should be released back to this {@link KeyedResourcePool} after the - * caller no longer has use of it using {@link #release(Object, Object)}. - * - *

              The provided {@link Callable} must not return null; it may either return a non-null - * value or throw an exception. - */ - Optional tryAcquire(K key, Callable loader) throws ExecutionException; - - /** - * Release the provided value, relinquishing ownership of it. Future calls to - * {@link #tryAcquire(Object, Callable)} may return the released value. - */ - void release(K key, V value); -} diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/LockedKeyedResourcePool.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/LockedKeyedResourcePool.java deleted file mode 100644 index 8b1e0b17e618b..0000000000000 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/LockedKeyedResourcePool.java +++ /dev/null @@ -1,95 +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.direct; - -import static com.google.common.base.Preconditions.checkNotNull; -import static com.google.common.base.Preconditions.checkState; - -import com.google.common.base.Optional; -import com.google.common.util.concurrent.ExecutionError; -import com.google.common.util.concurrent.UncheckedExecutionException; -import java.util.concurrent.Callable; -import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.ConcurrentMap; -import java.util.concurrent.ExecutionException; - -/** - * A {@link KeyedResourcePool} which is limited to at most one outstanding instance at a time for - * each key. - */ -class LockedKeyedResourcePool implements KeyedResourcePool { - /** - * A map from each key to an {@link Optional} of the associated value. At most one value is stored - * per key, and it is obtained by at most one thread at a time. - * - *

              For each key in this map: - * - *

                - *
              • If there is no associated value, then no value has been stored yet. - *
              • If the value is {@code Optional.absent()} then the value is currently in use. - *
              • If the value is {@code Optional.present()} then the contained value is available for use. - *
              - */ - public static LockedKeyedResourcePool create() { - return new LockedKeyedResourcePool<>(); - } - - private final ConcurrentMap> cache; - - private LockedKeyedResourcePool() { - cache = new ConcurrentHashMap<>(); - } - - @Override - public Optional tryAcquire(K key, Callable loader) throws ExecutionException { - Optional value = cache.replace(key, Optional.absent()); - if (value == null) { - // No value already existed, so populate the cache with the value returned by the loader - cache.putIfAbsent(key, Optional.of(load(loader))); - // Some other thread may obtain the result after the putIfAbsent, so retry acquisition - value = cache.replace(key, Optional.absent()); - } - return value; - } - - private V load(Callable loader) throws ExecutionException { - try { - return loader.call(); - } catch (Error t) { - throw new ExecutionError(t); - } catch (RuntimeException e) { - throw new UncheckedExecutionException(e); - } catch (Exception e) { - throw new ExecutionException(e); - } - } - - @Override - public void release(K key, V value) { - Optional replaced = cache.replace(key, Optional.of(value)); - checkNotNull(replaced, "Tried to release before a value was acquired"); - checkState( - !replaced.isPresent(), - "Released a value to a %s where there is already a value present for key %s (%s). " - + "At most one value may be present at a time.", - LockedKeyedResourcePool.class.getSimpleName(), - key, - replaced); - } -} diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoSingleEvaluatorFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoSingleEvaluatorFactory.java index 9a08e8f5c491a..0584e41ba2be3 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoSingleEvaluatorFactory.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoSingleEvaluatorFactory.java @@ -91,7 +91,7 @@ private TransformEvaluator createSingleEvaluator( stepContext, inputBundle, application, - (OldDoFn) fnLocal.get(), + fnLocal.get(), application.getTransform().getSideInputs(), mainOutputTag, Collections.>emptyList(), 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 648571454b5ca..3dd44a71087df 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 @@ -116,41 +116,4 @@ public void cleanup() throws Exception { throw toThrow; } } - - @Override - public void cleanup() throws Exception { - Collection thrownInCleanup = new ArrayList<>(); - for (TransformEvaluatorFactory factory : factories.values()) { - try { - factory.cleanup(); - } catch (Exception e) { - if (e instanceof InterruptedException) { - Thread.currentThread().interrupt(); - } - thrownInCleanup.add(e); - } - } - finished.set(true); - if (!thrownInCleanup.isEmpty()) { - LOG.error("Exceptions {} thrown while cleaning up evaluators", thrownInCleanup); - Exception toThrow = null; - for (Exception e : thrownInCleanup) { - if (toThrow == null) { - toThrow = e; - } else { - toThrow.addSuppressed(e); - } - } - throw toThrow; - } - } - - /** - * A factory to create Transform Evaluator Registries. - */ - public static class Factory { - public TransformEvaluatorRegistry create() { - return TransformEvaluatorRegistry.defaultRegistry(); - } - } } diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/LockedKeyedResourcePoolTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/LockedKeyedResourcePoolTest.java deleted file mode 100644 index e1e24a37276ab..0000000000000 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/LockedKeyedResourcePoolTest.java +++ /dev/null @@ -1,163 +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.direct; - -import static org.hamcrest.Matchers.equalTo; -import static org.hamcrest.Matchers.is; -import static org.junit.Assert.assertThat; - -import com.google.common.base.Optional; -import com.google.common.util.concurrent.ExecutionError; -import com.google.common.util.concurrent.UncheckedExecutionException; -import java.util.concurrent.Callable; -import java.util.concurrent.ExecutionException; -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 LockedKeyedResourcePool}. - */ -@RunWith(JUnit4.class) -public class LockedKeyedResourcePoolTest { - @Rule public ExpectedException thrown = ExpectedException.none(); - private LockedKeyedResourcePool cache = - LockedKeyedResourcePool.create(); - - @Test - public void acquireReleaseAcquireLastLoadedOrReleased() throws ExecutionException { - Optional returned = cache.tryAcquire("foo", new Callable() { - @Override - public Integer call() throws Exception { - return 3; - } - }); - assertThat(returned.get(), equalTo(3)); - - cache.release("foo", 4); - Optional reacquired = cache.tryAcquire("foo", new Callable() { - @Override - public Integer call() throws Exception { - return 5; - } - }); - assertThat(reacquired.get(), equalTo(4)); - } - - @Test - public void acquireReleaseReleaseThrows() throws ExecutionException { - Optional returned = cache.tryAcquire("foo", new Callable() { - @Override - public Integer call() throws Exception { - return 3; - } - }); - assertThat(returned.get(), equalTo(3)); - - cache.release("foo", 4); - thrown.expect(IllegalStateException.class); - thrown.expectMessage("already a value present"); - thrown.expectMessage("At most one"); - cache.release("foo", 4); - } - - @Test - public void releaseBeforeAcquireThrows() { - thrown.expect(NullPointerException.class); - thrown.expectMessage("before a value was acquired"); - cache.release("bar", 3); - } - - @Test - public void multipleAcquireWithoutReleaseAbsent() throws ExecutionException { - Optional returned = cache.tryAcquire("foo", new Callable() { - @Override - public Integer call() throws Exception { - return 3; - } - }); - Optional secondReturned = cache.tryAcquire("foo", new Callable() { - @Override - public Integer call() throws Exception { - return 3; - } - }); - assertThat(secondReturned.isPresent(), is(false)); - } - - @Test - public void acquireMultipleKeysSucceeds() throws ExecutionException { - Optional returned = cache.tryAcquire("foo", new Callable() { - @Override - public Integer call() throws Exception { - return 3; - } - }); - Optional secondReturned = cache.tryAcquire("bar", new Callable() { - @Override - public Integer call() throws Exception { - return 4; - } - }); - - assertThat(returned.get(), equalTo(3)); - assertThat(secondReturned.get(), equalTo(4)); - } - - @Test - public void acquireThrowsExceptionWrapped() throws ExecutionException { - final Exception cause = new Exception("checkedException"); - thrown.expect(ExecutionException.class); - thrown.expectCause(equalTo(cause)); - cache.tryAcquire("foo", new Callable() { - @Override - public Integer call() throws Exception { - throw cause; - } - }); - } - - @Test - public void acquireThrowsRuntimeExceptionWrapped() throws ExecutionException { - final RuntimeException cause = new RuntimeException("UncheckedException"); - thrown.expect(UncheckedExecutionException.class); - thrown.expectCause(equalTo(cause)); - cache.tryAcquire("foo", new Callable() { - @Override - public Integer call() throws Exception { - throw cause; - } - }); - } - - @Test - public void acquireThrowsErrorWrapped() throws ExecutionException { - final Error cause = new Error("Error"); - thrown.expect(ExecutionError.class); - thrown.expectCause(equalTo(cause)); - cache.tryAcquire("foo", new Callable() { - @Override - public Integer call() throws Exception { - throw cause; - } - }); - } -} diff --git a/sdks/java/io/mongodb/src/test/java/org/apache/beam/sdk/io/mongodb/package-info.java b/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/streaming/package-info.java similarity index 89% rename from sdks/java/io/mongodb/src/test/java/org/apache/beam/sdk/io/mongodb/package-info.java rename to runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/streaming/package-info.java index fd08b5854d3f8..58f41b6a8bbde 100644 --- a/sdks/java/io/mongodb/src/test/java/org/apache/beam/sdk/io/mongodb/package-info.java +++ b/runners/flink/examples/src/main/java/org/apache/beam/runners/flink/examples/streaming/package-info.java @@ -17,6 +17,6 @@ */ /** - * Transforms for reading and writing from MongoDB. + * Flink Beam runner exemple. */ -package org.apache.beam.sdk.io.mongodb; +package org.apache.beam.runners.flink.examples.streaming; diff --git a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/package-info.java b/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/package-info.java deleted file mode 100644 index 57f1e599ee502..0000000000000 --- a/runners/flink/runner/src/test/java/org/apache/beam/runners/flink/package-info.java +++ /dev/null @@ -1,22 +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. - */ - -/** - * Internal implementation of the Beam runner for Apache Flink. - */ -package org.apache.beam.runners.flink; diff --git a/runners/gearpump/pom.xml b/runners/gearpump/pom.xml index 6576ba63b59ad..6c104eb73c80f 100644 --- a/runners/gearpump/pom.xml +++ b/runners/gearpump/pom.xml @@ -225,7 +225,7 @@ org.apache.beam beam-sdks-java-core - test-jar + tests test diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpPipelineResult.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpPipelineResult.java index 2011a4b68544f..e7c621e03d1c4 100644 --- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpPipelineResult.java +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpPipelineResult.java @@ -23,6 +23,7 @@ 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.joda.time.Duration; @@ -60,4 +61,9 @@ public AggregatorValues getAggregatorValues(Aggregator aggregator) new UnsupportedOperationException()); } + @Override + public MetricResults metrics() { + return null; + } + } diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpPipelineRunner.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpPipelineRunner.java index ad7bb3e06a719..9e322278a42ed 100644 --- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpPipelineRunner.java +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpPipelineRunner.java @@ -53,8 +53,6 @@ * A {@link PipelineRunner} that executes the operations in the * pipeline by first translating them to Gearpump Stream DSL * and then executing them on a Gearpump cluster. - *

              > - * This is based on DataflowPipelineRunner. */ @SuppressWarnings({"rawtypes", "unchecked"}) public class GearpumpPipelineRunner extends PipelineRunner { diff --git a/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/package-info.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/TransformTranslator.java similarity index 70% rename from sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/package-info.java rename to runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/TransformTranslator.java index 44dbf4a92cd27..c8587d3914298 100644 --- a/sdks/java/io/kinesis/src/test/java/org/apache/beam/sdk/io/kinesis/package-info.java +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/TransformTranslator.java @@ -16,7 +16,15 @@ * limitations under the License. */ +package org.apache.beam.runners.gearpump.translators; + +import java.io.Serializable; + +import org.apache.beam.sdk.transforms.PTransform; + /** - * Transforms for reading and writing from Amazon Kinesis. + * translates {@link PTransform} to Gearpump functions. */ -package org.apache.beam.sdk.io.kinesis; +public interface TransformTranslator extends Serializable { + void translate(T transform, TranslationContext context); +} diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/io/UnboundedSourceWrapper.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/io/UnboundedSourceWrapper.java new file mode 100644 index 0000000000000..dfdecb2a21b41 --- /dev/null +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/io/UnboundedSourceWrapper.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.runners.gearpump.translators.io; + +import java.io.IOException; + +import org.apache.beam.sdk.io.Source; +import org.apache.beam.sdk.io.UnboundedSource; +import org.apache.beam.sdk.options.PipelineOptions; + +/** + * wrapper over UnboundedSource for Gearpump DataSource API. + */ +public class UnboundedSourceWrapper + extends GearpumpSource { + + private final UnboundedSource source; + + public UnboundedSourceWrapper(UnboundedSource source, + PipelineOptions options) { + super(options); + this.source = source; + } + + @Override + protected Source.Reader createReader(PipelineOptions options) throws IOException { + return source.createReader(options, null); + } +} diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/utils/GearpumpDoFnRunner.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/utils/GearpumpDoFnRunner.java index e2055759b6047..ec86a8d00fe59 100644 --- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/utils/GearpumpDoFnRunner.java +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/utils/GearpumpDoFnRunner.java @@ -18,7 +18,8 @@ package org.apache.beam.runners.gearpump.translators.utils; -import com.google.common.base.Preconditions; +import static org.apache.beam.sdk.repackaged.com.google.common.base.Preconditions.checkNotNull; + import com.google.common.collect.Iterables; import com.google.common.collect.Sets; @@ -59,6 +60,7 @@ import org.joda.time.Instant; + /** * a serializable {@link SimpleDoFnRunner}. */ @@ -330,20 +332,20 @@ public void outputWithTimestamp(OutputT output, Instant timestamp) { @Override public void sideOutput(TupleTag tag, T output) { - Preconditions.checkNotNull(tag, "TupleTag passed to sideOutput cannot be null"); + checkNotNull(tag, "TupleTag passed to sideOutput cannot be null"); sideOutputWindowedValue(tag, output, null, null, PaneInfo.NO_FIRING); } @Override public void sideOutputWithTimestamp(TupleTag tag, T output, Instant timestamp) { - Preconditions.checkNotNull(tag, "TupleTag passed to sideOutputWithTimestamp cannot be null"); + checkNotNull(tag, "TupleTag passed to sideOutputWithTimestamp cannot be null"); sideOutputWindowedValue(tag, output, timestamp, null, PaneInfo.NO_FIRING); } @Override protected Aggregator createAggregatorInternal( String name, Combine.CombineFn combiner) { - Preconditions.checkNotNull(combiner, + checkNotNull(combiner, "Combiner passed to createAggregator cannot be null"); throw new UnsupportedOperationException("aggregator not supported in Gearpump runner"); } @@ -386,7 +388,7 @@ public InputT element() { @Override public T sideInput(PCollectionView view) { - Preconditions.checkNotNull(view, "View passed to sideInput cannot be null"); + checkNotNull(view, "View passed to sideInput cannot be null"); Iterator windowIter = windows().iterator(); BoundedWindow window; if (!windowIter.hasNext()) { @@ -435,13 +437,13 @@ public void outputWithTimestamp(OutputT output, Instant timestamp) { @Override public void sideOutput(TupleTag tag, T output) { - Preconditions.checkNotNull(tag, "Tag passed to sideOutput cannot be null"); + checkNotNull(tag, "Tag passed to sideOutput cannot be null"); context.sideOutputWindowedValue(tag, windowedValue.withValue(output)); } @Override public void sideOutputWithTimestamp(TupleTag tag, T output, Instant timestamp) { - Preconditions.checkNotNull(tag, "Tag passed to sideOutputWithTimestamp cannot be null"); + checkNotNull(tag, "Tag passed to sideOutputWithTimestamp cannot be null"); context.sideOutputWindowedValue( tag, output, timestamp, windowedValue.getWindows(), windowedValue.getPane()); } diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/DoFnInfo.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/DoFnInfo.java index b84a1a8a34a1f..b211c04d88a89 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/DoFnInfo.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/DoFnInfo.java @@ -77,4 +77,3 @@ public Map> getOutputMap() { return outputMap; } } - diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/DoFnFunction.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/DoFnFunction.java index 69c450eb30dbe..4dfbee65a1f2d 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/DoFnFunction.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/DoFnFunction.java @@ -45,8 +45,6 @@ public class DoFnFunction implements FlatMapFunction>, WindowedValue> { private final Accumulator accum; private final OldDoFn mFunction; - private static final Logger LOG = LoggerFactory.getLogger(DoFnFunction.class); - private final SparkRuntimeContext mRuntimeContext; private final Map, KV, BroadcastHelper>> mSideInputs; private final WindowFn windowFn; 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 07767867afe2d..2286832b5d218 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 @@ -134,7 +134,7 @@ public PipelineExecutionException(Throwable cause) { */ public static Pipeline create(PipelineOptions options) { Pipeline pipeline = new Pipeline(PipelineRunner.fromOptions(options), options); - LOG.info("Creating {}", pipeline); + LOG.debug("Creating {}", pipeline); return pipeline; } 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 78ea988f3a555..2dbcda7feeadc 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 @@ -23,6 +23,7 @@ import com.google.common.annotations.VisibleForTesting; import com.google.protobuf.ByteString; + import java.io.IOException; import java.io.OutputStream; import java.nio.ByteBuffer; @@ -33,20 +34,8 @@ import java.nio.charset.StandardCharsets; import java.util.NoSuchElementException; import java.util.regex.Pattern; + import javax.annotation.Nullable; -import org.apache.beam.sdk.coders.Coder; -import org.apache.beam.sdk.coders.Coder.Context; -import org.apache.beam.sdk.coders.StringUtf8Coder; -import org.apache.beam.sdk.coders.VoidCoder; -import org.apache.beam.sdk.io.Read.Bounded; -import org.apache.beam.sdk.options.PipelineOptions; -import org.apache.beam.sdk.transforms.PTransform; -import org.apache.beam.sdk.transforms.display.DisplayData; -import org.apache.beam.sdk.util.IOChannelUtils; -import org.apache.beam.sdk.util.MimeTypes; -import org.apache.beam.sdk.values.PBegin; -import org.apache.beam.sdk.values.PCollection; -import org.apache.beam.sdk.values.PDone; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.Coder.Context; 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 1ec4103fe6d2b..ede1507853a14 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 @@ -57,7 +57,6 @@ public static PipelineRunner fromOptions(PipelineOptio .fromFactoryMethod("fromOptions") .withArg(PipelineOptions.class, options) .build(); - System.out.println("runner: " + result.getClass().getName()); return result; } 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 3202000224cd2..f1bf09d9c2152 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 @@ -95,7 +95,6 @@ public static TestPipeline create() { } public static TestPipeline fromOptions(PipelineOptions options) { - System.out.println(options); return new TestPipeline(PipelineRunner.fromOptions(options), options); } @@ -134,7 +133,6 @@ public static PipelineOptions testingPipelineOptions() { @Nullable String beamTestPipelineOptions = System.getProperty(PROPERTY_BEAM_TEST_PIPELINE_OPTIONS); - System.out.println("options " + beamTestPipelineOptions); PipelineOptions options = Strings.isNullOrEmpty(beamTestPipelineOptions) ? PipelineOptionsFactory.create() 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 f2fa87cb056bd..018877fb67a57 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 @@ -423,20 +423,6 @@ public RestrictionTracker restrictionTracker() { ///////////////////////////////////////////////////////////////////////////// - - /** - * Annotation for the method to use to prepare an instance for processing bundles of elements. The - * method annotated with this must satisfy the following constraints - *

                - *
              • It must have zero arguments. - *
              - */ - @Documented - @Retention(RetentionPolicy.RUNTIME) - @Target(ElementType.METHOD) - public @interface Setup { - } - /** * Annotation for declaring and dereferencing state cells. * 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 a7dc136255993..a3a306a16d3f7 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 @@ -668,18 +668,6 @@ public UnboundMulti withOutputTags( return new UnboundMulti<>(name, sideInputs, mainOutputTag, sideOutputTags); } - /** - * Returns a new {@link ParDo} {@link PTransform} that's like this - * transform but which will invoke the given {@link DoFn} - * function, and which 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. - */ - public Bound of(DoFn fn) { - return of(adapt(fn), fn.getClass()); - } - /** * Returns a new {@link ParDo} {@link PTransform} that's like this * transform but that will invoke the given {@link OldDoFn} diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/LatestFnTests.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/LatestFnTests.java deleted file mode 100644 index 84b5b68d6b4f0..0000000000000 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/LatestFnTests.java +++ /dev/null @@ -1,233 +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.transforms; - -import static org.hamcrest.MatcherAssert.assertThat; -import static org.hamcrest.Matchers.instanceOf; -import static org.hamcrest.Matchers.isOneOf; -import static org.hamcrest.Matchers.nullValue; -import static org.junit.Assert.assertEquals; - -import com.google.common.collect.Lists; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Objects; -import org.apache.beam.sdk.coders.CannotProvideCoderException; -import org.apache.beam.sdk.coders.CoderRegistry; -import org.apache.beam.sdk.coders.NullableCoder; -import org.apache.beam.sdk.coders.VarLongCoder; -import org.apache.beam.sdk.values.TimestampedValue; -import org.joda.time.Instant; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.ExpectedException; -import org.junit.runner.RunWith; -import org.junit.runners.JUnit4; - -/** - * Unit tests for {@link Latest.LatestFn}. - * */ -@RunWith(JUnit4.class) -public class LatestFnTests { - private static final Instant INSTANT = new Instant(100); - private static final long VALUE = 100 * INSTANT.getMillis(); - - private static final TimestampedValue TV = TimestampedValue.of(VALUE, INSTANT); - private static final TimestampedValue TV_MINUS_TEN = - TimestampedValue.of(VALUE - 10, INSTANT.minus(10)); - private static final TimestampedValue TV_PLUS_TEN = - TimestampedValue.of(VALUE + 10, INSTANT.plus(10)); - - @Rule - public final ExpectedException thrown = ExpectedException.none(); - - private final Latest.LatestFn fn = new Latest.LatestFn<>(); - private final Instant baseTimestamp = Instant.now(); - - @Test - public void testDefaultValue() { - assertThat(fn.defaultValue(), nullValue()); - } - - @Test - public void testCreateAccumulator() { - assertEquals(TimestampedValue.atMinimumTimestamp(null), fn.createAccumulator()); - } - - @Test - public void testAddInputInitialAdd() { - TimestampedValue input = TV; - assertEquals(input, fn.addInput(fn.createAccumulator(), input)); - } - - @Test - public void testAddInputMinTimestamp() { - TimestampedValue input = TimestampedValue.atMinimumTimestamp(1234L); - assertEquals(input, fn.addInput(fn.createAccumulator(), input)); - } - - @Test - public void testAddInputEarlierValue() { - assertEquals(TV, fn.addInput(TV, TV_MINUS_TEN)); - } - - @Test - public void testAddInputLaterValue() { - assertEquals(TV_PLUS_TEN, fn.addInput(TV, TV_PLUS_TEN)); - } - - @Test - public void testAddInputSameTimestamp() { - TimestampedValue accum = TimestampedValue.of(100L, INSTANT); - TimestampedValue input = TimestampedValue.of(200L, INSTANT); - - assertThat("Latest for values with the same timestamp is chosen arbitrarily", - fn.addInput(accum, input), isOneOf(accum, input)); - } - - @Test - public void testAddInputNullAccumulator() { - thrown.expect(NullPointerException.class); - thrown.expectMessage("accumulators"); - fn.addInput(null, TV); - } - - @Test - public void testAddInputNullInput() { - thrown.expect(NullPointerException.class); - thrown.expectMessage("input"); - fn.addInput(TV, null); - } - - @Test - public void testAddInputNullValue() { - TimestampedValue input = TimestampedValue.of(null, INSTANT.plus(10)); - assertEquals("Null values are allowed", input, fn.addInput(TV, input)); - } - - @Test - public void testMergeAccumulatorsMultipleValues() { - Iterable> accums = Lists.newArrayList( - TV, - TV_PLUS_TEN, - TV_MINUS_TEN - ); - - assertEquals(TV_PLUS_TEN, fn.mergeAccumulators(accums)); - } - - @Test - public void testMergeAccumulatorsSingleValue() { - assertEquals(TV, fn.mergeAccumulators(Lists.newArrayList(TV))); - } - - @Test - public void testMergeAccumulatorsEmptyIterable() { - ArrayList> emptyAccums = Lists.newArrayList(); - assertEquals(TimestampedValue.atMinimumTimestamp(null), fn.mergeAccumulators(emptyAccums)); - } - - @Test - public void testMergeAccumulatorsDefaultAccumulator() { - TimestampedValue defaultAccum = fn.createAccumulator(); - assertEquals(TV, fn.mergeAccumulators(Lists.newArrayList(TV, defaultAccum))); - } - - @Test - public void testMergeAccumulatorsAllDefaultAccumulators() { - TimestampedValue defaultAccum = fn.createAccumulator(); - assertEquals(defaultAccum, fn.mergeAccumulators( - Lists.newArrayList(defaultAccum, defaultAccum))); - } - - @Test - public void testMergeAccumulatorsNullIterable() { - thrown.expect(NullPointerException.class); - thrown.expectMessage("accumulators"); - fn.mergeAccumulators(null); - } - - @Test - public void testExtractOutput() { - assertEquals(TV.getValue(), fn.extractOutput(TV)); - } - - @Test - public void testExtractOutputDefaultAggregator() { - TimestampedValue accum = fn.createAccumulator(); - assertThat(fn.extractOutput(accum), nullValue()); - } - - @Test - public void testExtractOutputNullValue() { - TimestampedValue accum = TimestampedValue.of(null, baseTimestamp); - assertEquals(null, fn.extractOutput(accum)); - } - - @Test - public void testAggregator() throws Exception { - LatestAggregatorsFn doFn = new LatestAggregatorsFn<>(TV_MINUS_TEN.getValue()); - DoFnTester harness = DoFnTester.of(doFn); - for (TimestampedValue element : Arrays.asList(TV, TV_PLUS_TEN, TV_MINUS_TEN)) { - harness.processTimestampedElement(element); - } - - assertEquals(TV_PLUS_TEN.getValue(), harness.getAggregatorValue(doFn.allValuesAgg)); - assertEquals(TV_MINUS_TEN.getValue(), harness.getAggregatorValue(doFn.specialValueAgg)); - assertThat(harness.getAggregatorValue(doFn.noValuesAgg), nullValue()); - } - - @Test - public void testDefaultCoderHandlesNull() throws CannotProvideCoderException { - Latest.LatestFn fn = new Latest.LatestFn<>(); - - CoderRegistry registry = new CoderRegistry(); - TimestampedValue.TimestampedValueCoder inputCoder = - TimestampedValue.TimestampedValueCoder.of(VarLongCoder.of()); - - assertThat("Default output coder should handle null values", - fn.getDefaultOutputCoder(registry, inputCoder), instanceOf(NullableCoder.class)); - assertThat("Default accumulator coder should handle null values", - fn.getAccumulatorCoder(registry, inputCoder), instanceOf(NullableCoder.class)); - } - - static class LatestAggregatorsFn extends DoFn { - private final T specialValue; - LatestAggregatorsFn(T specialValue) { - this.specialValue = specialValue; - } - - Aggregator, T> allValuesAgg = - createAggregator("allValues", new Latest.LatestFn()); - - Aggregator, T> specialValueAgg = - createAggregator("oneValue", new Latest.LatestFn()); - - Aggregator, T> noValuesAgg = - createAggregator("noValues", new Latest.LatestFn()); - - @ProcessElement - public void processElement(ProcessContext c) { - TimestampedValue val = TimestampedValue.of(c.element(), c.timestamp()); - allValuesAgg.addValue(val); - if (Objects.equals(c.element(), specialValue)) { - specialValueAgg.addValue(val); - } - } - } -} From 94bd47cdb7e4b8f1d874ace1c60e4251636a8110 Mon Sep 17 00:00:00 2001 From: manuzhang Date: Wed, 26 Oct 2016 16:18:39 +0800 Subject: [PATCH 121/346] remove "pipeline" in runner name --- .../{GearpumpPipelineRunner.java => GearpumpRunner.java} | 8 ++++---- ...eRunnerRegistrar.java => GearpumpRunnerRegistrar.java} | 8 ++++---- .../apache/beam/runners/gearpump/TestGearpumpRunner.java | 4 ++-- .../runners/gearpump/examples/StreamingWordCount.java | 4 ++-- 4 files changed, 12 insertions(+), 12 deletions(-) rename runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/{GearpumpPipelineRunner.java => GearpumpRunner.java} (96%) rename runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/{GearpumpPipelineRunnerRegistrar.java => GearpumpRunnerRegistrar.java} (92%) diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpPipelineRunner.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpRunner.java similarity index 96% rename from runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpPipelineRunner.java rename to runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpRunner.java index 9e322278a42ed..ed0813d68d619 100644 --- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpPipelineRunner.java +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpRunner.java @@ -55,21 +55,21 @@ * and then executing them on a Gearpump cluster. */ @SuppressWarnings({"rawtypes", "unchecked"}) -public class GearpumpPipelineRunner extends PipelineRunner { +public class GearpumpRunner extends PipelineRunner { private final GearpumpPipelineOptions options; private static final String GEARPUMP_SERIALIZERS = "gearpump.serializers"; private static final String DEFAULT_APPNAME = "beam_gearpump_app"; - public GearpumpPipelineRunner(GearpumpPipelineOptions options) { + public GearpumpRunner(GearpumpPipelineOptions options) { this.options = options; } - public static GearpumpPipelineRunner fromOptions(PipelineOptions options) { + public static GearpumpRunner fromOptions(PipelineOptions options) { GearpumpPipelineOptions pipelineOptions = PipelineOptionsValidator.validate(GearpumpPipelineOptions.class, options); - return new GearpumpPipelineRunner(pipelineOptions); + return new GearpumpRunner(pipelineOptions); } diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpPipelineRunnerRegistrar.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpRunnerRegistrar.java similarity index 92% rename from runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpPipelineRunnerRegistrar.java rename to runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpRunnerRegistrar.java index ca173d10883e3..b77e1e3b7dbaa 100644 --- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpPipelineRunnerRegistrar.java +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpRunnerRegistrar.java @@ -28,16 +28,16 @@ /** * Contains the {@link PipelineRunnerRegistrar} and {@link PipelineOptionsRegistrar} for the - * {@link GearpumpPipelineRunner}. + * {@link GearpumpRunner}. * * {@link AutoService} will register Gearpump's implementations of the {@link PipelineRunner} * and {@link PipelineOptions} as available pipeline runner services. */ -public class GearpumpPipelineRunnerRegistrar { - private GearpumpPipelineRunnerRegistrar() { } +public class GearpumpRunnerRegistrar { + private GearpumpRunnerRegistrar() { } /** - * Registers the {@link GearpumpPipelineRunner}. + * Registers the {@link GearpumpRunner}. */ @AutoService(PipelineRunnerRegistrar.class) public static class Runner implements PipelineRunnerRegistrar { diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/TestGearpumpRunner.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/TestGearpumpRunner.java index cedd31ff4ab21..89d31a6d8eb37 100644 --- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/TestGearpumpRunner.java +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/TestGearpumpRunner.java @@ -33,14 +33,14 @@ */ public class TestGearpumpRunner extends PipelineRunner { - private final GearpumpPipelineRunner delegate; + private final GearpumpRunner delegate; private final EmbeddedCluster cluster; private TestGearpumpRunner(GearpumpPipelineOptions options) { cluster = EmbeddedCluster.apply(); cluster.start(); options.setEmbeddedCluster(cluster); - delegate = GearpumpPipelineRunner.fromOptions(options); + delegate = GearpumpRunner.fromOptions(options); } public static TestGearpumpRunner fromOptions(PipelineOptions options) { diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/examples/StreamingWordCount.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/examples/StreamingWordCount.java index ba50de79dda68..1d85c2532bab0 100644 --- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/examples/StreamingWordCount.java +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/examples/StreamingWordCount.java @@ -19,7 +19,7 @@ package org.apache.beam.runners.gearpump.examples; import org.apache.beam.runners.gearpump.GearpumpPipelineOptions; -import org.apache.beam.runners.gearpump.GearpumpPipelineRunner; +import org.apache.beam.runners.gearpump.GearpumpRunner; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.io.Read; import org.apache.beam.sdk.options.PipelineOptionsFactory; @@ -75,7 +75,7 @@ public void processElement(ProcessContext c) { public static void main(String[] args) { GearpumpPipelineOptions options = PipelineOptionsFactory .fromArgs(args).as(GearpumpPipelineOptions.class); - options.setRunner(GearpumpPipelineRunner.class); + options.setRunner(GearpumpRunner.class); options.setApplicationName("StreamingWordCount"); options.setParallelism(1); From 3f06382d20a7b3bc686d46d834e489bdc79625b7 Mon Sep 17 00:00:00 2001 From: manuzhang Date: Wed, 26 Oct 2016 20:15:35 +0800 Subject: [PATCH 122/346] upgrade gearpump-runner to 0.4.0-incubating-SNAPSHOT --- runners/gearpump/pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/runners/gearpump/pom.xml b/runners/gearpump/pom.xml index 6c104eb73c80f..abd135f3ba821 100644 --- a/runners/gearpump/pom.xml +++ b/runners/gearpump/pom.xml @@ -23,7 +23,7 @@ org.apache.beam beam-runners-parent - 0.3.0-incubating-SNAPSHOT + 0.4.0-incubating-SNAPSHOT ../pom.xml From 45570b9c7ebb11080deca3346fc601c69796612a Mon Sep 17 00:00:00 2001 From: manuzhang Date: Mon, 31 Oct 2016 11:52:22 +0800 Subject: [PATCH 123/346] [BEAM-79] Port Gearpump runner from OldDoFn to new DoFn --- .../gearpump/GearpumpPipelineTranslator.java | 2 +- .../ParDoBoundMultiTranslator.java | 17 +- .../translators/ParDoBoundTranslator.java | 3 +- .../translators/functions/DoFnFunction.java | 19 +- .../translators/utils/DoFnRunnerFactory.java | 77 +++ .../translators/utils/GearpumpDoFnRunner.java | 516 ------------------ .../utils/NoOpAggregatorFactory.java | 41 ++ 7 files changed, 143 insertions(+), 532 deletions(-) create mode 100644 runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/utils/DoFnRunnerFactory.java delete mode 100644 runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/utils/GearpumpDoFnRunner.java create mode 100644 runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/utils/NoOpAggregatorFactory.java diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpPipelineTranslator.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpPipelineTranslator.java index 5045ae46e20b7..8588fffd32af5 100644 --- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpPipelineTranslator.java +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpPipelineTranslator.java @@ -108,7 +108,7 @@ public void visitPrimitiveTransform(TransformTreeNode node) { @Override public void visitValue(PValue value, TransformTreeNode producer) { - LOG.info("visiting value {}", value); + LOG.debug("visiting value {}", value); } /** diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/ParDoBoundMultiTranslator.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/ParDoBoundMultiTranslator.java index 2b49684c991dd..54f1c3f29f349 100644 --- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/ParDoBoundMultiTranslator.java +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/ParDoBoundMultiTranslator.java @@ -27,11 +27,11 @@ import org.apache.beam.runners.core.DoFnRunner; import org.apache.beam.runners.core.DoFnRunners; import org.apache.beam.runners.gearpump.GearpumpPipelineOptions; -import org.apache.beam.runners.gearpump.translators.utils.GearpumpDoFnRunner; +import org.apache.beam.runners.gearpump.translators.utils.DoFnRunnerFactory; +import org.apache.beam.runners.gearpump.translators.utils.NoOpAggregatorFactory; import org.apache.beam.runners.gearpump.translators.utils.NoOpSideInputReader; import org.apache.beam.runners.gearpump.translators.utils.NoOpStepContext; 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.util.SideInputReader; import org.apache.beam.sdk.util.WindowedValue; @@ -64,7 +64,7 @@ public void translate(ParDo.BoundMulti transform, TranslationCo JavaStream, OutputT>>> outputStream = inputStream.flatMap( new DoFnMultiFunction<>( context.getPipelineOptions(), - transform.getFn(), + transform.getNewFn(), transform.getMainOutputTag(), transform.getSideOutputTags(), inputT.getWindowingStrategy(), @@ -87,18 +87,19 @@ private static class DoFnMultiFunction implements FlatMapFunction, WindowedValue, OutputT>>>, DoFnRunners.OutputManager { - private final DoFnRunner doFnRunner; + private final DoFnRunnerFactory doFnRunnerFactory; + private DoFnRunner doFnRunner; private final List, OutputT>>> outputs = Lists .newArrayList(); public DoFnMultiFunction( GearpumpPipelineOptions pipelineOptions, - OldDoFn doFn, + DoFn doFn, TupleTag mainOutputTag, TupleTagList sideOutputTags, WindowingStrategy windowingStrategy, SideInputReader sideInputReader) { - this.doFnRunner = new GearpumpDoFnRunner<>( + this.doFnRunnerFactory = new DoFnRunnerFactory<>( pipelineOptions, doFn, sideInputReader, @@ -106,12 +107,16 @@ public DoFnMultiFunction( mainOutputTag, sideOutputTags.getAll(), new NoOpStepContext(), + new NoOpAggregatorFactory(), windowingStrategy ); } @Override public Iterator, OutputT>>> apply(WindowedValue wv) { + if (null == doFnRunner) { + doFnRunner = doFnRunnerFactory.createRunner(); + } doFnRunner.startBundle(); doFnRunner.processElement(wv); doFnRunner.finishBundle(); diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/ParDoBoundTranslator.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/ParDoBoundTranslator.java index b97cbb4c73da4..a796c833ea4ef 100644 --- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/ParDoBoundTranslator.java +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/ParDoBoundTranslator.java @@ -21,7 +21,6 @@ import org.apache.beam.runners.gearpump.translators.functions.DoFnFunction; import org.apache.beam.runners.gearpump.translators.utils.NoOpSideInputReader; 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.util.WindowedValue; import org.apache.beam.sdk.util.WindowingStrategy; @@ -39,7 +38,7 @@ public class ParDoBoundTranslator implements @Override public void translate(ParDo.Bound transform, TranslationContext context) { - OldDoFn doFn = transform.getFn(); + DoFn doFn = transform.getNewFn(); PCollection output = context.getOutput(transform); WindowingStrategy windowingStrategy = output.getWindowingStrategy(); diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/functions/DoFnFunction.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/functions/DoFnFunction.java index 8d163569f00db..42969febc6994 100644 --- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/functions/DoFnFunction.java +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/functions/DoFnFunction.java @@ -26,10 +26,10 @@ import org.apache.beam.runners.core.DoFnRunner; import org.apache.beam.runners.core.DoFnRunners; import org.apache.beam.runners.gearpump.GearpumpPipelineOptions; -import org.apache.beam.runners.gearpump.translators.utils.GearpumpDoFnRunner; +import org.apache.beam.runners.gearpump.translators.utils.DoFnRunnerFactory; +import org.apache.beam.runners.gearpump.translators.utils.NoOpAggregatorFactory; import org.apache.beam.runners.gearpump.translators.utils.NoOpStepContext; import org.apache.beam.sdk.transforms.DoFn; -import org.apache.beam.sdk.transforms.OldDoFn; import org.apache.beam.sdk.util.SideInputReader; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.WindowingStrategy; @@ -44,17 +44,17 @@ public class DoFnFunction implements FlatMapFunction, WindowedValue>, DoFnRunners.OutputManager { - private final TupleTag mainTag = new TupleTag() { - }; - private final DoFnRunner doFnRunner; + private final TupleTag mainTag = new TupleTag() {}; private List> outputs = Lists.newArrayList(); + private final DoFnRunnerFactory doFnRunnerFactory; + private DoFnRunner doFnRunner; public DoFnFunction( GearpumpPipelineOptions pipelineOptions, - OldDoFn doFn, + DoFn doFn, WindowingStrategy windowingStrategy, SideInputReader sideInputReader) { - this.doFnRunner = new GearpumpDoFnRunner<>( + this.doFnRunnerFactory = new DoFnRunnerFactory<>( pipelineOptions, doFn, sideInputReader, @@ -62,6 +62,7 @@ public DoFnFunction( mainTag, TupleTagList.empty().getAll(), new NoOpStepContext(), + new NoOpAggregatorFactory(), windowingStrategy ); } @@ -70,6 +71,10 @@ public DoFnFunction( public Iterator> apply(WindowedValue value) { outputs = Lists.newArrayList(); + if (null == doFnRunner) { + doFnRunner = doFnRunnerFactory.createRunner(); + } + doFnRunner.startBundle(); doFnRunner.processElement(value); doFnRunner.finishBundle(); diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/utils/DoFnRunnerFactory.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/utils/DoFnRunnerFactory.java new file mode 100644 index 0000000000000..7119a87033b66 --- /dev/null +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/utils/DoFnRunnerFactory.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.runners.gearpump.translators.utils; + +import java.io.Serializable; +import java.util.List; + +import org.apache.beam.runners.core.DoFnRunner; +import org.apache.beam.runners.core.DoFnRunners; +import org.apache.beam.runners.core.SimpleDoFnRunner; +import org.apache.beam.runners.gearpump.GearpumpPipelineOptions; +import org.apache.beam.sdk.options.PipelineOptions; +import org.apache.beam.sdk.transforms.Aggregator.AggregatorFactory; +import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.util.ExecutionContext; +import org.apache.beam.sdk.util.SideInputReader; +import org.apache.beam.sdk.util.WindowingStrategy; +import org.apache.beam.sdk.values.TupleTag; + +/** + * a serializable {@link SimpleDoFnRunner}. + */ +public class DoFnRunnerFactory implements Serializable { + + private final DoFn fn; + private final transient PipelineOptions options; + private final SideInputReader sideInputReader; + private final DoFnRunners.OutputManager outputManager; + private final TupleTag mainOutputTag; + private final List> sideOutputTags; + private final ExecutionContext.StepContext stepContext; + private final AggregatorFactory aggregatorFactory; + private final WindowingStrategy windowingStrategy; + + public DoFnRunnerFactory( + GearpumpPipelineOptions pipelineOptions, + DoFn doFn, + SideInputReader sideInputReader, + DoFnRunners.OutputManager outputManager, + TupleTag mainOutputTag, + List> sideOutputTags, + ExecutionContext.StepContext stepContext, + AggregatorFactory aggregatorFactory, + WindowingStrategy windowingStrategy) { + this.fn = doFn; + this.options = pipelineOptions; + this.sideInputReader = sideInputReader; + this.outputManager = outputManager; + this.mainOutputTag = mainOutputTag; + this.sideOutputTags = sideOutputTags; + this.stepContext = stepContext; + this.aggregatorFactory = aggregatorFactory; + this.windowingStrategy = windowingStrategy; + } + + public DoFnRunner createRunner() { + return DoFnRunners.createDefault(options, fn, sideInputReader, outputManager, mainOutputTag, + sideOutputTags, stepContext, aggregatorFactory, windowingStrategy); + } + +} diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/utils/GearpumpDoFnRunner.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/utils/GearpumpDoFnRunner.java deleted file mode 100644 index ec86a8d00fe59..0000000000000 --- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/utils/GearpumpDoFnRunner.java +++ /dev/null @@ -1,516 +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.gearpump.translators.utils; - -import static org.apache.beam.sdk.repackaged.com.google.common.base.Preconditions.checkNotNull; - -import com.google.common.collect.Iterables; -import com.google.common.collect.Sets; - -import java.io.IOException; -import java.io.Serializable; -import java.util.Collection; -import java.util.Iterator; -import java.util.List; -import java.util.Set; - -import org.apache.beam.runners.core.DoFnRunner; -import org.apache.beam.runners.core.DoFnRunners; -import org.apache.beam.runners.core.SimpleDoFnRunner; -import org.apache.beam.runners.gearpump.GearpumpPipelineOptions; -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.Combine; -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.transforms.windowing.GlobalWindow; -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; -import org.apache.beam.sdk.util.SideInputReader; -import org.apache.beam.sdk.util.SystemDoFnInternal; -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.StateInternals; -import org.apache.beam.sdk.values.PCollectionView; -import org.apache.beam.sdk.values.TupleTag; - -import org.joda.time.Instant; - - -/** - * a serializable {@link SimpleDoFnRunner}. - */ -public class GearpumpDoFnRunner implements DoFnRunner, - Serializable { - - private final OldDoFn fn; - private final transient PipelineOptions options; - private final SideInputReader sideInputReader; - private final DoFnRunners.OutputManager outputManager; - private final TupleTag mainOutputTag; - private final List> sideOutputTags; - private final ExecutionContext.StepContext stepContext; - private final WindowFn windowFn; - private DoFnContext context; - - public GearpumpDoFnRunner( - GearpumpPipelineOptions pipelineOptions, - OldDoFn doFn, - SideInputReader sideInputReader, - DoFnRunners.OutputManager outputManager, - TupleTag mainOutputTag, - List> sideOutputTags, - ExecutionContext.StepContext stepContext, - WindowingStrategy windowingStrategy) { - this.fn = doFn; - this.options = pipelineOptions; - this.sideInputReader = sideInputReader; - this.outputManager = outputManager; - this.mainOutputTag = mainOutputTag; - this.sideOutputTags = sideOutputTags; - this.stepContext = stepContext; - this.windowFn = windowingStrategy == null ? null : windowingStrategy.getWindowFn(); - } - - @Override - public void startBundle() { - // This can contain user code. Wrap it in case it throws an exception. - try { - if (null == context) { - this.context = new DoFnContext<>( - options, - fn, - sideInputReader, - outputManager, - mainOutputTag, - sideOutputTags, - stepContext, - windowFn - ); - } - fn.startBundle(context); - } catch (Throwable t) { - // Exception in user code. - throw wrapUserCodeException(t); - } - } - - @Override - public void processElement(WindowedValue elem) { - if (elem.getWindows().size() <= 1 - || (!OldDoFn.RequiresWindowAccess.class.isAssignableFrom(fn.getClass()) - && context.sideInputReader.isEmpty())) { - invokeProcessElement(elem); - } else { - // We could modify the windowed value (and the processContext) to - // avoid repeated allocations, but this is more straightforward. - for (BoundedWindow window : elem.getWindows()) { - invokeProcessElement(WindowedValue.of( - elem.getValue(), elem.getTimestamp(), window, elem.getPane())); - } - } - } - - @Override - public void finishBundle() { - // This can contain user code. Wrap it in case it throws an exception. - try { - fn.finishBundle(context); - } catch (Throwable t) { - // Exception in user code. - throw wrapUserCodeException(t); - } - } - - private void invokeProcessElement(WindowedValue elem) { - final OldDoFn.ProcessContext processContext = - new DoFnProcessContext<>(fn, context, elem); - // This can contain user code. Wrap it in case it throws an exception. - try { - fn.processElement(processContext); - } catch (Exception ex) { - throw wrapUserCodeException(ex); - } - } - - private RuntimeException wrapUserCodeException(Throwable t) { - throw UserCodeException.wrapIf(!isSystemDoFn(), t); - } - - private boolean isSystemDoFn() { - return fn.getClass().isAnnotationPresent(SystemDoFnInternal.class); - } - - /** - * A concrete implementation of {@code DoFn.Context} used for running a {@link DoFn}. - * - * @param the type of the DoFn's (main) input elements - * @param the type of the DoFn's (main) output elements - */ - private static class DoFnContext - extends OldDoFn.Context { - private static final int MAX_SIDE_OUTPUTS = 1000; - - final transient PipelineOptions options; - final OldDoFn fn; - final SideInputReader sideInputReader; - final DoFnRunners.OutputManager outputManager; - final TupleTag mainOutputTag; - final ExecutionContext.StepContext stepContext; - final WindowFn windowFn; - - /** - * The set of known output tags, some of which may be undeclared, so we can throw an - * exception when it exceeds {@link #MAX_SIDE_OUTPUTS}. - */ - private final Set> outputTags; - - public DoFnContext(PipelineOptions options, - OldDoFn fn, - SideInputReader sideInputReader, - DoFnRunners.OutputManager outputManager, - TupleTag mainOutputTag, - List> sideOutputTags, - ExecutionContext.StepContext stepContext, - WindowFn windowFn) { - fn.super(); - this.options = options; - this.fn = fn; - this.sideInputReader = sideInputReader; - this.outputManager = outputManager; - this.mainOutputTag = mainOutputTag; - this.outputTags = Sets.newHashSet(); - - outputTags.add(mainOutputTag); - for (TupleTag sideOutputTag : sideOutputTags) { - outputTags.add(sideOutputTag); - } - - this.stepContext = stepContext; - this.windowFn = windowFn; - super.setupDelegateAggregators(); - } - - ////////////////////////////////////////////////////////////////////////////// - - @Override - public PipelineOptions getPipelineOptions() { - return options; - } - - WindowedValue makeWindowedValue( - T output, Instant timestamp, Collection windows, PaneInfo pane) { - final Instant inputTimestamp = timestamp; - - if (timestamp == null) { - timestamp = BoundedWindow.TIMESTAMP_MIN_VALUE; - } - - if (windows == null) { - try { - // The windowFn can never succeed at accessing the element, so its type does not - // matter here - @SuppressWarnings("unchecked") - WindowFn objectWindowFn = (WindowFn) windowFn; - windows = objectWindowFn.assignWindows(objectWindowFn.new AssignContext() { - @Override - public Object element() { - throw new UnsupportedOperationException( - "WindowFn attempted to access input element when none was available"); - } - - @Override - public Instant timestamp() { - if (inputTimestamp == null) { - throw new UnsupportedOperationException( - "WindowFn attempted to access input timestamp when none was available"); - } - return inputTimestamp; - } - - @Override - public BoundedWindow window() { - throw new UnsupportedOperationException( - "WindowFn attempted to access input windows when none were available"); - } - }); - } catch (Exception e) { - throw UserCodeException.wrap(e); - } - } - - return WindowedValue.of(output, timestamp, windows, pane); - } - - public T sideInput(PCollectionView view, BoundedWindow mainInputWindow) { - if (!sideInputReader.contains(view)) { - throw new IllegalArgumentException("calling sideInput() with unknown view"); - } - BoundedWindow sideInputWindow = - view.getWindowingStrategyInternal().getWindowFn().getSideInputWindow(mainInputWindow); - return sideInputReader.get(view, sideInputWindow); - } - - void outputWindowedValue( - OutputT output, - Instant timestamp, - Collection windows, - PaneInfo pane) { - outputWindowedValue(makeWindowedValue(output, timestamp, windows, pane)); - } - - void outputWindowedValue(WindowedValue windowedElem) { - outputManager.output(mainOutputTag, windowedElem); - if (stepContext != null) { - stepContext.noteOutput(windowedElem); - } - } - - protected void sideOutputWindowedValue(TupleTag tag, - T output, - Instant timestamp, - Collection windows, - PaneInfo pane) { - sideOutputWindowedValue(tag, makeWindowedValue(output, timestamp, windows, pane)); - } - - protected void sideOutputWindowedValue(TupleTag tag, WindowedValue windowedElem) { - if (!outputTags.contains(tag)) { - // This tag wasn't declared nor was it seen before during this execution. - // Thus, this must be a new, undeclared and unconsumed output. - // To prevent likely user errors, enforce the limit on the number of side - // outputs. - if (outputTags.size() >= MAX_SIDE_OUTPUTS) { - throw new IllegalArgumentException( - "the number of side outputs has exceeded a limit of " + MAX_SIDE_OUTPUTS); - } - outputTags.add(tag); - } - - outputManager.output(tag, windowedElem); - if (stepContext != null) { - stepContext.noteSideOutput(tag, windowedElem); - } - } - - // Following implementations of output, outputWithTimestamp, and sideOutput - // are only accessible in DoFn.startBundle and DoFn.finishBundle, and will be shadowed by - // ProcessContext's versions in DoFn.processElement. - @Override - public void output(OutputT output) { - outputWindowedValue(output, null, null, PaneInfo.NO_FIRING); - } - - @Override - public void outputWithTimestamp(OutputT output, Instant timestamp) { - outputWindowedValue(output, timestamp, null, PaneInfo.NO_FIRING); - } - - @Override - public void sideOutput(TupleTag tag, T output) { - checkNotNull(tag, "TupleTag passed to sideOutput cannot be null"); - sideOutputWindowedValue(tag, output, null, null, PaneInfo.NO_FIRING); - } - - @Override - public void sideOutputWithTimestamp(TupleTag tag, T output, Instant timestamp) { - checkNotNull(tag, "TupleTag passed to sideOutputWithTimestamp cannot be null"); - sideOutputWindowedValue(tag, output, timestamp, null, PaneInfo.NO_FIRING); - } - - @Override - protected Aggregator createAggregatorInternal( - String name, Combine.CombineFn combiner) { - checkNotNull(combiner, - "Combiner passed to createAggregator cannot be null"); - throw new UnsupportedOperationException("aggregator not supported in Gearpump runner"); - } - } - - - /** - * A concrete implementation of {@code DoFn.ProcessContext} used for - * running a {@link DoFn} over a single element. - * - * @param the type of the DoFn's (main) input elements - * @param the type of the DoFn's (main) output elements - */ - private static class DoFnProcessContext - extends OldDoFn.ProcessContext { - - - final OldDoFn fn; - final DoFnContext context; - final WindowedValue windowedValue; - - public DoFnProcessContext(OldDoFn fn, - DoFnContext context, - WindowedValue windowedValue) { - fn.super(); - this.fn = fn; - this.context = context; - this.windowedValue = windowedValue; - } - - @Override - public PipelineOptions getPipelineOptions() { - return context.getPipelineOptions(); - } - - @Override - public InputT element() { - return windowedValue.getValue(); - } - - @Override - public T sideInput(PCollectionView view) { - checkNotNull(view, "View passed to sideInput cannot be null"); - Iterator windowIter = windows().iterator(); - BoundedWindow window; - if (!windowIter.hasNext()) { - if (context.windowFn instanceof GlobalWindows) { - // TODO: Remove this once GroupByKeyOnly no longer outputs elements - // without windows - window = GlobalWindow.INSTANCE; - } else { - throw new IllegalStateException( - "sideInput called when main input element is not in any windows"); - } - } else { - window = windowIter.next(); - if (windowIter.hasNext()) { - throw new IllegalStateException( - "sideInput called when main input element is in multiple windows"); - } - } - return context.sideInput(view, window); - } - - @Override - public BoundedWindow window() { - if (!(fn instanceof OldDoFn.RequiresWindowAccess)) { - throw new UnsupportedOperationException( - "window() is only available in the context of a DoFn marked as RequiresWindow."); - } - return Iterables.getOnlyElement(windows()); - } - - @Override - public PaneInfo pane() { - return windowedValue.getPane(); - } - - @Override - public void output(OutputT output) { - context.outputWindowedValue(windowedValue.withValue(output)); - } - - @Override - public void outputWithTimestamp(OutputT output, Instant timestamp) { - context.outputWindowedValue(output, timestamp, - windowedValue.getWindows(), windowedValue.getPane()); - } - - @Override - public void sideOutput(TupleTag tag, T output) { - checkNotNull(tag, "Tag passed to sideOutput cannot be null"); - context.sideOutputWindowedValue(tag, windowedValue.withValue(output)); - } - - @Override - public void sideOutputWithTimestamp(TupleTag tag, T output, Instant timestamp) { - checkNotNull(tag, "Tag passed to sideOutputWithTimestamp cannot be null"); - context.sideOutputWindowedValue( - tag, output, timestamp, windowedValue.getWindows(), windowedValue.getPane()); - } - - @Override - public Instant timestamp() { - return windowedValue.getTimestamp(); - } - - public Collection windows() { - return windowedValue.getWindows(); - } - - @Override - public WindowingInternals windowingInternals() { - return new WindowingInternals() { - @Override - public void outputWindowedValue(OutputT output, Instant timestamp, - Collection windows, PaneInfo pane) { - context.outputWindowedValue(output, timestamp, windows, pane); - } - - @Override - public Collection windows() { - return windowedValue.getWindows(); - } - - @Override - public PaneInfo pane() { - return windowedValue.getPane(); - } - - @Override - 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(); - } - - @Override - public T sideInput(PCollectionView view, BoundedWindow mainInputWindow) { - return context.sideInput(view, mainInputWindow); - } - }; - } - - @Override - protected Aggregator - createAggregatorInternal( - String name, Combine.CombineFn combiner) { - return context.createAggregatorInternal(name, combiner); - } - } -} diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/utils/NoOpAggregatorFactory.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/utils/NoOpAggregatorFactory.java new file mode 100644 index 0000000000000..cd404a53af7ab --- /dev/null +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/utils/NoOpAggregatorFactory.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.runners.gearpump.translators.utils; + +import java.io.Serializable; + +import org.apache.beam.sdk.transforms.Aggregator; +import org.apache.beam.sdk.transforms.Aggregator.AggregatorFactory; +import org.apache.beam.sdk.transforms.Combine; +import org.apache.beam.sdk.util.ExecutionContext; + +/** + * no-op aggregator factory. + */ +public class NoOpAggregatorFactory implements AggregatorFactory, Serializable { + + @Override + public Aggregator createAggregatorForDoFn( + Class fnClass, + ExecutionContext.StepContext stepContext, + String aggregatorName, + Combine.CombineFn combine) { + return null; + } +} From 2a96a17f2fe9a391ec5b2f0b4bea223530c1ba34 Mon Sep 17 00:00:00 2001 From: manuzhang Date: Tue, 8 Nov 2016 11:25:27 +0800 Subject: [PATCH 124/346] [BEAM-79] update GearpumpPipelineResult --- .../apache/beam/runners/gearpump/GearpumpPipelineResult.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpPipelineResult.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpPipelineResult.java index e7c621e03d1c4..ed1201ddcc6bb 100644 --- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpPipelineResult.java +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpPipelineResult.java @@ -44,12 +44,12 @@ public State cancel() throws IOException { } @Override - public State waitUntilFinish(Duration duration) throws IOException, InterruptedException { + public State waitUntilFinish(Duration duration) { return null; } @Override - public State waitUntilFinish() throws IOException, InterruptedException { + public State waitUntilFinish() { return null; } From 2afc0cd99e33bc724345a2e5b0498820d05b460c Mon Sep 17 00:00:00 2001 From: manuzhang Date: Tue, 6 Dec 2016 11:28:24 +0800 Subject: [PATCH 125/346] [BEAM-79] fix gearpump runner build failure --- runners/gearpump/pom.xml | 9 ++- .../gearpump/GearpumpPipelineTranslator.java | 12 ++-- .../translators/TranslationContext.java | 4 +- .../gearpump/translators/io/ValuesSource.java | 71 +++++++++---------- 4 files changed, 47 insertions(+), 49 deletions(-) diff --git a/runners/gearpump/pom.xml b/runners/gearpump/pom.xml index abd135f3ba821..04bd724d29900 100644 --- a/runners/gearpump/pom.xml +++ b/runners/gearpump/pom.xml @@ -169,11 +169,6 @@ org.apache.beam beam-runners-core-java
              - - com.google.code.findbugs - annotations - 3.0.1 - org.slf4j slf4j-api @@ -198,6 +193,10 @@ com.google.guava guava + + com.google.code.findbugs + jsr305 + org.apache.gearpump gearpump-shaded-metrics-graphite_2.11 diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpPipelineTranslator.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpPipelineTranslator.java index 8588fffd32af5..84dfeec529e73 100644 --- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpPipelineTranslator.java +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpPipelineTranslator.java @@ -31,7 +31,7 @@ import org.apache.beam.runners.gearpump.translators.TranslationContext; 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; @@ -48,7 +48,7 @@ * into Gearpump {@link Graph}. */ @SuppressWarnings({"rawtypes", "unchecked"}) -public class GearpumpPipelineTranslator implements Pipeline.PipelineVisitor { +public class GearpumpPipelineTranslator extends Pipeline.PipelineVisitor.Defaults { private static final Logger LOG = LoggerFactory.getLogger( GearpumpPipelineTranslator.class); @@ -83,18 +83,18 @@ public void translate(Pipeline pipeline) { } @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()); @@ -107,7 +107,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/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/TranslationContext.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/TranslationContext.java index d3bc75d39e7b5..d9d6a8e74fc27 100644 --- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/TranslationContext.java +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/TranslationContext.java @@ -24,7 +24,7 @@ import java.util.Map; import org.apache.beam.runners.gearpump.GearpumpPipelineOptions; -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.PInput; @@ -53,7 +53,7 @@ public TranslationContext(JavaStreamApp streamApp, GearpumpPipelineOptions pipel } - 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/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/io/ValuesSource.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/io/ValuesSource.java index 9359e35ac05cd..3b67f09036446 100644 --- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/io/ValuesSource.java +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/io/ValuesSource.java @@ -21,16 +21,14 @@ import java.io.ByteArrayInputStream; import java.io.ByteArrayOutputStream; import java.io.IOException; -import java.io.Serializable; import java.util.Collections; import java.util.Iterator; -import java.util.LinkedList; -import java.util.List; import java.util.NoSuchElementException; import javax.annotation.Nullable; import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.coders.IterableCoder; import org.apache.beam.sdk.io.UnboundedSource; import org.apache.beam.sdk.options.PipelineOptions; @@ -41,26 +39,33 @@ */ public class ValuesSource extends UnboundedSource { - private final Iterable values; - private final Coder coder; + private final byte[] values; + private final IterableCoder iterableCoder; public ValuesSource(Iterable values, Coder coder) { - this.values = encode(values, coder); - this.coder = coder; + this.iterableCoder = IterableCoder.of(coder); + this.values = encode(values, iterableCoder); } - private Iterable encode(Iterable values, Coder coder) { - List bytes = new LinkedList<>(); - for (T t: values) { - try { - ByteArrayOutputStream stream = new ByteArrayOutputStream(); - coder.encode(t, stream, Coder.Context.OUTER); - bytes.add(stream.toByteArray()); - } catch (IOException e) { - throw new RuntimeException(e); - } + private byte[] encode(Iterable values, IterableCoder coder) { + ByteArrayOutputStream stream = new ByteArrayOutputStream(); + try { + coder.encode(values, stream, Coder.Context.OUTER); + } catch (IOException ex) { + throw new RuntimeException(ex); + } + return stream.toByteArray(); + } + + private Iterable decode(byte[] bytes) throws IOException{ + ByteArrayInputStream inputStream = new ByteArrayInputStream(bytes); + try { + return iterableCoder.decode(inputStream, Coder.Context.OUTER); + } catch (IOException ex) { + throw new RuntimeException(ex); + } finally { + inputStream.close(); } - return bytes; } @Override @@ -72,7 +77,11 @@ public java.util.List> generateInit @Override public UnboundedReader createReader(PipelineOptions options, @Nullable CheckpointMark checkpointMark) { - return new ValuesReader<>(values, coder, this); + try { + return new ValuesReader<>(decode(values), iterableCoder, this); + } catch (IOException e) { + throw new RuntimeException(e); + } } @Nullable @@ -87,32 +96,22 @@ public void validate() { @Override public Coder getDefaultOutputCoder() { - return coder; + return iterableCoder.getElemCoder(); } - private static class ValuesReader extends UnboundedReader implements Serializable { - - private final Iterable values; - private final Coder coder; + private static class ValuesReader extends UnboundedReader { private final UnboundedSource source; - private transient Iterator iterator; + private final Iterable values; + private transient Iterator iterator; private T current; - public ValuesReader(Iterable values, Coder coder, + public ValuesReader(Iterable values, IterableCoder coder, UnboundedSource source) { this.values = values; - this.coder = coder; this.source = source; } - private T decode(byte[] bytes) throws IOException { - ByteArrayInputStream inputStream = new ByteArrayInputStream(bytes); - try { - return coder.decode(inputStream, Coder.Context.OUTER); - } finally { - inputStream.close(); - } - } + @Override public boolean start() throws IOException { @@ -125,7 +124,7 @@ public boolean start() throws IOException { @Override public boolean advance() throws IOException { if (iterator.hasNext()) { - current = decode(iterator.next()); + current = iterator.next(); return true; } else { return false; From 46d3563ec37e7f5b39ae564ac73c7ca2b0185bb5 Mon Sep 17 00:00:00 2001 From: manuzhang Date: Fri, 16 Dec 2016 16:48:16 +0800 Subject: [PATCH 126/346] Upgrade Gearpump version --- runners/gearpump/pom.xml | 39 +------ .../translators/GroupByKeyTranslator.java | 109 +++++++++++++----- .../translators/io/GearpumpSource.java | 28 +++-- .../gearpump/translators/io/ValuesSource.java | 21 ++-- .../translators/utils/TranslatorUtils.java | 35 ++++++ 5 files changed, 148 insertions(+), 84 deletions(-) create mode 100644 runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/utils/TranslatorUtils.java diff --git a/runners/gearpump/pom.xml b/runners/gearpump/pom.xml index 04bd724d29900..9320561fc7c64 100644 --- a/runners/gearpump/pom.xml +++ b/runners/gearpump/pom.xml @@ -43,7 +43,7 @@ UTF-8 UTF-8 - 0.8.1 + 0.8.3-SNAPSHOT @@ -121,29 +121,12 @@ gearpump-core_2.11 ${gearpump.version} provided - - - com.google.code.findbugs - jsr305 - - - - - org.apache.gearpump - gearpump-daemon_2.11 - ${gearpump.version} - - - org.apache.gearpump - gearpump-experimental-cgroup_2.11 - - com.typesafe config - provided 1.3.0 + provided org.scala-lang @@ -169,10 +152,6 @@ org.apache.beam beam-runners-core-java - - org.slf4j - slf4j-api - joda-time joda-time @@ -197,20 +176,6 @@ com.google.code.findbugs jsr305 - - org.apache.gearpump - gearpump-shaded-metrics-graphite_2.11 - ${gearpump.version} - assembly - test - - - org.apache.gearpump - gearpump-shaded-guava_2.11 - ${gearpump.version} - assembly - test - junit junit diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/GroupByKeyTranslator.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/GroupByKeyTranslator.java index 43e3336b913a5..d64f1bf601037 100644 --- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/GroupByKeyTranslator.java +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/GroupByKeyTranslator.java @@ -19,23 +19,33 @@ package org.apache.beam.runners.gearpump.translators; import com.google.common.collect.Iterables; +import com.google.common.collect.Lists; -import java.util.ArrayList; -import java.util.Collections; -import java.util.Iterator; +import java.io.Serializable; +import java.time.Instant; +import java.util.Collection; +import java.util.LinkedList; import java.util.List; +import org.apache.beam.runners.gearpump.translators.utils.TranslatorUtils; import org.apache.beam.sdk.transforms.GroupByKey; 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.WindowedValue; import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.PCollection; import org.apache.gearpump.streaming.dsl.javaapi.JavaStream; -import org.apache.gearpump.streaming.javaapi.dsl.functions.FlatMapFunction; +import org.apache.gearpump.streaming.dsl.window.api.Accumulating$; +import org.apache.gearpump.streaming.dsl.window.api.EventTimeTrigger$; +import org.apache.gearpump.streaming.dsl.window.api.Window; +import org.apache.gearpump.streaming.dsl.window.api.WindowFn; +import org.apache.gearpump.streaming.dsl.window.impl.Bucket; import org.apache.gearpump.streaming.javaapi.dsl.functions.GroupByFunction; import org.apache.gearpump.streaming.javaapi.dsl.functions.MapFunction; import org.apache.gearpump.streaming.javaapi.dsl.functions.ReduceFunction; - +import scala.collection.JavaConversions; /** @@ -44,56 +54,97 @@ public class GroupByKeyTranslator implements TransformTranslator> { @Override public void translate(GroupByKey transform, TranslationContext context) { + PCollection> input = context.getInput(transform); JavaStream>> inputStream = - context.getInputStream(context.getInput(transform)); + context.getInputStream(input); int parallelism = context.getPipelineOptions().getParallelism(); JavaStream>>> outputStream = inputStream - .flatMap(new KeyedByKeyAndWindow(), "keyed_by_Key_and_Window") - .groupBy(new GroupByKeyAndWindow(), parallelism, "group_by_Key_and_Window") - .map(new ExtractKeyValue(), "extract_Key_and_Value") + .window(Window.apply(new GearpumpWindowFn(input.getWindowingStrategy().getWindowFn()), + EventTimeTrigger$.MODULE$, Accumulating$.MODULE$), "assign_window") + .groupBy(new GroupByFn(), parallelism, "group_by_Key_and_Window") + .map(new ValueToIterable(), "map_value_to_iterable") .reduce(new MergeValue(), "merge_value"); context.setOutputStream(context.getOutput(transform), outputStream); } - private static class KeyedByKeyAndWindow implements - FlatMapFunction>, WindowedValue, V>>> { + private static class GearpumpWindowFn implements WindowFn, + Serializable { + + private org.apache.beam.sdk.transforms.windowing.WindowFn windowFn; + + GearpumpWindowFn(org.apache.beam.sdk.transforms.windowing.WindowFn windowFn) { + this.windowFn = windowFn; + } @Override - public Iterator, V>>> apply(WindowedValue> wv) { - List, V>>> ret = new ArrayList<>(wv.getWindows().size - ()); - for (BoundedWindow window : wv.getWindows()) { - KV keyWin = KV.of(wv.getValue().getKey(), window); - ret.add(WindowedValue.of(KV.of(keyWin, wv.getValue().getValue()), - wv.getTimestamp(), window, wv.getPane())); + public scala.collection.immutable.List apply(final Instant timestamp) { + try { + Collection windows = windowFn.assignWindows(windowFn.new AssignContext() { + @Override + public T element() { + throw new UnsupportedOperationException(); + } + + @Override + public org.joda.time.Instant timestamp() { + return TranslatorUtils.java8TimeToJodaTime(timestamp); + } + + @Override + public W window() { + throw new UnsupportedOperationException(); + } + }); + + List buckets = new LinkedList<>(); + for (BoundedWindow window : windows) { + buckets.add(getBucket(window)); + } + return JavaConversions.asScalaBuffer(buckets).toList(); + } catch (Exception e) { + throw new RuntimeException(e); + } + } + + private Bucket getBucket(BoundedWindow window) { + if (window instanceof IntervalWindow) { + IntervalWindow intervalWindow = (IntervalWindow) window; + Instant start = TranslatorUtils.jodaTimeToJava8Time(intervalWindow.start()); + Instant end = TranslatorUtils.jodaTimeToJava8Time(intervalWindow.end()); + return new Bucket(start, end); + } else if (window instanceof GlobalWindow) { + Instant end = TranslatorUtils.jodaTimeToJava8Time(window.maxTimestamp()); + return new Bucket(Instant.MIN, end); + } else { + throw new RuntimeException("unknown window " + window.getClass().getName()); } - return ret.iterator(); } } - private static class GroupByKeyAndWindow implements - GroupByFunction, V>>, KV> { + private static class GroupByFn implements + GroupByFunction>, K> { @Override - public KV apply(WindowedValue, V>> wv) { + public K apply(WindowedValue> wv) { return wv.getValue().getKey(); } } - private static class ExtractKeyValue implements - MapFunction, V>>, - WindowedValue>>> { + private static class ValueToIterable + implements MapFunction>, WindowedValue>>> { + + @Override - public WindowedValue>> apply(WindowedValue, V>> wv) { - return WindowedValue.of(KV.of(wv.getValue().getKey().getKey(), - (Iterable) Collections.singletonList(wv.getValue().getValue())), - wv.getTimestamp(), wv.getWindows(), wv.getPane()); + public WindowedValue>> apply(WindowedValue> wv) { + Iterable values = Lists.newArrayList(wv.getValue().getValue()); + return wv.withValue(KV.of(wv.getValue().getKey(), values)); } } private static class MergeValue implements ReduceFunction>>> { + @Override public WindowedValue>> apply(WindowedValue>> wv1, WindowedValue>> wv2) { diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/io/GearpumpSource.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/io/GearpumpSource.java index 8f2beb20195fa..b266590a66c0e 100644 --- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/io/GearpumpSource.java +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/io/GearpumpSource.java @@ -22,8 +22,11 @@ import com.fasterxml.jackson.databind.ObjectMapper; import java.io.IOException; +import java.time.Instant; +import org.apache.beam.runners.gearpump.translators.utils.TranslatorUtils; 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.transforms.windowing.GlobalWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; @@ -33,19 +36,17 @@ import org.apache.gearpump.streaming.source.DataSource; import org.apache.gearpump.streaming.task.TaskContext; -import org.joda.time.Instant; - /** * common methods for {@link BoundedSourceWrapper} and {@link UnboundedSourceWrapper}. */ public abstract class GearpumpSource implements DataSource { - protected final byte[] serializedOptions; + private final byte[] serializedOptions; - protected Source.Reader reader; - protected boolean available = false; + private Source.Reader reader; + private boolean available = false; - public GearpumpSource(PipelineOptions options) { + GearpumpSource(PipelineOptions options) { try { this.serializedOptions = new ObjectMapper().writeValueAsBytes(options); } catch (JsonProcessingException e) { @@ -56,7 +57,7 @@ public GearpumpSource(PipelineOptions options) { protected abstract Source.Reader createReader(PipelineOptions options) throws IOException; @Override - public void open(TaskContext context, long startTime) { + public void open(TaskContext context, Instant startTime) { try { PipelineOptions options = new ObjectMapper() .readValue(serializedOptions, PipelineOptions.class); @@ -68,13 +69,14 @@ public void open(TaskContext context, long startTime) { close(); } } + @Override public Message read() { Message message = null; try { if (available) { T data = reader.getCurrent(); - Instant timestamp = reader.getCurrentTimestamp(); + org.joda.time.Instant timestamp = reader.getCurrentTimestamp(); available = reader.advance(); message = Message.apply( WindowedValue.of(data, timestamp, GlobalWindow.INSTANCE, PaneInfo.NO_FIRING), @@ -99,4 +101,14 @@ public void close() { } } + @Override + public Instant getWatermark() { + if (reader instanceof UnboundedSource.UnboundedReader) { + return TranslatorUtils.jodaTimeToJava8Time( + ((UnboundedSource.UnboundedReader) reader).getWatermark()); + } else { + return Instant.now(); + } + } + } diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/io/ValuesSource.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/io/ValuesSource.java index 3b67f09036446..f5a5eb459a711 100644 --- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/io/ValuesSource.java +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/io/ValuesSource.java @@ -32,6 +32,7 @@ import org.apache.beam.sdk.io.UnboundedSource; import org.apache.beam.sdk.options.PipelineOptions; +import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.joda.time.Instant; /** @@ -48,23 +49,19 @@ public ValuesSource(Iterable values, Coder coder) { } private byte[] encode(Iterable values, IterableCoder coder) { - ByteArrayOutputStream stream = new ByteArrayOutputStream(); - try { + try (ByteArrayOutputStream stream = new ByteArrayOutputStream()) { coder.encode(values, stream, Coder.Context.OUTER); + return stream.toByteArray(); } catch (IOException ex) { throw new RuntimeException(ex); } - return stream.toByteArray(); } private Iterable decode(byte[] bytes) throws IOException{ - ByteArrayInputStream inputStream = new ByteArrayInputStream(bytes); - try { + try (ByteArrayInputStream inputStream = new ByteArrayInputStream(bytes)) { return iterableCoder.decode(inputStream, Coder.Context.OUTER); } catch (IOException ex) { throw new RuntimeException(ex); - } finally { - inputStream.close(); } } @@ -78,7 +75,7 @@ public java.util.List> generateInit public UnboundedReader createReader(PipelineOptions options, @Nullable CheckpointMark checkpointMark) { try { - return new ValuesReader<>(decode(values), iterableCoder, this); + return new ValuesReader<>(decode(values), this); } catch (IOException e) { throw new RuntimeException(e); } @@ -105,7 +102,7 @@ private static class ValuesReader extends UnboundedReader { private transient Iterator iterator; private T current; - public ValuesReader(Iterable values, IterableCoder coder, + ValuesReader(Iterable values, UnboundedSource source) { this.values = values; this.source = source; @@ -147,7 +144,11 @@ public void close() throws IOException { @Override public Instant getWatermark() { - return Instant.now(); + if (iterator.hasNext()) { + return Instant.now(); + } else { + return BoundedWindow.TIMESTAMP_MAX_VALUE; + } } @Override diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/utils/TranslatorUtils.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/utils/TranslatorUtils.java new file mode 100644 index 0000000000000..9b722754a4814 --- /dev/null +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/utils/TranslatorUtils.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.gearpump.translators.utils; + +import java.time.Instant; + +/** + * Utility methods for translators. + */ +public class TranslatorUtils { + + public static Instant jodaTimeToJava8Time(org.joda.time.Instant time) { + return Instant.ofEpochMilli(time.getMillis()); + } + + public static org.joda.time.Instant java8TimeToJodaTime(Instant time) { + return new org.joda.time.Instant(time.toEpochMilli()); + } +} From 85d54ab20f21297da25059ed7b4c8ed02e93bb74 Mon Sep 17 00:00:00 2001 From: manuzhang Date: Fri, 16 Dec 2016 16:49:06 +0800 Subject: [PATCH 127/346] Add Window.Bound translator --- .../gearpump/GearpumpPipelineTranslator.java | 3 + .../translators/WindowBoundTranslator.java | 97 +++++++++++++++++++ 2 files changed, 100 insertions(+) create mode 100644 runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/WindowBoundTranslator.java diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpPipelineTranslator.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpPipelineTranslator.java index 84dfeec529e73..20624ed382644 100644 --- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpPipelineTranslator.java +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpPipelineTranslator.java @@ -29,6 +29,7 @@ import org.apache.beam.runners.gearpump.translators.ReadUnboundedTranslator; import org.apache.beam.runners.gearpump.translators.TransformTranslator; import org.apache.beam.runners.gearpump.translators.TranslationContext; +import org.apache.beam.runners.gearpump.translators.WindowBoundTranslator; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.io.Read; import org.apache.beam.sdk.runners.TransformHierarchy; @@ -37,6 +38,7 @@ 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.windowing.Window; import org.apache.beam.sdk.values.PValue; import org.apache.gearpump.util.Graph; @@ -71,6 +73,7 @@ public class GearpumpPipelineTranslator extends Pipeline.PipelineVisitor.Default registerTransformTranslator(Flatten.FlattenPCollectionList.class, new FlattenPCollectionTranslator()); registerTransformTranslator(ParDo.BoundMulti.class, new ParDoBoundMultiTranslator()); + registerTransformTranslator(Window.Bound.class, new WindowBoundTranslator()); registerTransformTranslator(Create.Values.class, new CreateValuesTranslator()); } diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/WindowBoundTranslator.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/WindowBoundTranslator.java new file mode 100644 index 0000000000000..11f30fcae3e95 --- /dev/null +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/WindowBoundTranslator.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.gearpump.translators; + +import com.google.common.collect.Iterables; + +import java.util.Collection; +import java.util.Iterator; +import java.util.LinkedList; +import java.util.List; + +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.WindowedValue; +import org.apache.beam.sdk.util.WindowingStrategy; +import org.apache.beam.sdk.values.PCollection; +import org.apache.gearpump.streaming.dsl.javaapi.JavaStream; +import org.apache.gearpump.streaming.javaapi.dsl.functions.FlatMapFunction; +import org.joda.time.Instant; + +/** + * {@link Window.Bound} is translated to Gearpump flatMap function. + */ +@SuppressWarnings("unchecked") +public class WindowBoundTranslator implements TransformTranslator> { + + @Override + public void translate(Window.Bound transform, TranslationContext context) { + PCollection input = context.getInput(transform); + JavaStream> inputStream = context.getInputStream(input); + WindowingStrategy outputStrategy = + transform.getOutputStrategyInternal(input.getWindowingStrategy()); + WindowFn windowFn = + (WindowFn) outputStrategy.getWindowFn(); + JavaStream> outputStream = + inputStream.flatMap(new AssignWindows(windowFn), "assign_windows"); + context.setOutputStream(context.getOutput(transform), outputStream); + } + + + private static class AssignWindows implements + FlatMapFunction, WindowedValue> { + + private final WindowFn fn; + + AssignWindows(WindowFn fn) { + this.fn = fn; + } + + @Override + public Iterator> apply(final WindowedValue value) { + List> ret = new LinkedList<>(); + try { + Collection windows = fn.assignWindows(fn.new AssignContext() { + @Override + public T element() { + return value.getValue(); + } + + @Override + public Instant timestamp() { + return value.getTimestamp(); + } + + @Override + public BoundedWindow window() { + return Iterables.getOnlyElement(value.getWindows()); + } + }); + for (BoundedWindow window: windows) { + ret.add(WindowedValue.of( + value.getValue(), value.getTimestamp(), window, value.getPane())); + } + } catch (Exception e) { + throw new RuntimeException(e); + } + return ret.iterator(); + } + } +} From c37de00266b35ca6f04ddbe7b564f9054b26b622 Mon Sep 17 00:00:00 2001 From: manuzhang Date: Fri, 16 Dec 2016 16:49:38 +0800 Subject: [PATCH 128/346] Skip window assignment when windows don't change --- .../beam/runners/gearpump/GearpumpRunner.java | 60 ++++--------------- 1 file changed, 11 insertions(+), 49 deletions(-) diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpRunner.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpRunner.java index ed0813d68d619..84778705f682e 100644 --- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpRunner.java +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpRunner.java @@ -23,20 +23,19 @@ import java.util.HashMap; import java.util.Map; -import org.apache.beam.runners.core.AssignWindows; import org.apache.beam.runners.gearpump.translators.TranslationContext; import org.apache.beam.sdk.Pipeline; 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.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.PTransform; import org.apache.beam.sdk.transforms.ParDo; -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.IdentityWindowFn; import org.apache.beam.sdk.util.WindowingStrategy; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionList; @@ -75,9 +74,10 @@ public static GearpumpRunner fromOptions(PipelineOptions options) { public OutputT apply( PTransform transform, InputT input) { - if (Window.Bound.class.equals(transform.getClass())) { + if (Window.Bound.class.equals(transform.getClass()) + && isNullOrIdentityWindowFn(((Window.Bound) transform).getWindowFn())) { return (OutputT) super.apply( - new AssignWindowsAndSetStrategy((Window.Bound) transform), input); + ParDo.of(new IdentityFn()), input); } else if (Flatten.FlattenPCollectionList.class.equals(transform.getClass()) && ((PCollectionList) input).size() == 0) { return (OutputT) Pipeline.applyTransform(input.getPipeline().begin(), Create.of()); @@ -139,53 +139,15 @@ private Config registerSerializers(Config config, Map userSerial return config.withValue(GEARPUMP_SERIALIZERS, ConfigValueFactory.fromMap(serializers)); } + private static class IdentityFn extends DoFn { - /** - * copied from DirectPipelineRunner. - * used to replace Window.Bound till window function is added to Gearpump Stream DSL - */ - private static class AssignWindowsAndSetStrategy - extends PTransform, PCollection> { - - private final Window.Bound wrapped; - - AssignWindowsAndSetStrategy(Window.Bound wrapped) { - this.wrapped = wrapped; - } - - @Override - public PCollection apply(PCollection input) { - WindowingStrategy outputStrategy = - wrapped.getOutputStrategyInternal(input.getWindowingStrategy()); - - WindowFn windowFn = - (WindowFn) outputStrategy.getWindowFn(); - - if (!windowFn.isNonMerging()) { - throw new UnsupportedOperationException( - "merging window is not supported in Gearpump pipeline"); - } - - // 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); - } + @ProcessElement + public void process(ProcessContext c) { + c.output(c.element()); } } - private static class IdentityFn extends OldDoFn { - @Override - public void processElement(ProcessContext c) { - c.output(c.element()); - } + private boolean isNullOrIdentityWindowFn(WindowFn windowFn) { + return windowFn == null || windowFn.getClass().equals(IdentityWindowFn.class); } } From cb8c5e5e59348897e0bc04d5ffb885c15a18c6af Mon Sep 17 00:00:00 2001 From: manuzhang Date: Fri, 16 Dec 2016 16:50:08 +0800 Subject: [PATCH 129/346] Remove print to stdout --- .../gearpump/translators/FlattenPCollectionTranslator.java | 1 - 1 file changed, 1 deletion(-) diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/FlattenPCollectionTranslator.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/FlattenPCollectionTranslator.java index b06d5a840817e..b740ab5c6cacf 100644 --- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/FlattenPCollectionTranslator.java +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/FlattenPCollectionTranslator.java @@ -33,7 +33,6 @@ public class FlattenPCollectionTranslator implements @Override public void translate(Flatten.FlattenPCollectionList transform, TranslationContext context) { JavaStream merged = null; - System.out.println("PCollectionList size " + context.getInput(transform).size()); for (PCollection collection : context.getInput(transform).getAll()) { JavaStream inputStream = context.getInputStream(collection); if (null == merged) { From 8e0e81999c118b7d24402088a0a63601c967b3c0 Mon Sep 17 00:00:00 2001 From: manuzhang Date: Fri, 16 Dec 2016 16:50:30 +0800 Subject: [PATCH 130/346] Fix NoOpAggregatorFactory --- .../utils/NoOpAggregatorFactory.java | 24 ++++++++++++++++++- 1 file changed, 23 insertions(+), 1 deletion(-) diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/utils/NoOpAggregatorFactory.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/utils/NoOpAggregatorFactory.java index cd404a53af7ab..bfc73bf62e029 100644 --- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/utils/NoOpAggregatorFactory.java +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/utils/NoOpAggregatorFactory.java @@ -36,6 +36,28 @@ public Aggregator createAggregatorFor ExecutionContext.StepContext stepContext, String aggregatorName, Combine.CombineFn combine) { - return null; + return new NoOpAggregator<>(); } + + private static class NoOpAggregator implements Aggregator, + java.io.Serializable { + private static final long serialVersionUID = 1L; + + @Override + public void addValue(InputT value) { + } + + @Override + public String getName() { + // TODO Auto-generated method stub + return null; + } + + @Override + public Combine.CombineFn getCombineFn() { + // TODO Auto-generated method stub + return null; + } + + }; } From c2fb7c077243a51d821886e56bb21ceb78c792d2 Mon Sep 17 00:00:00 2001 From: manuzhang Date: Tue, 20 Dec 2016 15:57:21 +0800 Subject: [PATCH 131/346] [BEAM-79] Update to latest Gearpump API --- .../java/org/apache/beam/runners/gearpump/GearpumpRunner.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpRunner.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpRunner.java index 84778705f682e..4083922a72c72 100644 --- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpRunner.java +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpRunner.java @@ -107,7 +107,7 @@ public GearpumpPipelineResult run(Pipeline pipeline) { TranslationContext translationContext = new TranslationContext(streamApp, options); GearpumpPipelineTranslator translator = new GearpumpPipelineTranslator(translationContext); translator.translate(pipeline); - streamApp.run(); + streamApp.submit(); return null; } From 647034cfc6ee419548b6da222e6d134792366a26 Mon Sep 17 00:00:00 2001 From: manuzhang Date: Wed, 21 Dec 2016 09:32:35 +0800 Subject: [PATCH 132/346] [BEAM-79] Upgrade to beam-0.5.0-incubating-SNAPSHOT --- runners/gearpump/pom.xml | 2 +- .../gearpump/examples/StreamingWordCount.java | 14 +++++++------- .../translators/ParDoBoundMultiTranslator.java | 2 +- .../gearpump/translators/ParDoBoundTranslator.java | 2 +- .../gearpump/translators/TranslationContext.java | 3 +-- .../translators/utils/DoFnRunnerFactory.java | 2 +- .../translators/utils/NoOpAggregatorFactory.java | 2 +- 7 files changed, 13 insertions(+), 14 deletions(-) diff --git a/runners/gearpump/pom.xml b/runners/gearpump/pom.xml index 9320561fc7c64..bb35ad71e6f3d 100644 --- a/runners/gearpump/pom.xml +++ b/runners/gearpump/pom.xml @@ -23,7 +23,7 @@ org.apache.beam beam-runners-parent - 0.4.0-incubating-SNAPSHOT + 0.5.0-incubating-SNAPSHOT ../pom.xml diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/examples/StreamingWordCount.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/examples/StreamingWordCount.java index 1d85c2532bab0..b2d762a200eef 100644 --- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/examples/StreamingWordCount.java +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/examples/StreamingWordCount.java @@ -24,7 +24,7 @@ import org.apache.beam.sdk.io.Read; 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.Window; @@ -42,10 +42,10 @@ */ public class StreamingWordCount { - static class ExtractWordsFn extends OldDoFn { + static class ExtractWordsFn extends DoFn { - @Override - public void processElement(ProcessContext c) { + @ProcessElement + public void process(ProcessContext c) { // Split the line into words. String[] words = c.element().split("[^a-zA-Z']+"); @@ -58,11 +58,11 @@ public void processElement(ProcessContext c) { } } - static class FormatAsStringFn extends OldDoFn, String> { + static class FormatAsStringFn extends DoFn, String> { private static final Logger LOG = LoggerFactory.getLogger(FormatAsStringFn.class); - @Override - public void processElement(ProcessContext c) { + @ProcessElement + public void process(ProcessContext c) { String row = c.element().getKey() + " - " + c.element().getValue() + " @ " + c.timestamp().toString(); diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/ParDoBoundMultiTranslator.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/ParDoBoundMultiTranslator.java index 54f1c3f29f349..24f973421ebb0 100644 --- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/ParDoBoundMultiTranslator.java +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/ParDoBoundMultiTranslator.java @@ -64,7 +64,7 @@ public void translate(ParDo.BoundMulti transform, TranslationCo JavaStream, OutputT>>> outputStream = inputStream.flatMap( new DoFnMultiFunction<>( context.getPipelineOptions(), - transform.getNewFn(), + transform.getFn(), transform.getMainOutputTag(), transform.getSideOutputTags(), inputT.getWindowingStrategy(), diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/ParDoBoundTranslator.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/ParDoBoundTranslator.java index a796c833ea4ef..689bc08e0f47e 100644 --- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/ParDoBoundTranslator.java +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/ParDoBoundTranslator.java @@ -38,7 +38,7 @@ public class ParDoBoundTranslator implements @Override public void translate(ParDo.Bound transform, TranslationContext context) { - DoFn doFn = transform.getNewFn(); + DoFn doFn = transform.getFn(); PCollection output = context.getOutput(transform); WindowingStrategy windowingStrategy = output.getWindowingStrategy(); diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/TranslationContext.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/TranslationContext.java index d9d6a8e74fc27..63fb619ce0c1a 100644 --- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/TranslationContext.java +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/TranslationContext.java @@ -54,8 +54,7 @@ public TranslationContext(JavaStreamApp streamApp, GearpumpPipelineOptions pipel } public void setCurrentTransform(TransformHierarchy.Node treeNode) { - this.currentTransform = AppliedPTransform.of(treeNode.getFullName(), - treeNode.getInput(), treeNode.getOutput(), (PTransform) treeNode.getTransform()); + this.currentTransform = treeNode.toAppliedPTransform(); } public GearpumpPipelineOptions getPipelineOptions() { diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/utils/DoFnRunnerFactory.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/utils/DoFnRunnerFactory.java index 7119a87033b66..7e1402f4ebeb8 100644 --- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/utils/DoFnRunnerFactory.java +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/utils/DoFnRunnerFactory.java @@ -21,12 +21,12 @@ import java.io.Serializable; import java.util.List; +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.SimpleDoFnRunner; import org.apache.beam.runners.gearpump.GearpumpPipelineOptions; import org.apache.beam.sdk.options.PipelineOptions; -import org.apache.beam.sdk.transforms.Aggregator.AggregatorFactory; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.util.ExecutionContext; import org.apache.beam.sdk.util.SideInputReader; diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/utils/NoOpAggregatorFactory.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/utils/NoOpAggregatorFactory.java index bfc73bf62e029..22ffc4d926b09 100644 --- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/utils/NoOpAggregatorFactory.java +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/utils/NoOpAggregatorFactory.java @@ -20,8 +20,8 @@ import java.io.Serializable; +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; import org.apache.beam.sdk.util.ExecutionContext; From 21554764056c45ea18be1e844b4ca1bfb71e544a Mon Sep 17 00:00:00 2001 From: manuzhang Date: Tue, 20 Dec 2016 10:39:56 +0800 Subject: [PATCH 133/346] [BEAM-1180] Implement GearpumpPipelineResult --- runners/gearpump/pom.xml | 1 + .../gearpump/GearpumpPipelineResult.java | 59 +++++++++++++++++-- .../beam/runners/gearpump/GearpumpRunner.java | 4 +- .../runners/gearpump/TestGearpumpRunner.java | 4 ++ .../translators/GroupByKeyTranslator.java | 1 - 5 files changed, 62 insertions(+), 7 deletions(-) diff --git a/runners/gearpump/pom.xml b/runners/gearpump/pom.xml index bb35ad71e6f3d..777ad3490dfd9 100644 --- a/runners/gearpump/pom.xml +++ b/runners/gearpump/pom.xml @@ -69,6 +69,7 @@ org.apache.beam:beam-sdks-java-core + -noverify diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpPipelineResult.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpPipelineResult.java index ed1201ddcc6bb..9c8f7b345f45c 100644 --- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpPipelineResult.java +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpPipelineResult.java @@ -18,6 +18,7 @@ package org.apache.beam.runners.gearpump; import java.io.IOException; +import java.util.List; import org.apache.beam.sdk.AggregatorRetrievalException; import org.apache.beam.sdk.AggregatorValues; @@ -26,31 +27,62 @@ import org.apache.beam.sdk.metrics.MetricResults; import org.apache.beam.sdk.transforms.Aggregator; +import org.apache.gearpump.cluster.MasterToAppMaster; +import org.apache.gearpump.cluster.MasterToAppMaster.AppMasterData; +import org.apache.gearpump.cluster.client.ClientContext; import org.joda.time.Duration; +import scala.collection.JavaConverters; +import scala.collection.Seq; /** * Result of executing a {@link Pipeline} with Gearpump. */ public class GearpumpPipelineResult implements PipelineResult { + + private final ClientContext client; + private final int appId; + private final Duration defaultWaitDuration = Duration.standardSeconds(60); + private final Duration defaultWaitInterval = Duration.standardSeconds(10); + + public GearpumpPipelineResult(ClientContext client, int appId) { + this.client = client; + this.appId = appId; + } + @Override public State getState() { - return null; + return getGearpumpState(); } @Override public State cancel() throws IOException { - return null; + client.shutdown(appId); + return State.CANCELLED; } @Override public State waitUntilFinish(Duration duration) { - return null; + long start = System.currentTimeMillis(); + do { + try { + Thread.sleep(defaultWaitInterval.getMillis()); + } catch (InterruptedException e) { + throw new RuntimeException(e); + } + } while (State.RUNNING == getGearpumpState() + && (System.currentTimeMillis() - start) < duration.getMillis()); + + if (State.RUNNING == getGearpumpState()) { + return State.DONE; + } else { + return State.FAILED; + } } @Override public State waitUntilFinish() { - return null; + return waitUntilFinish(defaultWaitDuration); } @Override @@ -66,4 +98,23 @@ public MetricResults metrics() { return null; } + private State getGearpumpState() { + String status = null; + List apps = + JavaConverters.seqAsJavaListConverter( + (Seq) client.listApps().appMasters()).asJava(); + for (AppMasterData app: apps) { + if (app.appId() == appId) { + status = app.status(); + } + } + if (null == status || status.equals(MasterToAppMaster.AppMasterNonExist())) { + return State.UNKNOWN; + } else if (status.equals(MasterToAppMaster.AppMasterActive())) { + return State.RUNNING; + } else { + return State.STOPPED; + } + } + } diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpRunner.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpRunner.java index 4083922a72c72..9c44da3d1cb1e 100644 --- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpRunner.java +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpRunner.java @@ -107,9 +107,9 @@ public GearpumpPipelineResult run(Pipeline pipeline) { TranslationContext translationContext = new TranslationContext(streamApp, options); GearpumpPipelineTranslator translator = new GearpumpPipelineTranslator(translationContext); translator.translate(pipeline); - streamApp.submit(); + int appId = streamApp.submit(); - return null; + return new GearpumpPipelineResult(clientContext, appId); } private ClientContext getClientContext(GearpumpPipelineOptions options, Config config) { diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/TestGearpumpRunner.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/TestGearpumpRunner.java index 89d31a6d8eb37..ee31fb509d0a8 100644 --- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/TestGearpumpRunner.java +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/TestGearpumpRunner.java @@ -19,6 +19,7 @@ package org.apache.beam.runners.gearpump; import org.apache.beam.sdk.Pipeline; +import org.apache.beam.sdk.PipelineResult; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.options.PipelineOptionsValidator; import org.apache.beam.sdk.runners.PipelineRunner; @@ -52,7 +53,10 @@ public static TestGearpumpRunner fromOptions(PipelineOptions options) { @Override public GearpumpPipelineResult run(Pipeline pipeline) { GearpumpPipelineResult result = delegate.run(pipeline); + PipelineResult.State state = result.waitUntilFinish(); cluster.stop(); + assert(state == PipelineResult.State.DONE); + return result; } diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/GroupByKeyTranslator.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/GroupByKeyTranslator.java index d64f1bf601037..989957fd5003c 100644 --- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/GroupByKeyTranslator.java +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/GroupByKeyTranslator.java @@ -134,7 +134,6 @@ public K apply(WindowedValue> wv) { private static class ValueToIterable implements MapFunction>, WindowedValue>>> { - @Override public WindowedValue>> apply(WindowedValue> wv) { Iterable values = Lists.newArrayList(wv.getValue().getValue()); From cfdc971f45ff716b7bd88b3e054ca7077454ab07 Mon Sep 17 00:00:00 2001 From: manuzhang Date: Thu, 5 Jan 2017 13:47:42 +0800 Subject: [PATCH 134/346] update ROS configurations --- runners/gearpump/pom.xml | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/runners/gearpump/pom.xml b/runners/gearpump/pom.xml index 777ad3490dfd9..4e3722c176209 100644 --- a/runners/gearpump/pom.xml +++ b/runners/gearpump/pom.xml @@ -64,6 +64,12 @@ org.apache.beam.sdk.testing.RunnableOnService + + org.apache.beam.sdk.testing.UsesStatefulParDo, + org.apache.beam.sdk.testing.UsesTimersInParDo, + org.apache.beam.sdk.testing.UsesSplittableParDo, + org.apache.beam.sdk.testing.UsesMetrics + none true From ea633d2c02a621df09c5f4a6d0ab3824271c7db2 Mon Sep 17 00:00:00 2001 From: manuzhang Date: Sat, 7 Jan 2017 10:47:03 +0800 Subject: [PATCH 135/346] activate ROS on Gearpump by default --- runners/gearpump/pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/runners/gearpump/pom.xml b/runners/gearpump/pom.xml index 4e3722c176209..7c6fa7633dbce 100644 --- a/runners/gearpump/pom.xml +++ b/runners/gearpump/pom.xml @@ -49,7 +49,7 @@ local-runnable-on-service-tests - false + true From e63d42d1113728badc66285e7ce7a8ce204a82d9 Mon Sep 17 00:00:00 2001 From: manuzhang Date: Sat, 7 Jan 2017 23:07:23 +0800 Subject: [PATCH 136/346] fix group by window --- .../beam/runners/gearpump/GearpumpRunner.java | 3 ++- .../translators/GroupByKeyTranslator.java | 4 +-- .../translators/TranslationContext.java | 1 - .../translators/WindowBoundTranslator.java | 27 +++++++++++++++++-- .../translators/io/GearpumpSource.java | 4 +-- 5 files changed, 30 insertions(+), 9 deletions(-) diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpRunner.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpRunner.java index 9c44da3d1cb1e..01fdb3b2e0982 100644 --- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpRunner.java +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpRunner.java @@ -102,8 +102,9 @@ public GearpumpPipelineResult run(Pipeline pipeline) { options.getSerializers()); ClientContext clientContext = getClientContext(options, config); options.setClientContext(clientContext); + UserConfig userConfig = UserConfig.empty(); JavaStreamApp streamApp = new JavaStreamApp( - appName, clientContext, UserConfig.empty()); + appName, clientContext, userConfig); TranslationContext translationContext = new TranslationContext(streamApp, options); GearpumpPipelineTranslator translator = new GearpumpPipelineTranslator(translationContext); translator.translate(pipeline); diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/GroupByKeyTranslator.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/GroupByKeyTranslator.java index 989957fd5003c..8e3ffe35818f5 100644 --- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/GroupByKeyTranslator.java +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/GroupByKeyTranslator.java @@ -37,7 +37,7 @@ import org.apache.beam.sdk.values.PCollection; import org.apache.gearpump.streaming.dsl.javaapi.JavaStream; -import org.apache.gearpump.streaming.dsl.window.api.Accumulating$; +import org.apache.gearpump.streaming.dsl.window.api.Discarding$; import org.apache.gearpump.streaming.dsl.window.api.EventTimeTrigger$; import org.apache.gearpump.streaming.dsl.window.api.Window; import org.apache.gearpump.streaming.dsl.window.api.WindowFn; @@ -60,7 +60,7 @@ public void translate(GroupByKey transform, TranslationContext context) { int parallelism = context.getPipelineOptions().getParallelism(); JavaStream>>> outputStream = inputStream .window(Window.apply(new GearpumpWindowFn(input.getWindowingStrategy().getWindowFn()), - EventTimeTrigger$.MODULE$, Accumulating$.MODULE$), "assign_window") + EventTimeTrigger$.MODULE$, Discarding$.MODULE$), "assign_window") .groupBy(new GroupByFn(), parallelism, "group_by_Key_and_Window") .map(new ValueToIterable(), "map_value_to_iterable") .reduce(new MergeValue(), "merge_value"); diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/TranslationContext.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/TranslationContext.java index 63fb619ce0c1a..b2cff8a9b184b 100644 --- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/TranslationContext.java +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/TranslationContext.java @@ -50,7 +50,6 @@ public class TranslationContext { public TranslationContext(JavaStreamApp streamApp, GearpumpPipelineOptions pipelineOptions) { this.streamApp = streamApp; this.pipelineOptions = pipelineOptions; - } public void setCurrentTransform(TransformHierarchy.Node treeNode) { diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/WindowBoundTranslator.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/WindowBoundTranslator.java index 11f30fcae3e95..32dd5de807e4f 100644 --- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/WindowBoundTranslator.java +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/WindowBoundTranslator.java @@ -31,8 +31,12 @@ import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.WindowingStrategy; import org.apache.beam.sdk.values.PCollection; +import org.apache.gearpump.Message; +import org.apache.gearpump.cluster.UserConfig; import org.apache.gearpump.streaming.dsl.javaapi.JavaStream; +import org.apache.gearpump.streaming.javaapi.Task; import org.apache.gearpump.streaming.javaapi.dsl.functions.FlatMapFunction; +import org.apache.gearpump.streaming.task.TaskContext; import org.joda.time.Instant; /** @@ -50,11 +54,13 @@ public void translate(Window.Bound transform, TranslationContext context) { WindowFn windowFn = (WindowFn) outputStrategy.getWindowFn(); JavaStream> outputStream = - inputStream.flatMap(new AssignWindows(windowFn), "assign_windows"); + inputStream + .flatMap(new AssignWindows(windowFn), "assign_windows") + .process(AssignTimestampTask.class, 1, UserConfig.empty(), "assign_timestamp"); + context.setOutputStream(context.getOutput(transform), outputStream); } - private static class AssignWindows implements FlatMapFunction, WindowedValue> { @@ -94,4 +100,21 @@ public BoundedWindow window() { return ret.iterator(); } } + + /** + * Assign WindowedValue timestamp to Gearpump message. + * @param element type of WindowedValue + */ + public static class AssignTimestampTask extends Task { + + public AssignTimestampTask(TaskContext taskContext, UserConfig userConfig) { + super(taskContext, userConfig); + } + + @Override + public void onNext(Message message) { + final WindowedValue value = (WindowedValue) message.msg(); + context.output(Message.apply(value, value.getTimestamp().getMillis())); + } + } } diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/io/GearpumpSource.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/io/GearpumpSource.java index b266590a66c0e..6e5b2defd77d5 100644 --- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/io/GearpumpSource.java +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/io/GearpumpSource.java @@ -28,8 +28,6 @@ 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.transforms.windowing.GlobalWindow; -import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.util.WindowedValue; import org.apache.gearpump.Message; @@ -79,7 +77,7 @@ public Message read() { org.joda.time.Instant timestamp = reader.getCurrentTimestamp(); available = reader.advance(); message = Message.apply( - WindowedValue.of(data, timestamp, GlobalWindow.INSTANCE, PaneInfo.NO_FIRING), + WindowedValue.valueInGlobalWindow(data), timestamp.getMillis()); } } catch (Exception e) { From 3bf82638096ae7aa91c7d3c862c2994772bee51b Mon Sep 17 00:00:00 2001 From: manuzhang Date: Sat, 14 Jan 2017 13:36:07 +0800 Subject: [PATCH 137/346] update to latest gearpump dsl function interface --- .../translators/GroupByKeyTranslator.java | 12 ++++---- .../ParDoBoundMultiTranslator.java | 29 ++++++++++++++----- .../translators/WindowBoundTranslator.java | 4 +-- .../translators/functions/DoFnFunction.java | 21 +++++++++++--- 4 files changed, 46 insertions(+), 20 deletions(-) diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/GroupByKeyTranslator.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/GroupByKeyTranslator.java index 8e3ffe35818f5..4eaf7552fd44b 100644 --- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/GroupByKeyTranslator.java +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/GroupByKeyTranslator.java @@ -36,15 +36,15 @@ import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; +import org.apache.gearpump.streaming.dsl.api.functions.MapFunction; +import org.apache.gearpump.streaming.dsl.api.functions.ReduceFunction; import org.apache.gearpump.streaming.dsl.javaapi.JavaStream; +import org.apache.gearpump.streaming.dsl.javaapi.functions.GroupByFunction; import org.apache.gearpump.streaming.dsl.window.api.Discarding$; import org.apache.gearpump.streaming.dsl.window.api.EventTimeTrigger$; import org.apache.gearpump.streaming.dsl.window.api.Window; import org.apache.gearpump.streaming.dsl.window.api.WindowFn; import org.apache.gearpump.streaming.dsl.window.impl.Bucket; -import org.apache.gearpump.streaming.javaapi.dsl.functions.GroupByFunction; -import org.apache.gearpump.streaming.javaapi.dsl.functions.MapFunction; -import org.apache.gearpump.streaming.javaapi.dsl.functions.ReduceFunction; import scala.collection.JavaConversions; @@ -122,7 +122,7 @@ private Bucket getBucket(BoundedWindow window) { } } - private static class GroupByFn implements + private static class GroupByFn extends GroupByFunction>, K> { @Override @@ -132,7 +132,7 @@ public K apply(WindowedValue> wv) { } private static class ValueToIterable - implements MapFunction>, WindowedValue>>> { + extends MapFunction>, WindowedValue>>> { @Override public WindowedValue>> apply(WindowedValue> wv) { @@ -141,7 +141,7 @@ public WindowedValue>> apply(WindowedValue> wv) { } } - private static class MergeValue implements + private static class MergeValue extends ReduceFunction>>> { @Override diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/ParDoBoundMultiTranslator.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/ParDoBoundMultiTranslator.java index 24f973421ebb0..0d5b8bcccbf71 100644 --- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/ParDoBoundMultiTranslator.java +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/ParDoBoundMultiTranslator.java @@ -33,6 +33,7 @@ import org.apache.beam.runners.gearpump.translators.utils.NoOpStepContext; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.ParDo; +import org.apache.beam.sdk.transforms.reflect.DoFnInvokers; import org.apache.beam.sdk.util.SideInputReader; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.WindowingStrategy; @@ -41,10 +42,10 @@ import org.apache.beam.sdk.values.TupleTag; import org.apache.beam.sdk.values.TupleTagList; +import org.apache.gearpump.streaming.dsl.api.functions.FilterFunction; +import org.apache.gearpump.streaming.dsl.api.functions.MapFunction; import org.apache.gearpump.streaming.dsl.javaapi.JavaStream; -import org.apache.gearpump.streaming.javaapi.dsl.functions.FilterFunction; -import org.apache.gearpump.streaming.javaapi.dsl.functions.FlatMapFunction; -import org.apache.gearpump.streaming.javaapi.dsl.functions.MapFunction; +import org.apache.gearpump.streaming.dsl.javaapi.functions.FlatMapFunction; /** * {@link ParDo.BoundMulti} is translated to Gearpump flatMap function @@ -83,12 +84,13 @@ public void translate(ParDo.BoundMulti transform, TranslationCo /** * Gearpump {@link FlatMapFunction} wrapper over Beam {@link DoFnMultiFunction}. */ - private static class DoFnMultiFunction implements - FlatMapFunction, WindowedValue, OutputT>>>, - DoFnRunners.OutputManager { + private static class DoFnMultiFunction + extends FlatMapFunction, WindowedValue, OutputT>>> + implements DoFnRunners.OutputManager { private final DoFnRunnerFactory doFnRunnerFactory; private DoFnRunner doFnRunner; + private final DoFn doFn; private final List, OutputT>>> outputs = Lists .newArrayList(); @@ -99,6 +101,7 @@ public DoFnMultiFunction( TupleTagList sideOutputTags, WindowingStrategy windowingStrategy, SideInputReader sideInputReader) { + this.doFn = doFn; this.doFnRunnerFactory = new DoFnRunnerFactory<>( pipelineOptions, doFn, @@ -112,6 +115,16 @@ public DoFnMultiFunction( ); } + @Override + public void setup() { + DoFnInvokers.invokerFor(doFn).invokeSetup(); + } + + @Override + public void teardown() { + DoFnInvokers.invokerFor(doFn).invokeTeardown(); + } + @Override public Iterator, OutputT>>> apply(WindowedValue wv) { if (null == doFnRunner) { @@ -133,7 +146,7 @@ public void output(TupleTag tag, WindowedValue output) { } } - private static class FilterByOutputTag implements + private static class FilterByOutputTag extends FilterFunction, OutputT>>> { private final TupleTag tupleTag; @@ -148,7 +161,7 @@ public boolean apply(WindowedValue, OutputT>> wv) { } } - private static class ExtractOutput implements + private static class ExtractOutput extends MapFunction, OutputT>>, WindowedValue> { @Override diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/WindowBoundTranslator.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/WindowBoundTranslator.java index 32dd5de807e4f..d3c50a59cad7a 100644 --- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/WindowBoundTranslator.java +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/WindowBoundTranslator.java @@ -34,8 +34,8 @@ import org.apache.gearpump.Message; import org.apache.gearpump.cluster.UserConfig; import org.apache.gearpump.streaming.dsl.javaapi.JavaStream; +import org.apache.gearpump.streaming.dsl.javaapi.functions.FlatMapFunction; import org.apache.gearpump.streaming.javaapi.Task; -import org.apache.gearpump.streaming.javaapi.dsl.functions.FlatMapFunction; import org.apache.gearpump.streaming.task.TaskContext; import org.joda.time.Instant; @@ -61,7 +61,7 @@ public void translate(Window.Bound transform, TranslationContext context) { context.setOutputStream(context.getOutput(transform), outputStream); } - private static class AssignWindows implements + private static class AssignWindows extends FlatMapFunction, WindowedValue> { private final WindowFn fn; diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/functions/DoFnFunction.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/functions/DoFnFunction.java index 42969febc6994..a66d3a44c20e5 100644 --- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/functions/DoFnFunction.java +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/functions/DoFnFunction.java @@ -30,30 +30,33 @@ import org.apache.beam.runners.gearpump.translators.utils.NoOpAggregatorFactory; import org.apache.beam.runners.gearpump.translators.utils.NoOpStepContext; import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.transforms.reflect.DoFnInvokers; import org.apache.beam.sdk.util.SideInputReader; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.WindowingStrategy; import org.apache.beam.sdk.values.TupleTag; import org.apache.beam.sdk.values.TupleTagList; - -import org.apache.gearpump.streaming.javaapi.dsl.functions.FlatMapFunction; +import org.apache.gearpump.streaming.dsl.javaapi.functions.FlatMapFunction; /** * Gearpump {@link FlatMapFunction} wrapper over Beam {@link DoFn}. */ -public class DoFnFunction implements - FlatMapFunction, WindowedValue>, DoFnRunners.OutputManager { +public class DoFnFunction extends + FlatMapFunction, WindowedValue> implements + DoFnRunners.OutputManager { private final TupleTag mainTag = new TupleTag() {}; private List> outputs = Lists.newArrayList(); private final DoFnRunnerFactory doFnRunnerFactory; private DoFnRunner doFnRunner; + private final DoFn doFn; public DoFnFunction( GearpumpPipelineOptions pipelineOptions, DoFn doFn, WindowingStrategy windowingStrategy, SideInputReader sideInputReader) { + this.doFn = doFn; this.doFnRunnerFactory = new DoFnRunnerFactory<>( pipelineOptions, doFn, @@ -67,6 +70,16 @@ public DoFnFunction( ); } + @Override + public void setup() { + DoFnInvokers.invokerFor(doFn).invokeSetup(); + } + + @Override + public void teardown() { + DoFnInvokers.invokerFor(doFn).invokeTeardown(); + } + @Override public Iterator> apply(WindowedValue value) { outputs = Lists.newArrayList(); From f6aaf0d9ecd6b67ad6f7eed413af3fae3b3bdf6f Mon Sep 17 00:00:00 2001 From: manuzhang Date: Sat, 14 Jan 2017 21:41:40 +0800 Subject: [PATCH 138/346] support OutputTimeFn --- .../translators/GroupByKeyTranslator.java | 57 ++++++++++++++++--- .../translators/WindowBoundTranslator.java | 20 +++++-- 2 files changed, 64 insertions(+), 13 deletions(-) diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/GroupByKeyTranslator.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/GroupByKeyTranslator.java index 4eaf7552fd44b..e16a1787f81a8 100644 --- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/GroupByKeyTranslator.java +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/GroupByKeyTranslator.java @@ -32,6 +32,7 @@ import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.GlobalWindow; import org.apache.beam.sdk.transforms.windowing.IntervalWindow; +import org.apache.beam.sdk.transforms.windowing.OutputTimeFn; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; @@ -58,12 +59,16 @@ public void translate(GroupByKey transform, TranslationContext context) { JavaStream>> inputStream = context.getInputStream(input); int parallelism = context.getPipelineOptions().getParallelism(); + OutputTimeFn outputTimeFn = (OutputTimeFn) + input.getWindowingStrategy().getOutputTimeFn(); JavaStream>>> outputStream = inputStream .window(Window.apply(new GearpumpWindowFn(input.getWindowingStrategy().getWindowFn()), EventTimeTrigger$.MODULE$, Discarding$.MODULE$), "assign_window") .groupBy(new GroupByFn(), parallelism, "group_by_Key_and_Window") .map(new ValueToIterable(), "map_value_to_iterable") - .reduce(new MergeValue(), "merge_value"); + .map(new KeyedByTimestamp(), "keyed_by_timestamp") + .reduce(new Merge(outputTimeFn), "merge") + .map(new Values(), "values"); context.setOutputStream(context.getOutput(transform), outputStream); } @@ -141,15 +146,53 @@ public WindowedValue>> apply(WindowedValue> wv) { } } - private static class MergeValue extends - ReduceFunction>>> { + private static class KeyedByTimestamp + extends MapFunction>>, + KV>>>> { @Override - public WindowedValue>> apply(WindowedValue>> wv1, - WindowedValue>> wv2) { - return WindowedValue.of(KV.of(wv1.getValue().getKey(), + public KV>>> apply( + WindowedValue>> wv) { + return KV.of(wv.getTimestamp(), wv); + } + } + + private static class Merge extends + ReduceFunction>>>> { + + private final OutputTimeFn outputTimeFn; + + Merge(OutputTimeFn outputTimeFn) { + this.outputTimeFn = outputTimeFn; + } + + @Override + public KV>>> apply( + KV>>> kv1, + KV>>> kv2) { + org.joda.time.Instant t1 = kv1.getKey(); + org.joda.time.Instant t2 = kv2.getKey(); + + WindowedValue>> wv1 = kv1.getValue(); + WindowedValue>> wv2 = kv2.getValue(); + + return KV.of(outputTimeFn.combine(t1, t2), + WindowedValue.of(KV.of(wv1.getValue().getKey(), Iterables.concat(wv1.getValue().getValue(), wv2.getValue().getValue())), - wv1.getTimestamp(), wv1.getWindows(), wv1.getPane()); + wv1.getTimestamp(), wv1.getWindows(), wv1.getPane())); + } + } + + private static class Values extends + MapFunction>>>, + WindowedValue>>> { + + @Override + public WindowedValue>> apply(KV>>> kv) { + org.joda.time.Instant timestamp = kv.getKey(); + WindowedValue>> wv = kv.getValue(); + return WindowedValue.of(wv.getValue(), timestamp, wv.getWindows(), wv.getPane()); } } } diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/WindowBoundTranslator.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/WindowBoundTranslator.java index d3c50a59cad7a..9bf1936127ddd 100644 --- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/WindowBoundTranslator.java +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/WindowBoundTranslator.java @@ -26,6 +26,7 @@ import java.util.List; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; +import org.apache.beam.sdk.transforms.windowing.OutputTimeFn; import org.apache.beam.sdk.transforms.windowing.Window; import org.apache.beam.sdk.transforms.windowing.WindowFn; import org.apache.beam.sdk.util.WindowedValue; @@ -53,9 +54,11 @@ public void translate(Window.Bound transform, TranslationContext context) { transform.getOutputStrategyInternal(input.getWindowingStrategy()); WindowFn windowFn = (WindowFn) outputStrategy.getWindowFn(); + OutputTimeFn outputTimeFn = (OutputTimeFn) + outputStrategy.getOutputTimeFn(); JavaStream> outputStream = inputStream - .flatMap(new AssignWindows(windowFn), "assign_windows") + .flatMap(new AssignWindows(windowFn, outputTimeFn), "assign_windows") .process(AssignTimestampTask.class, 1, UserConfig.empty(), "assign_timestamp"); context.setOutputStream(context.getOutput(transform), outputStream); @@ -64,17 +67,21 @@ public void translate(Window.Bound transform, TranslationContext context) { private static class AssignWindows extends FlatMapFunction, WindowedValue> { - private final WindowFn fn; + private final WindowFn windowFn; + private final OutputTimeFn outputTimeFn; - AssignWindows(WindowFn fn) { - this.fn = fn; + AssignWindows( + WindowFn windowFn, + OutputTimeFn outputTimeFn) { + this.windowFn = windowFn; + this.outputTimeFn = outputTimeFn; } @Override public Iterator> apply(final WindowedValue value) { List> ret = new LinkedList<>(); try { - Collection windows = fn.assignWindows(fn.new AssignContext() { + Collection windows = windowFn.assignWindows(windowFn.new AssignContext() { @Override public T element() { return value.getValue(); @@ -91,8 +98,9 @@ public BoundedWindow window() { } }); for (BoundedWindow window: windows) { + Instant timestamp = outputTimeFn.assignOutputTime(value.getTimestamp(), window); ret.add(WindowedValue.of( - value.getValue(), value.getTimestamp(), window, value.getPane())); + value.getValue(), timestamp, window, value.getPane())); } } catch (Exception e) { throw new RuntimeException(e); From 364a3f089747ff4761cb5b54c963c8a8013574a0 Mon Sep 17 00:00:00 2001 From: manuzhang Date: Mon, 16 Jan 2017 11:16:05 +0800 Subject: [PATCH 139/346] return encoded key for GroupByKey translation --- .../translators/GroupByKeyTranslator.java | 24 +++++++++++++++---- 1 file changed, 20 insertions(+), 4 deletions(-) diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/GroupByKeyTranslator.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/GroupByKeyTranslator.java index e16a1787f81a8..ac8e2183c903a 100644 --- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/GroupByKeyTranslator.java +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/GroupByKeyTranslator.java @@ -22,17 +22,22 @@ import com.google.common.collect.Lists; import java.io.Serializable; +import java.nio.ByteBuffer; import java.time.Instant; import java.util.Collection; import java.util.LinkedList; import java.util.List; import org.apache.beam.runners.gearpump.translators.utils.TranslatorUtils; +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.transforms.GroupByKey; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.GlobalWindow; import org.apache.beam.sdk.transforms.windowing.IntervalWindow; import org.apache.beam.sdk.transforms.windowing.OutputTimeFn; +import org.apache.beam.sdk.util.CoderUtils; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; @@ -56,6 +61,7 @@ public class GroupByKeyTranslator implements TransformTranslator transform, TranslationContext context) { PCollection> input = context.getInput(transform); + Coder inputKeyCoder = ((KvCoder) input.getCoder()).getKeyCoder(); JavaStream>> inputStream = context.getInputStream(input); int parallelism = context.getPipelineOptions().getParallelism(); @@ -64,7 +70,7 @@ public void translate(GroupByKey transform, TranslationContext context) { JavaStream>>> outputStream = inputStream .window(Window.apply(new GearpumpWindowFn(input.getWindowingStrategy().getWindowFn()), EventTimeTrigger$.MODULE$, Discarding$.MODULE$), "assign_window") - .groupBy(new GroupByFn(), parallelism, "group_by_Key_and_Window") + .groupBy(new GroupByFn(inputKeyCoder), parallelism, "group_by_Key_and_Window") .map(new ValueToIterable(), "map_value_to_iterable") .map(new KeyedByTimestamp(), "keyed_by_timestamp") .reduce(new Merge(outputTimeFn), "merge") @@ -128,11 +134,21 @@ private Bucket getBucket(BoundedWindow window) { } private static class GroupByFn extends - GroupByFunction>, K> { + GroupByFunction>, ByteBuffer> { + + private final Coder keyCoder; + + GroupByFn(Coder keyCoder) { + this.keyCoder = keyCoder; + } @Override - public K apply(WindowedValue> wv) { - return wv.getValue().getKey(); + public ByteBuffer apply(WindowedValue> wv) { + try { + return ByteBuffer.wrap(CoderUtils.encodeToByteArray(keyCoder, wv.getValue().getKey())); + } catch (CoderException e) { + throw new RuntimeException(e); + } } } From b2d326ff73afca5c8e941c8006e9d74261a6b9df Mon Sep 17 00:00:00 2001 From: manuzhang Date: Mon, 16 Jan 2017 12:31:26 +0800 Subject: [PATCH 140/346] fix ParDo.BoundMulti translation --- .../gearpump/translators/ParDoBoundMultiTranslator.java | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/ParDoBoundMultiTranslator.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/ParDoBoundMultiTranslator.java index 0d5b8bcccbf71..bf7073b4430d7 100644 --- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/ParDoBoundMultiTranslator.java +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/ParDoBoundMultiTranslator.java @@ -91,8 +91,7 @@ private static class DoFnMultiFunction private final DoFnRunnerFactory doFnRunnerFactory; private DoFnRunner doFnRunner; private final DoFn doFn; - private final List, OutputT>>> outputs = Lists - .newArrayList(); + private List, OutputT>>> outputs; public DoFnMultiFunction( GearpumpPipelineOptions pipelineOptions, @@ -127,6 +126,8 @@ public void teardown() { @Override public Iterator, OutputT>>> apply(WindowedValue wv) { + outputs = Lists.newArrayList(); + if (null == doFnRunner) { doFnRunner = doFnRunnerFactory.createRunner(); } @@ -166,6 +167,7 @@ private static class ExtractOutput extends @Override public WindowedValue apply(WindowedValue, OutputT>> wv) { + // System.out.println(wv.getValue().getKey() + ":" + wv.getValue().getValue()); return WindowedValue.of(wv.getValue().getValue(), wv.getTimestamp(), wv.getWindows(), wv.getPane()); } From 7613ec44cedf12d1e7bf80e8bb6a505f09653c4f Mon Sep 17 00:00:00 2001 From: manuzhang Date: Mon, 16 Jan 2017 13:25:12 +0800 Subject: [PATCH 141/346] reduce timeout to wait for result --- .../apache/beam/runners/gearpump/GearpumpPipelineResult.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpPipelineResult.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpPipelineResult.java index 9c8f7b345f45c..3dd78de61a461 100644 --- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpPipelineResult.java +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpPipelineResult.java @@ -42,8 +42,8 @@ public class GearpumpPipelineResult implements PipelineResult { private final ClientContext client; private final int appId; - private final Duration defaultWaitDuration = Duration.standardSeconds(60); - private final Duration defaultWaitInterval = Duration.standardSeconds(10); + private final Duration defaultWaitDuration = Duration.standardSeconds(30); + private final Duration defaultWaitInterval = Duration.standardSeconds(5); public GearpumpPipelineResult(ClientContext client, int appId) { this.client = client; From 85dcfbd153acb4e450a4f0f94fc54b19b76507d3 Mon Sep 17 00:00:00 2001 From: manuzhang Date: Fri, 20 Jan 2017 08:33:04 +0800 Subject: [PATCH 142/346] Remove cache for Gearpump on travis --- .travis.yml | 1 + 1 file changed, 1 insertion(+) diff --git a/.travis.yml b/.travis.yml index a806477a13d81..7dcd5d12bd992 100644 --- a/.travis.yml +++ b/.travis.yml @@ -59,6 +59,7 @@ before_install: install: # Removing this here protects from inadvertent caching - rm -rf "$HOME/.m2/repository/org/apache/beam" + - rm -rf "$HOME/.m2/repository/org/apache/gearpump" script: - travis_retry mvn --batch-mode --update-snapshots --no-snapshot-updates $MAVEN_OVERRIDE install && travis_retry bash -ex .travis/test_wordcount.sh From d814857a6c372ba3f87106d49d3ce1ef7c3c7766 Mon Sep 17 00:00:00 2001 From: manuzhang Date: Fri, 20 Jan 2017 13:21:24 +0800 Subject: [PATCH 143/346] note thread is interrupted on InterruptedException --- .../beam/runners/gearpump/GearpumpPipelineResult.java | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpPipelineResult.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpPipelineResult.java index 3dd78de61a461..9e5351787a1ec 100644 --- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpPipelineResult.java +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpPipelineResult.java @@ -67,7 +67,13 @@ public State waitUntilFinish(Duration duration) { do { try { Thread.sleep(defaultWaitInterval.getMillis()); - } catch (InterruptedException e) { + } catch (Exception e) { + if (e instanceof InterruptedException) { + Thread.currentThread().interrupt(); + } + if (e instanceof RuntimeException) { + throw (RuntimeException) e; + } throw new RuntimeException(e); } } while (State.RUNNING == getGearpumpState() From 4fd216b6b9f7427a7df118a60202752ebeb1836d Mon Sep 17 00:00:00 2001 From: manuzhang Date: Tue, 24 Jan 2017 13:16:07 +0800 Subject: [PATCH 144/346] [BEAM-79] Fix PostCommit test confs for Gearpump runner --- .../job_beam_PostCommit_Java_RunnableOnService_Gearpump.groovy | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.jenkins/job_beam_PostCommit_Java_RunnableOnService_Gearpump.groovy b/.jenkins/job_beam_PostCommit_Java_RunnableOnService_Gearpump.groovy index f07bbc6f7a096..847ade309d990 100644 --- a/.jenkins/job_beam_PostCommit_Java_RunnableOnService_Gearpump.groovy +++ b/.jenkins/job_beam_PostCommit_Java_RunnableOnService_Gearpump.groovy @@ -37,5 +37,5 @@ mavenJob('beam_PostCommit_Java_RunnableOnService_Gearpump') { 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" ]\'') + goals('-B -e clean verify -am -pl runners/gearpump -Plocal-runnable-on-service-tests -Prunnable-on-service-tests') } From 7af6472082cbc7f3853e87831ed4bdc72978a3a3 Mon Sep 17 00:00:00 2001 From: manuzhang Date: Tue, 7 Feb 2017 22:14:18 +0800 Subject: [PATCH 145/346] [BEAM-79] Support merging windows in GearpumpRunner --- runners/gearpump/pom.xml | 5 - .../gearpump/GearpumpPipelineResult.java | 8 +- .../beam/runners/gearpump/GearpumpRunner.java | 24 +--- .../translators/GroupByKeyTranslator.java | 133 ++++++++++-------- .../translators/WindowBoundTranslator.java | 53 +------ .../translators/io/GearpumpSource.java | 6 +- .../translators/utils/DoFnRunnerFactory.java | 1 + .../translators/utils/TranslatorUtils.java | 20 +++ .../utils/TranslatorUtilsTest.java | 75 ++++++++++ 9 files changed, 186 insertions(+), 139 deletions(-) create mode 100644 runners/gearpump/src/test/java/org/apache/beam/runners/gearpump/translators/utils/TranslatorUtilsTest.java diff --git a/runners/gearpump/pom.xml b/runners/gearpump/pom.xml index 7c6fa7633dbce..6f91c502d7884 100644 --- a/runners/gearpump/pom.xml +++ b/runners/gearpump/pom.xml @@ -93,11 +93,6 @@ org.apache.beam.sdk.transforms.ViewTest, org.apache.beam.sdk.transforms.join.CoGroupByKeyTest - - - org.apache.beam.sdk.transforms.windowing.WindowingTest, - org.apache.beam.sdk.util.ReshuffleTest - diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpPipelineResult.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpPipelineResult.java index 9e5351787a1ec..a3740b7e73ec0 100644 --- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpPipelineResult.java +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpPipelineResult.java @@ -27,7 +27,7 @@ import org.apache.beam.sdk.metrics.MetricResults; import org.apache.beam.sdk.transforms.Aggregator; -import org.apache.gearpump.cluster.MasterToAppMaster; +import org.apache.gearpump.cluster.ApplicationStatus; import org.apache.gearpump.cluster.MasterToAppMaster.AppMasterData; import org.apache.gearpump.cluster.client.ClientContext; import org.joda.time.Duration; @@ -105,7 +105,7 @@ public MetricResults metrics() { } private State getGearpumpState() { - String status = null; + ApplicationStatus status = null; List apps = JavaConverters.seqAsJavaListConverter( (Seq) client.listApps().appMasters()).asJava(); @@ -114,9 +114,9 @@ private State getGearpumpState() { status = app.status(); } } - if (null == status || status.equals(MasterToAppMaster.AppMasterNonExist())) { + if (null == status || status instanceof ApplicationStatus.NONEXIST$) { return State.UNKNOWN; - } else if (status.equals(MasterToAppMaster.AppMasterActive())) { + } else if (status instanceof ApplicationStatus.ACTIVE$) { return State.RUNNING; } else { return State.STOPPED; diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpRunner.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpRunner.java index 01fdb3b2e0982..9ca1eb2cf0e76 100644 --- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpRunner.java +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpRunner.java @@ -29,13 +29,8 @@ import org.apache.beam.sdk.options.PipelineOptionsValidator; 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.Flatten; import org.apache.beam.sdk.transforms.PTransform; -import org.apache.beam.sdk.transforms.ParDo; -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.util.WindowingStrategy; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionList; @@ -74,11 +69,7 @@ public static GearpumpRunner fromOptions(PipelineOptions options) { public OutputT apply( PTransform transform, InputT input) { - if (Window.Bound.class.equals(transform.getClass()) - && isNullOrIdentityWindowFn(((Window.Bound) transform).getWindowFn())) { - return (OutputT) super.apply( - ParDo.of(new IdentityFn()), input); - } else if (Flatten.FlattenPCollectionList.class.equals(transform.getClass()) + if (Flatten.FlattenPCollectionList.class.equals(transform.getClass()) && ((PCollectionList) input).size() == 0) { return (OutputT) Pipeline.applyTransform(input.getPipeline().begin(), Create.of()); } else if (Create.Values.class.equals(transform.getClass())) { @@ -108,7 +99,7 @@ public GearpumpPipelineResult run(Pipeline pipeline) { TranslationContext translationContext = new TranslationContext(streamApp, options); GearpumpPipelineTranslator translator = new GearpumpPipelineTranslator(translationContext); translator.translate(pipeline); - int appId = streamApp.submit(); + int appId = streamApp.submit().appId(); return new GearpumpPipelineResult(clientContext, appId); } @@ -140,15 +131,4 @@ private Config registerSerializers(Config config, Map userSerial return config.withValue(GEARPUMP_SERIALIZERS, ConfigValueFactory.fromMap(serializers)); } - private static class IdentityFn extends DoFn { - - @ProcessElement - public void process(ProcessContext c) { - c.output(c.element()); - } - } - - private boolean isNullOrIdentityWindowFn(WindowFn windowFn) { - return windowFn == null || windowFn.getClass().equals(IdentityWindowFn.class); - } } diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/GroupByKeyTranslator.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/GroupByKeyTranslator.java index ac8e2183c903a..69a1d11ef11d3 100644 --- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/GroupByKeyTranslator.java +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/GroupByKeyTranslator.java @@ -23,9 +23,8 @@ import java.io.Serializable; import java.nio.ByteBuffer; -import java.time.Instant; +import java.util.ArrayList; import java.util.Collection; -import java.util.LinkedList; import java.util.List; import org.apache.beam.runners.gearpump.translators.utils.TranslatorUtils; @@ -34,9 +33,8 @@ import org.apache.beam.sdk.coders.KvCoder; import org.apache.beam.sdk.transforms.GroupByKey; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.transforms.windowing.GlobalWindow; -import org.apache.beam.sdk.transforms.windowing.IntervalWindow; import org.apache.beam.sdk.transforms.windowing.OutputTimeFn; +import org.apache.beam.sdk.transforms.windowing.WindowFn; import org.apache.beam.sdk.util.CoderUtils; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; @@ -48,15 +46,14 @@ import org.apache.gearpump.streaming.dsl.javaapi.functions.GroupByFunction; import org.apache.gearpump.streaming.dsl.window.api.Discarding$; import org.apache.gearpump.streaming.dsl.window.api.EventTimeTrigger$; -import org.apache.gearpump.streaming.dsl.window.api.Window; -import org.apache.gearpump.streaming.dsl.window.api.WindowFn; -import org.apache.gearpump.streaming.dsl.window.impl.Bucket; -import scala.collection.JavaConversions; - +import org.apache.gearpump.streaming.dsl.window.api.WindowFunction; +import org.apache.gearpump.streaming.dsl.window.api.Windows; +import org.apache.gearpump.streaming.dsl.window.impl.Window; /** * {@link GroupByKey} is translated to Gearpump groupBy function. */ +@SuppressWarnings({"rawtypes", "unchecked"}) public class GroupByKeyTranslator implements TransformTranslator> { @Override public void translate(GroupByKey transform, TranslationContext context) { @@ -67,69 +64,51 @@ public void translate(GroupByKey transform, TranslationContext context) { int parallelism = context.getPipelineOptions().getParallelism(); OutputTimeFn outputTimeFn = (OutputTimeFn) input.getWindowingStrategy().getOutputTimeFn(); + WindowFn, BoundedWindow> windowFn = (WindowFn, BoundedWindow>) + input.getWindowingStrategy().getWindowFn(); JavaStream>>> outputStream = inputStream - .window(Window.apply(new GearpumpWindowFn(input.getWindowingStrategy().getWindowFn()), + .window(Windows.apply( + new GearpumpWindowFn(windowFn.isNonMerging()), EventTimeTrigger$.MODULE$, Discarding$.MODULE$), "assign_window") .groupBy(new GroupByFn(inputKeyCoder), parallelism, "group_by_Key_and_Window") .map(new ValueToIterable(), "map_value_to_iterable") - .map(new KeyedByTimestamp(), "keyed_by_timestamp") - .reduce(new Merge(outputTimeFn), "merge") + .map(new KeyedByTimestamp((OutputTimeFn) + input.getWindowingStrategy().getOutputTimeFn()), "keyed_by_timestamp") + .reduce(new Merge<>(windowFn, outputTimeFn), "merge") .map(new Values(), "values"); context.setOutputStream(context.getOutput(transform), outputStream); } - private static class GearpumpWindowFn implements WindowFn, - Serializable { + private static class GearpumpWindowFn + implements WindowFunction>, Serializable { - private org.apache.beam.sdk.transforms.windowing.WindowFn windowFn; + private final boolean isNonMerging; - GearpumpWindowFn(org.apache.beam.sdk.transforms.windowing.WindowFn windowFn) { - this.windowFn = windowFn; + public GearpumpWindowFn(boolean isNonMerging) { + this.isNonMerging = isNonMerging; } @Override - public scala.collection.immutable.List apply(final Instant timestamp) { + public Window[] apply(Context> context) { try { - Collection windows = windowFn.assignWindows(windowFn.new AssignContext() { - @Override - public T element() { - throw new UnsupportedOperationException(); - } - - @Override - public org.joda.time.Instant timestamp() { - return TranslatorUtils.java8TimeToJodaTime(timestamp); - } - - @Override - public W window() { - throw new UnsupportedOperationException(); - } - }); - - List buckets = new LinkedList<>(); - for (BoundedWindow window : windows) { - buckets.add(getBucket(window)); - } - return JavaConversions.asScalaBuffer(buckets).toList(); + return toGearpumpWindows(context.element().getWindows().toArray(new BoundedWindow[0])); } catch (Exception e) { throw new RuntimeException(e); } } - private Bucket getBucket(BoundedWindow window) { - if (window instanceof IntervalWindow) { - IntervalWindow intervalWindow = (IntervalWindow) window; - Instant start = TranslatorUtils.jodaTimeToJava8Time(intervalWindow.start()); - Instant end = TranslatorUtils.jodaTimeToJava8Time(intervalWindow.end()); - return new Bucket(start, end); - } else if (window instanceof GlobalWindow) { - Instant end = TranslatorUtils.jodaTimeToJava8Time(window.maxTimestamp()); - return new Bucket(Instant.MIN, end); - } else { - throw new RuntimeException("unknown window " + window.getClass().getName()); + @Override + public boolean isNonMerging() { + return isNonMerging; + } + + private Window[] toGearpumpWindows(BoundedWindow[] windows) { + Window[] gwins = new Window[windows.length]; + for (int i = 0; i < windows.length; i++) { + gwins[i] = TranslatorUtils.boundedWindowToGearpumpWindow(windows[i]); } + return gwins; } } @@ -166,19 +145,30 @@ private static class KeyedByTimestamp extends MapFunction>>, KV>>>> { + private final OutputTimeFn outputTimeFn; + + public KeyedByTimestamp(OutputTimeFn outputTimeFn) { + this.outputTimeFn = outputTimeFn; + } + @Override public KV>>> apply( WindowedValue>> wv) { - return KV.of(wv.getTimestamp(), wv); + org.joda.time.Instant timestamp = outputTimeFn.assignOutputTime(wv.getTimestamp(), + Iterables.getOnlyElement(wv.getWindows())); + return KV.of(timestamp, wv); } } private static class Merge extends ReduceFunction>>>> { + private final WindowFn, BoundedWindow> windowFn; private final OutputTimeFn outputTimeFn; - Merge(OutputTimeFn outputTimeFn) { + Merge(WindowFn, BoundedWindow> windowFn, + OutputTimeFn outputTimeFn) { + this.windowFn = windowFn; this.outputTimeFn = outputTimeFn; } @@ -189,13 +179,40 @@ public KV>>> apply( org.joda.time.Instant t1 = kv1.getKey(); org.joda.time.Instant t2 = kv2.getKey(); - WindowedValue>> wv1 = kv1.getValue(); - WindowedValue>> wv2 = kv2.getValue(); + final WindowedValue>> wv1 = kv1.getValue(); + final WindowedValue>> wv2 = kv2.getValue(); + + final List mergedWindows = new ArrayList<>(); + if (!windowFn.isNonMerging()) { + try { + windowFn.mergeWindows(windowFn.new MergeContext() { + + @Override + public Collection windows() { + ArrayList windows = new ArrayList<>(); + windows.addAll(wv1.getWindows()); + windows.addAll(wv2.getWindows()); + return windows; + } + + @Override + public void merge(Collection toBeMerged, + BoundedWindow mergeResult) throws Exception { + mergedWindows.add(mergeResult); + } + }); + } catch (Exception e) { + throw new RuntimeException(e); + } + } else { + mergedWindows.addAll(wv1.getWindows()); + } - return KV.of(outputTimeFn.combine(t1, t2), + org.joda.time.Instant timestamp = outputTimeFn.combine(t1, t2); + return KV.of(timestamp, WindowedValue.of(KV.of(wv1.getValue().getKey(), - Iterables.concat(wv1.getValue().getValue(), wv2.getValue().getValue())), - wv1.getTimestamp(), wv1.getWindows(), wv1.getPane())); + Iterables.concat(wv1.getValue().getValue(), wv2.getValue().getValue())), timestamp, + mergedWindows, wv1.getPane())); } } diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/WindowBoundTranslator.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/WindowBoundTranslator.java index 9bf1936127ddd..c0de2df192507 100644 --- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/WindowBoundTranslator.java +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/WindowBoundTranslator.java @@ -21,23 +21,15 @@ import com.google.common.collect.Iterables; import java.util.Collection; -import java.util.Iterator; -import java.util.LinkedList; -import java.util.List; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.transforms.windowing.OutputTimeFn; import org.apache.beam.sdk.transforms.windowing.Window; import org.apache.beam.sdk.transforms.windowing.WindowFn; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.WindowingStrategy; import org.apache.beam.sdk.values.PCollection; -import org.apache.gearpump.Message; -import org.apache.gearpump.cluster.UserConfig; +import org.apache.gearpump.streaming.dsl.api.functions.MapFunction; import org.apache.gearpump.streaming.dsl.javaapi.JavaStream; -import org.apache.gearpump.streaming.dsl.javaapi.functions.FlatMapFunction; -import org.apache.gearpump.streaming.javaapi.Task; -import org.apache.gearpump.streaming.task.TaskContext; import org.joda.time.Instant; /** @@ -52,34 +44,25 @@ public void translate(Window.Bound transform, TranslationContext context) { JavaStream> inputStream = context.getInputStream(input); WindowingStrategy outputStrategy = transform.getOutputStrategyInternal(input.getWindowingStrategy()); - WindowFn windowFn = - (WindowFn) outputStrategy.getWindowFn(); - OutputTimeFn outputTimeFn = (OutputTimeFn) - outputStrategy.getOutputTimeFn(); + WindowFn windowFn = (WindowFn) outputStrategy.getWindowFn(); JavaStream> outputStream = inputStream - .flatMap(new AssignWindows(windowFn, outputTimeFn), "assign_windows") - .process(AssignTimestampTask.class, 1, UserConfig.empty(), "assign_timestamp"); + .map(new AssignWindows(windowFn), "assign_windows"); context.setOutputStream(context.getOutput(transform), outputStream); } private static class AssignWindows extends - FlatMapFunction, WindowedValue> { + MapFunction, WindowedValue> { private final WindowFn windowFn; - private final OutputTimeFn outputTimeFn; - AssignWindows( - WindowFn windowFn, - OutputTimeFn outputTimeFn) { + AssignWindows(WindowFn windowFn) { this.windowFn = windowFn; - this.outputTimeFn = outputTimeFn; } @Override - public Iterator> apply(final WindowedValue value) { - List> ret = new LinkedList<>(); + public WindowedValue apply(final WindowedValue value) { try { Collection windows = windowFn.assignWindows(windowFn.new AssignContext() { @Override @@ -97,32 +80,10 @@ public BoundedWindow window() { return Iterables.getOnlyElement(value.getWindows()); } }); - for (BoundedWindow window: windows) { - Instant timestamp = outputTimeFn.assignOutputTime(value.getTimestamp(), window); - ret.add(WindowedValue.of( - value.getValue(), timestamp, window, value.getPane())); - } + return WindowedValue.of(value.getValue(), value.getTimestamp(), windows, value.getPane()); } catch (Exception e) { throw new RuntimeException(e); } - return ret.iterator(); - } - } - - /** - * Assign WindowedValue timestamp to Gearpump message. - * @param element type of WindowedValue - */ - public static class AssignTimestampTask extends Task { - - public AssignTimestampTask(TaskContext taskContext, UserConfig userConfig) { - super(taskContext, userConfig); - } - - @Override - public void onNext(Message message) { - final WindowedValue value = (WindowedValue) message.msg(); - context.output(Message.apply(value, value.getTimestamp().getMillis())); } } } diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/io/GearpumpSource.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/io/GearpumpSource.java index 6e5b2defd77d5..3d0d7c8905653 100644 --- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/io/GearpumpSource.java +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/io/GearpumpSource.java @@ -62,9 +62,8 @@ public void open(TaskContext context, Instant startTime) { this.reader = createReader(options); this.available = reader.start(); } catch (Exception e) { - throw new RuntimeException(e); - } finally { close(); + throw new RuntimeException(e); } } @@ -81,9 +80,8 @@ public Message read() { timestamp.getMillis()); } } catch (Exception e) { - throw new RuntimeException(e); - } finally { close(); + throw new RuntimeException(e); } return message; } diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/utils/DoFnRunnerFactory.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/utils/DoFnRunnerFactory.java index 7e1402f4ebeb8..aaefb88a2748f 100644 --- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/utils/DoFnRunnerFactory.java +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/utils/DoFnRunnerFactory.java @@ -38,6 +38,7 @@ */ public class DoFnRunnerFactory implements Serializable { + private static final long serialVersionUID = 1083167395296383469L; private final DoFn fn; private final transient PipelineOptions options; private final SideInputReader sideInputReader; diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/utils/TranslatorUtils.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/utils/TranslatorUtils.java index 9b722754a4814..656fc6a147d26 100644 --- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/utils/TranslatorUtils.java +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/utils/TranslatorUtils.java @@ -20,6 +20,12 @@ import java.time.Instant; +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.gearpump.streaming.dsl.window.impl.Window; + + /** * Utility methods for translators. */ @@ -32,4 +38,18 @@ public static Instant jodaTimeToJava8Time(org.joda.time.Instant time) { public static org.joda.time.Instant java8TimeToJodaTime(Instant time) { return new org.joda.time.Instant(time.toEpochMilli()); } + + public static Window boundedWindowToGearpumpWindow(BoundedWindow window) { + Instant end = TranslatorUtils.jodaTimeToJava8Time(window.maxTimestamp().plus(1L)); + if (window instanceof IntervalWindow) { + IntervalWindow intervalWindow = (IntervalWindow) window; + Instant start = TranslatorUtils.jodaTimeToJava8Time(intervalWindow.start()); + return new Window(start, end); + } else if (window instanceof GlobalWindow) { + return new Window(TranslatorUtils.jodaTimeToJava8Time(BoundedWindow.TIMESTAMP_MIN_VALUE), + end); + } else { + throw new RuntimeException("unknown window " + window.getClass().getName()); + } + } } diff --git a/runners/gearpump/src/test/java/org/apache/beam/runners/gearpump/translators/utils/TranslatorUtilsTest.java b/runners/gearpump/src/test/java/org/apache/beam/runners/gearpump/translators/utils/TranslatorUtilsTest.java new file mode 100644 index 0000000000000..10976e8e96622 --- /dev/null +++ b/runners/gearpump/src/test/java/org/apache/beam/runners/gearpump/translators/utils/TranslatorUtilsTest.java @@ -0,0 +1,75 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.beam.runners.gearpump.translators.utils; + +import static org.hamcrest.Matchers.equalTo; +import static org.junit.Assert.assertThat; + +import com.google.common.collect.Lists; + +import java.time.Duration; +import java.time.Instant; +import java.util.List; + +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.values.KV; +import org.apache.gearpump.streaming.dsl.window.impl.Window; +import org.junit.Test; + + +/** + * Tests for {@link TranslatorUtils}. + */ +public class TranslatorUtilsTest { + + private static final List> TEST_VALUES = Lists.newArrayList( + KV.of(new org.joda.time.Instant(0), Instant.EPOCH), + KV.of(new org.joda.time.Instant(42), Instant.ofEpochMilli(42)), + KV.of(new org.joda.time.Instant(Long.MIN_VALUE), Instant.ofEpochMilli(Long.MIN_VALUE)), + KV.of(new org.joda.time.Instant(Long.MAX_VALUE), Instant.ofEpochMilli(Long.MAX_VALUE))); + + @Test + public void testJodaTimeAndJava8TimeConversion() { + for (KV kv: TEST_VALUES) { + assertThat(TranslatorUtils.jodaTimeToJava8Time(kv.getKey()), + equalTo(kv.getValue())); + assertThat(TranslatorUtils.java8TimeToJodaTime(kv.getValue()), + equalTo(kv.getKey())); + } + } + + @Test + public void testBoundedWindowToGearpumpWindow() { + assertThat(TranslatorUtils.boundedWindowToGearpumpWindow( + new IntervalWindow(new org.joda.time.Instant(0), + new org.joda.time.Instant(Long.MAX_VALUE))), + equalTo(Window.apply(Instant.EPOCH, Instant.ofEpochMilli(Long.MAX_VALUE)))); + assertThat(TranslatorUtils.boundedWindowToGearpumpWindow( + new IntervalWindow(new org.joda.time.Instant(Long.MIN_VALUE), + new org.joda.time.Instant(Long.MAX_VALUE))), + equalTo(Window.apply(Instant.ofEpochMilli(Long.MIN_VALUE), + Instant.ofEpochMilli(Long.MAX_VALUE)))); + BoundedWindow globalWindow = GlobalWindow.INSTANCE; + assertThat(TranslatorUtils.boundedWindowToGearpumpWindow(globalWindow), + equalTo(Window.apply(Instant.ofEpochMilli(Long.MIN_VALUE / 1000), + Instant.ofEpochMilli(Long.MAX_VALUE / 1000).minus(Duration.ofDays(1)).plusMillis(1)))); + } +} From 4eb50d152b91df46bd7f0478650cb4abac3808c6 Mon Sep 17 00:00:00 2001 From: manuzhang Date: Tue, 14 Feb 2017 12:33:31 +0800 Subject: [PATCH 146/346] [BEAM-79] Add SideInput support for GearpumpRunner --- runners/gearpump/pom.xml | 16 +- .../gearpump/GearpumpPipelineResult.java | 47 +-- .../gearpump/GearpumpPipelineTranslator.java | 8 + .../beam/runners/gearpump/GearpumpRunner.java | 373 +++++++++++++++++- .../runners/gearpump/TestGearpumpRunner.java | 39 +- ...eateGearpumpPCollectionViewTranslator.java | 44 +++ ...a => CreatePCollectionViewTranslator.java} | 35 +- .../translators/CreateValuesTranslator.java | 2 + .../FlattenPCollectionTranslator.java | 38 ++ .../translators/GroupByKeyTranslator.java | 82 ++-- .../ParDoBoundMultiTranslator.java | 165 +++----- .../translators/ParDoBoundTranslator.java | 32 +- .../translators/WindowBoundTranslator.java | 21 +- .../translators/functions/DoFnFunction.java | 158 ++++++-- .../translators/io/BoundedSourceWrapper.java | 1 + .../translators/io/GearpumpSource.java | 23 +- .../gearpump/translators/io/ValuesSource.java | 14 +- .../translators/utils/DoFnRunnerFactory.java | 20 +- .../translators/utils/TranslatorUtils.java | 147 +++++++ 19 files changed, 974 insertions(+), 291 deletions(-) create mode 100644 runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/CreateGearpumpPCollectionViewTranslator.java rename runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/{utils/NoOpSideInputReader.java => CreatePCollectionViewTranslator.java} (52%) diff --git a/runners/gearpump/pom.xml b/runners/gearpump/pom.xml index 6f91c502d7884..6a41dc064a466 100644 --- a/runners/gearpump/pom.xml +++ b/runners/gearpump/pom.xml @@ -75,23 +75,9 @@ org.apache.beam:beam-sdks-java-core - -noverify - - org.apache.beam.sdk.io.gcp.bigquery.BigQueryIOTest, - org.apache.beam.sdk.io.CountingInputTest, - org.apache.beam.sdk.io.CountingSourceTest, - org.apache.beam.sdk.testing.PAssertTest, - org.apache.beam.sdk.transforms.ApproximateUniqueTest, - org.apache.beam.sdk.transforms.CombineTest, - org.apache.beam.sdk.transforms.CombineFnsTest, - org.apache.beam.sdk.transforms.CountTest, - org.apache.beam.sdk.transforms.FlattenTest, - org.apache.beam.sdk.transforms.ParDoTest, - org.apache.beam.sdk.transforms.SampleTest, - org.apache.beam.sdk.transforms.ViewTest, - org.apache.beam.sdk.transforms.join.CoGroupByKeyTest + org.apache.beam.sdk.transforms.ParDoTest diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpPipelineResult.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpPipelineResult.java index a3740b7e73ec0..8f908983ee3d0 100644 --- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpPipelineResult.java +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpPipelineResult.java @@ -30,6 +30,7 @@ import org.apache.gearpump.cluster.ApplicationStatus; import org.apache.gearpump.cluster.MasterToAppMaster.AppMasterData; import org.apache.gearpump.cluster.client.ClientContext; +import org.apache.gearpump.cluster.client.RunningApplication; import org.joda.time.Duration; import scala.collection.JavaConverters; @@ -41,13 +42,11 @@ public class GearpumpPipelineResult implements PipelineResult { private final ClientContext client; - private final int appId; - private final Duration defaultWaitDuration = Duration.standardSeconds(30); - private final Duration defaultWaitInterval = Duration.standardSeconds(5); + private final RunningApplication app; - public GearpumpPipelineResult(ClientContext client, int appId) { + public GearpumpPipelineResult(ClientContext client, RunningApplication app) { this.client = client; - this.appId = appId; + this.app = app; } @Override @@ -57,38 +56,19 @@ public State getState() { @Override public State cancel() throws IOException { - client.shutdown(appId); + app.shutDown(); return State.CANCELLED; } @Override public State waitUntilFinish(Duration duration) { - long start = System.currentTimeMillis(); - do { - try { - Thread.sleep(defaultWaitInterval.getMillis()); - } catch (Exception e) { - if (e instanceof InterruptedException) { - Thread.currentThread().interrupt(); - } - if (e instanceof RuntimeException) { - throw (RuntimeException) e; - } - throw new RuntimeException(e); - } - } while (State.RUNNING == getGearpumpState() - && (System.currentTimeMillis() - start) < duration.getMillis()); - - if (State.RUNNING == getGearpumpState()) { - return State.DONE; - } else { - return State.FAILED; - } + return waitUntilFinish(); } @Override public State waitUntilFinish() { - return waitUntilFinish(defaultWaitDuration); + app.waitUntilFinish(); + return State.DONE; } @Override @@ -109,18 +89,19 @@ private State getGearpumpState() { List apps = JavaConverters.seqAsJavaListConverter( (Seq) client.listApps().appMasters()).asJava(); - for (AppMasterData app: apps) { - if (app.appId() == appId) { - status = app.status(); + for (AppMasterData appData: apps) { + if (appData.appId() == app.appId()) { + status = appData.status(); } } if (null == status || status instanceof ApplicationStatus.NONEXIST$) { return State.UNKNOWN; } else if (status instanceof ApplicationStatus.ACTIVE$) { return State.RUNNING; + } else if (status instanceof ApplicationStatus.SUCCEEDED$) { + return State.DONE; } else { - return State.STOPPED; + return State.FAILED; } } - } diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpPipelineTranslator.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpPipelineTranslator.java index 20624ed382644..4cc060c76fcda 100644 --- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpPipelineTranslator.java +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpPipelineTranslator.java @@ -20,6 +20,9 @@ import java.util.HashMap; import java.util.Map; + +import org.apache.beam.runners.gearpump.translators.CreateGearpumpPCollectionViewTranslator; +import org.apache.beam.runners.gearpump.translators.CreatePCollectionViewTranslator; import org.apache.beam.runners.gearpump.translators.CreateValuesTranslator; import org.apache.beam.runners.gearpump.translators.FlattenPCollectionTranslator; import org.apache.beam.runners.gearpump.translators.GroupByKeyTranslator; @@ -38,6 +41,7 @@ import org.apache.beam.sdk.transforms.GroupByKey; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.ParDo; +import org.apache.beam.sdk.transforms.View; import org.apache.beam.sdk.transforms.windowing.Window; import org.apache.beam.sdk.values.PValue; @@ -75,6 +79,10 @@ public class GearpumpPipelineTranslator extends Pipeline.PipelineVisitor.Default registerTransformTranslator(ParDo.BoundMulti.class, new ParDoBoundMultiTranslator()); registerTransformTranslator(Window.Bound.class, new WindowBoundTranslator()); registerTransformTranslator(Create.Values.class, new CreateValuesTranslator()); + registerTransformTranslator(View.CreatePCollectionView.class, + new CreatePCollectionViewTranslator()); + registerTransformTranslator(GearpumpRunner.CreateGearpumpPCollectionView.class, + new CreateGearpumpPCollectionViewTranslator<>()); } public GearpumpPipelineTranslator(TranslationContext translationContext) { diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpRunner.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpRunner.java index 9ca1eb2cf0e76..72f21260c8e8c 100644 --- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpRunner.java +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpRunner.java @@ -17,29 +17,45 @@ */ package org.apache.beam.runners.gearpump; +import com.google.common.collect.ImmutableMap; import com.typesafe.config.Config; import com.typesafe.config.ConfigValueFactory; +import java.util.ArrayList; +import java.util.Collections; import java.util.HashMap; +import java.util.List; import java.util.Map; import org.apache.beam.runners.gearpump.translators.TranslationContext; import org.apache.beam.sdk.Pipeline; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.coders.CoderRegistry; +import org.apache.beam.sdk.coders.KvCoder; +import org.apache.beam.sdk.coders.ListCoder; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.options.PipelineOptionsValidator; import org.apache.beam.sdk.runners.PipelineRunner; +import org.apache.beam.sdk.transforms.Combine; import org.apache.beam.sdk.transforms.Create; +import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.Flatten; import org.apache.beam.sdk.transforms.PTransform; -import org.apache.beam.sdk.util.WindowingStrategy; +import org.apache.beam.sdk.transforms.ParDo; +import org.apache.beam.sdk.transforms.View; +import org.apache.beam.sdk.util.InstanceBuilder; +import org.apache.beam.sdk.util.PCollectionViews; +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.PCollectionView; import org.apache.beam.sdk.values.PInput; import org.apache.beam.sdk.values.POutput; import org.apache.gearpump.cluster.ClusterConfig; import org.apache.gearpump.cluster.UserConfig; import org.apache.gearpump.cluster.client.ClientContext; +import org.apache.gearpump.cluster.client.RunningApplication; import org.apache.gearpump.cluster.embedded.EmbeddedCluster; import org.apache.gearpump.streaming.dsl.javaapi.JavaStreamApp; @@ -56,8 +72,21 @@ public class GearpumpRunner extends PipelineRunner { private static final String GEARPUMP_SERIALIZERS = "gearpump.serializers"; private static final String DEFAULT_APPNAME = "beam_gearpump_app"; + /** Custom transforms implementations. */ + private final Map, Class> overrides; + public GearpumpRunner(GearpumpPipelineOptions options) { this.options = options; + + ImmutableMap.Builder, Class> builder = ImmutableMap.builder(); + builder.put(Combine.GloballyAsSingletonView.class, + StreamingCombineGloballyAsSingletonView.class); + builder.put(View.AsMap.class, StreamingViewAsMap.class); + builder.put(View.AsMultimap.class, StreamingViewAsMultimap.class); + builder.put(View.AsSingleton.class, StreamingViewAsSingleton.class); + builder.put(View.AsList.class, StreamingViewAsList.class); + builder.put(View.AsIterable.class, StreamingViewAsIterable.class); + overrides = builder.build(); } public static GearpumpRunner fromOptions(PipelineOptions options) { @@ -69,15 +98,23 @@ public static GearpumpRunner fromOptions(PipelineOptions options) { public OutputT apply( PTransform transform, InputT input) { - if (Flatten.FlattenPCollectionList.class.equals(transform.getClass()) + if (overrides.containsKey(transform.getClass())) { + + Class> transformClass = + (Class>) transform.getClass(); + + Class> customTransformClass = + (Class>) overrides.get(transform.getClass()); + + PTransform customTransform = + InstanceBuilder.ofType(customTransformClass) + .withArg(transformClass, transform) + .build(); + + return Pipeline.applyTransform(input, customTransform); + } else if (Flatten.FlattenPCollectionList.class.equals(transform.getClass()) && ((PCollectionList) input).size() == 0) { return (OutputT) Pipeline.applyTransform(input.getPipeline().begin(), Create.of()); - } else if (Create.Values.class.equals(transform.getClass())) { - return (OutputT) PCollection - .createPrimitiveOutputInternal( - input.getPipeline(), - WindowingStrategy.globalDefault(), - PCollection.IsBounded.BOUNDED); } else { return super.apply(transform, input); } @@ -99,9 +136,9 @@ public GearpumpPipelineResult run(Pipeline pipeline) { TranslationContext translationContext = new TranslationContext(streamApp, options); GearpumpPipelineTranslator translator = new GearpumpPipelineTranslator(translationContext); translator.translate(pipeline); - int appId = streamApp.submit().appId(); + RunningApplication app = streamApp.submit(); - return new GearpumpPipelineResult(clientContext, appId); + return new GearpumpPipelineResult(clientContext, app); } private ClientContext getClientContext(GearpumpPipelineOptions options, Config config) { @@ -131,4 +168,320 @@ private Config registerSerializers(Config config, Map userSerial return config.withValue(GEARPUMP_SERIALIZERS, ConfigValueFactory.fromMap(serializers)); } + + + // The following codes are forked from DataflowRunner for View translator + /** + * Specialized implementation for + * {@link org.apache.beam.sdk.transforms.View.AsMap View.AsMap} + * for the Gearpump runner. + */ + private static class StreamingViewAsMap + extends PTransform>, PCollectionView>> { + + private static final long serialVersionUID = 4791080760092950304L; + + public StreamingViewAsMap(View.AsMap transform) {} + + @Override + public PCollectionView> expand(PCollection> input) { + PCollectionView> view = + PCollectionViews.mapView( + input.getPipeline(), + input.getWindowingStrategy(), + input.getCoder()); + + @SuppressWarnings({"rawtypes", "unchecked"}) + KvCoder inputCoder = (KvCoder) input.getCoder(); + try { + inputCoder.getKeyCoder().verifyDeterministic(); + } catch (Coder.NonDeterministicException e) { + // throw new RuntimeException(e); + } + + return input + .apply(Combine.globally(new Concatenate>()).withoutDefaults()) + .apply(CreateGearpumpPCollectionView., Map>of(view)); + } + + @Override + protected String getKindString() { + return "StreamingViewAsMap"; + } + } + + /** + * Specialized expansion for {@link + * org.apache.beam.sdk.transforms.View.AsMultimap View.AsMultimap} for the + * Gearpump runner. + */ + private static class StreamingViewAsMultimap + extends PTransform>, PCollectionView>>> { + + private static final long serialVersionUID = 5854899081751333352L; + + public StreamingViewAsMultimap(View.AsMultimap transform) {} + + @Override + public PCollectionView>> expand(PCollection> input) { + PCollectionView>> view = + PCollectionViews.multimapView( + input.getPipeline(), + input.getWindowingStrategy(), + input.getCoder()); + + @SuppressWarnings({"rawtypes", "unchecked"}) + KvCoder inputCoder = (KvCoder) input.getCoder(); + try { + inputCoder.getKeyCoder().verifyDeterministic(); + } catch (Coder.NonDeterministicException e) { + // throw new RuntimeException(e); + } + + return input + .apply(Combine.globally(new Concatenate>()).withoutDefaults()) + .apply(CreateGearpumpPCollectionView., Map>>of(view)); + } + + @Override + protected String getKindString() { + return "StreamingViewAsMultimap"; + } + } + + /** + * Specialized implementation for + * {@link org.apache.beam.sdk.transforms.View.AsIterable View.AsIterable} for the + * Gearpump runner. + */ + private static class StreamingViewAsIterable + extends PTransform, PCollectionView>> { + + private static final long serialVersionUID = -3399860618995613421L; + + public StreamingViewAsIterable(View.AsIterable transform) {} + + @Override + public PCollectionView> expand(PCollection input) { + PCollectionView> view = + PCollectionViews.iterableView( + input.getPipeline(), + input.getWindowingStrategy(), + input.getCoder()); + + return input.apply(Combine.globally(new Concatenate()).withoutDefaults()) + .apply(CreateGearpumpPCollectionView.>of(view)); + } + + @Override + protected String getKindString() { + return "StreamingViewAsIterable"; + } + } + + /** + * Specialized implementation for + * {@link org.apache.beam.sdk.transforms.View.AsList View.AsList} for the + * Gearpump runner. + */ + private static class StreamingViewAsList + extends PTransform, PCollectionView>> { + + private static final long serialVersionUID = -5018631473886330629L; + + public StreamingViewAsList(View.AsList transform) {} + + @Override + public PCollectionView> expand(PCollection input) { + PCollectionView> view = + PCollectionViews.listView( + input.getPipeline(), + input.getWindowingStrategy(), + input.getCoder()); + + return input.apply(Combine.globally(new Concatenate()).withoutDefaults()) + .apply(CreateGearpumpPCollectionView.>of(view)); + } + + @Override + protected String getKindString() { + return "StreamingViewAsList"; + } + } + private static class StreamingCombineGloballyAsSingletonView + extends PTransform, PCollectionView> { + + private static final long serialVersionUID = 9064900748869035738L; + private final Combine.GloballyAsSingletonView transform; + + public StreamingCombineGloballyAsSingletonView( + Combine.GloballyAsSingletonView transform) { + this.transform = transform; + } + + @Override + public PCollectionView expand(PCollection input) { + PCollection combined = + input.apply(Combine.globally(transform.getCombineFn()) + .withoutDefaults() + .withFanout(transform.getFanout())); + + PCollectionView view = PCollectionViews.singletonView( + combined.getPipeline(), + combined.getWindowingStrategy(), + transform.getInsertDefault(), + transform.getInsertDefault() + ? transform.getCombineFn().defaultValue() : null, + combined.getCoder()); + return combined + .apply(ParDo.of(new WrapAsList())) + .apply(CreateGearpumpPCollectionView.of(view)); + } + + @Override + protected String getKindString() { + return "StreamingCombineGloballyAsSingletonView"; + } + } + + private static class StreamingViewAsSingleton + extends PTransform, PCollectionView> { + + private static final long serialVersionUID = 5870455965625071546L; + private final View.AsSingleton transform; + + public StreamingViewAsSingleton(View.AsSingleton transform) { + this.transform = transform; + } + + @Override + public PCollectionView expand(PCollection input) { + Combine.Globally combine = Combine.globally( + new SingletonCombine<>(transform.hasDefaultValue(), transform.defaultValue())); + if (!transform.hasDefaultValue()) { + combine = combine.withoutDefaults(); + } + return input.apply(combine.asSingletonView()); + } + + @Override + protected String getKindString() { + return "StreamingViewAsSingleton"; + } + + private static class SingletonCombine extends Combine.BinaryCombineFn { + private boolean hasDefaultValue; + private T defaultValue; + + SingletonCombine(boolean hasDefaultValue, T defaultValue) { + this.hasDefaultValue = hasDefaultValue; + this.defaultValue = defaultValue; + } + + @Override + public T apply(T left, T right) { + throw new IllegalArgumentException("PCollection with more than one element " + + "accessed as a singleton view. Consider using Combine.globally().asSingleton() to " + + "combine the PCollection into a single value"); + } + + @Override + public T identity() { + if (hasDefaultValue) { + return defaultValue; + } else { + throw new IllegalArgumentException( + "Empty PCollection accessed as a singleton view. " + + "Consider setting withDefault to provide a default value"); + } + } + } + } + + private static class WrapAsList extends DoFn> { + @ProcessElement + public void processElement(ProcessContext c) { + c.output(Collections.singletonList(c.element())); + } + } + + /** + * Creates a primitive {@link PCollectionView}. + * + *

              For internal use only by runner implementors. + * + * @param The type of the elements of the input PCollection + * @param The type associated with the {@link PCollectionView} used as a side input + */ + public static class CreateGearpumpPCollectionView + extends PTransform>, PCollectionView> { + private static final long serialVersionUID = -2637073020800540542L; + private PCollectionView view; + + private CreateGearpumpPCollectionView(PCollectionView view) { + this.view = view; + } + + public static CreateGearpumpPCollectionView of( + PCollectionView view) { + return new CreateGearpumpPCollectionView<>(view); + } + + public PCollectionView getView() { + return view; + } + + @Override + public PCollectionView expand(PCollection> input) { + return view; + } + } + + /** + * Combiner that combines {@code T}s into a single {@code List} containing all inputs. + * + *

              For internal use by {@link StreamingViewAsMap}, {@link StreamingViewAsMultimap}, + * {@link StreamingViewAsList}, {@link StreamingViewAsIterable}. + * They require the input {@link PCollection} fits in memory. + * For a large {@link PCollection} this is expected to crash! + * + * @param the type of elements to concatenate. + */ + private static class Concatenate extends Combine.CombineFn, List> { + @Override + public List createAccumulator() { + return new ArrayList<>(); + } + + @Override + public List addInput(List accumulator, T input) { + accumulator.add(input); + return accumulator; + } + + @Override + public List mergeAccumulators(Iterable> accumulators) { + List result = createAccumulator(); + for (List accumulator : accumulators) { + result.addAll(accumulator); + } + return result; + } + + @Override + public List extractOutput(List accumulator) { + return accumulator; + } + + @Override + public Coder> getAccumulatorCoder(CoderRegistry registry, Coder inputCoder) { + return ListCoder.of(inputCoder); + } + + @Override + public Coder> getDefaultOutputCoder(CoderRegistry registry, Coder inputCoder) { + return ListCoder.of(inputCoder); + } + } + } diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/TestGearpumpRunner.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/TestGearpumpRunner.java index ee31fb509d0a8..c96bcb1f1df4f 100644 --- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/TestGearpumpRunner.java +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/TestGearpumpRunner.java @@ -18,8 +18,9 @@ package org.apache.beam.runners.gearpump; +import com.typesafe.config.Config; +import com.typesafe.config.ConfigValueFactory; import org.apache.beam.sdk.Pipeline; -import org.apache.beam.sdk.PipelineResult; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.options.PipelineOptionsValidator; import org.apache.beam.sdk.runners.PipelineRunner; @@ -27,7 +28,9 @@ import org.apache.beam.sdk.values.PInput; import org.apache.beam.sdk.values.POutput; +import org.apache.gearpump.cluster.ClusterConfig; import org.apache.gearpump.cluster.embedded.EmbeddedCluster; +import org.apache.gearpump.util.Constants; /** * Gearpump {@link PipelineRunner} for tests, which uses {@link EmbeddedCluster}. @@ -38,7 +41,10 @@ public class TestGearpumpRunner extends PipelineRunner { private final EmbeddedCluster cluster; private TestGearpumpRunner(GearpumpPipelineOptions options) { - cluster = EmbeddedCluster.apply(); + Config config = ClusterConfig.master(null); + config = config.withValue(Constants.APPLICATION_TOTAL_RETRIES(), + ConfigValueFactory.fromAnyRef(0)); + cluster = new EmbeddedCluster(config); cluster.start(); options.setEmbeddedCluster(cluster); delegate = GearpumpRunner.fromOptions(options); @@ -52,12 +58,31 @@ public static TestGearpumpRunner fromOptions(PipelineOptions options) { @Override public GearpumpPipelineResult run(Pipeline pipeline) { - GearpumpPipelineResult result = delegate.run(pipeline); - PipelineResult.State state = result.waitUntilFinish(); - cluster.stop(); - assert(state == PipelineResult.State.DONE); + try { + GearpumpPipelineResult result = delegate.run(pipeline); + result.waitUntilFinish(); + cluster.stop(); + return result; + } catch (Throwable e) { + // copied from TestFlinkRunner to pull out AssertionError + // which is wrapped in UserCodeException + Throwable cause = e; + Throwable oldCause; + do { + if (cause.getCause() == null) { + break; + } - return result; + oldCause = cause; + cause = cause.getCause(); + + } while (!oldCause.equals(cause)); + if (cause instanceof AssertionError) { + throw (AssertionError) cause; + } else { + throw e; + } + } } @Override diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/CreateGearpumpPCollectionViewTranslator.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/CreateGearpumpPCollectionViewTranslator.java new file mode 100644 index 0000000000000..d05c89dc3ab96 --- /dev/null +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/CreateGearpumpPCollectionViewTranslator.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.runners.gearpump.translators; + +import java.util.List; + +import org.apache.beam.runners.gearpump.GearpumpRunner; +import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.PCollectionView; +import org.apache.gearpump.streaming.dsl.javaapi.JavaStream; + + +/** + * CreateGearpumpPCollectionView bridges input stream to down stream + * transforms. + */ +public class CreateGearpumpPCollectionViewTranslator implements + TransformTranslator> { + + @Override + public void translate(GearpumpRunner.CreateGearpumpPCollectionView transform, + TranslationContext context) { + JavaStream>> inputStream = + context.getInputStream(context.getInput(transform)); + PCollectionView view = transform.getView(); + context.setOutputStream(view, inputStream); + } +} diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/utils/NoOpSideInputReader.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/CreatePCollectionViewTranslator.java similarity index 52% rename from runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/utils/NoOpSideInputReader.java rename to runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/CreatePCollectionViewTranslator.java index d1a91988d962b..e9e2e5d0c58fe 100644 --- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/utils/NoOpSideInputReader.java +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/CreatePCollectionViewTranslator.java @@ -16,33 +16,28 @@ * limitations under the License. */ -package org.apache.beam.runners.gearpump.translators.utils; +package org.apache.beam.runners.gearpump.translators; -import java.io.Serializable; +import java.util.List; -import javax.annotation.Nullable; - -import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.SideInputReader; +import org.apache.beam.sdk.transforms.View; +import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.PCollectionView; +import org.apache.gearpump.streaming.dsl.javaapi.JavaStream; /** - * no-op side input reader. + * View.CreatePCollectionView bridges input stream to down stream + * transforms. */ -public class NoOpSideInputReader implements SideInputReader, Serializable { - @Nullable - @Override - public T get(PCollectionView view, BoundedWindow window) { - return null; - } - - @Override - public boolean contains(PCollectionView view) { - return false; - } +public class CreatePCollectionViewTranslator implements + TransformTranslator> { @Override - public boolean isEmpty() { - return false; + public void translate(View.CreatePCollectionView transform, + TranslationContext context) { + JavaStream>> inputStream = + context.getInputStream(context.getInput(transform)); + PCollectionView view = transform.getView(); + context.setOutputStream(view, inputStream); } } diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/CreateValuesTranslator.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/CreateValuesTranslator.java index 452127aa2c9b3..e5dc6dd096c20 100644 --- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/CreateValuesTranslator.java +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/CreateValuesTranslator.java @@ -33,6 +33,8 @@ */ public class CreateValuesTranslator implements TransformTranslator> { + private static final long serialVersionUID = 5411841848199229738L; + @Override public void translate(Create.Values transform, TranslationContext context) { try { diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/FlattenPCollectionTranslator.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/FlattenPCollectionTranslator.java index b740ab5c6cacf..27e54b8f436fd 100644 --- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/FlattenPCollectionTranslator.java +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/FlattenPCollectionTranslator.java @@ -18,11 +18,22 @@ package org.apache.beam.runners.gearpump.translators; +import com.google.common.collect.Lists; + +import java.util.HashSet; +import java.util.Set; + +import org.apache.beam.runners.gearpump.translators.io.UnboundedSourceWrapper; +import org.apache.beam.runners.gearpump.translators.io.ValuesSource; +import org.apache.beam.sdk.coders.StringUtf8Coder; import org.apache.beam.sdk.transforms.Flatten; import org.apache.beam.sdk.values.PCollection; +import org.apache.gearpump.streaming.dsl.api.functions.MapFunction; import org.apache.gearpump.streaming.dsl.javaapi.JavaStream; + + /** * Flatten.FlattenPCollectionList is translated to Gearpump merge function. * Note only two-way merge is working now @@ -30,17 +41,44 @@ public class FlattenPCollectionTranslator implements TransformTranslator> { + private static final long serialVersionUID = -5552148802472944759L; + @Override public void translate(Flatten.FlattenPCollectionList transform, TranslationContext context) { JavaStream merged = null; + Set> unique = new HashSet<>(); for (PCollection collection : context.getInput(transform).getAll()) { + unique.add(collection); JavaStream inputStream = context.getInputStream(collection); if (null == merged) { merged = inputStream; } else { + // duplicate edges are not allowed in Gearpump graph + // so we route through a dummy node + if (unique.contains(collection)) { + inputStream = inputStream.map(new DummyFunction(), "dummy"); + } + merged = merged.merge(inputStream, transform.getName()); } } + + if (null == merged) { + UnboundedSourceWrapper unboundedSourceWrapper = new UnboundedSourceWrapper<>( + new ValuesSource<>(Lists.newArrayList("dummy"), + StringUtf8Coder.of()), context.getPipelineOptions()); + merged = context.getSourceStream(unboundedSourceWrapper); + } context.setOutputStream(context.getOutput(transform), merged); } + + private static class DummyFunction extends MapFunction { + + private static final long serialVersionUID = 5454396869997290471L; + + @Override + public T map(T t) { + return t; + } + } } diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/GroupByKeyTranslator.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/GroupByKeyTranslator.java index 69a1d11ef11d3..df8bfe95988c8 100644 --- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/GroupByKeyTranslator.java +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/GroupByKeyTranslator.java @@ -40,8 +40,8 @@ import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; +import org.apache.gearpump.streaming.dsl.api.functions.FoldFunction; import org.apache.gearpump.streaming.dsl.api.functions.MapFunction; -import org.apache.gearpump.streaming.dsl.api.functions.ReduceFunction; import org.apache.gearpump.streaming.dsl.javaapi.JavaStream; import org.apache.gearpump.streaming.dsl.javaapi.functions.GroupByFunction; import org.apache.gearpump.streaming.dsl.window.api.Discarding$; @@ -49,12 +49,16 @@ import org.apache.gearpump.streaming.dsl.window.api.WindowFunction; import org.apache.gearpump.streaming.dsl.window.api.Windows; import org.apache.gearpump.streaming.dsl.window.impl.Window; +import org.joda.time.Instant; /** * {@link GroupByKey} is translated to Gearpump groupBy function. */ @SuppressWarnings({"rawtypes", "unchecked"}) public class GroupByKeyTranslator implements TransformTranslator> { + + private static final long serialVersionUID = -8742202583992787659L; + @Override public void translate(GroupByKey transform, TranslationContext context) { PCollection> input = context.getInput(transform); @@ -66,15 +70,14 @@ public void translate(GroupByKey transform, TranslationContext context) { input.getWindowingStrategy().getOutputTimeFn(); WindowFn, BoundedWindow> windowFn = (WindowFn, BoundedWindow>) input.getWindowingStrategy().getWindowFn(); - JavaStream>>> outputStream = inputStream + JavaStream>>> outputStream = inputStream .window(Windows.apply( new GearpumpWindowFn(windowFn.isNonMerging()), EventTimeTrigger$.MODULE$, Discarding$.MODULE$), "assign_window") .groupBy(new GroupByFn(inputKeyCoder), parallelism, "group_by_Key_and_Window") - .map(new ValueToIterable(), "map_value_to_iterable") .map(new KeyedByTimestamp((OutputTimeFn) input.getWindowingStrategy().getOutputTimeFn()), "keyed_by_timestamp") - .reduce(new Merge<>(windowFn, outputTimeFn), "merge") + .fold(new Merge<>(windowFn, outputTimeFn), "merge") .map(new Values(), "values"); context.setOutputStream(context.getOutput(transform), outputStream); @@ -115,6 +118,7 @@ private Window[] toGearpumpWindows(BoundedWindow[] windows) { private static class GroupByFn extends GroupByFunction>, ByteBuffer> { + private static final long serialVersionUID = -807905402490735530L; private final Coder keyCoder; GroupByFn(Coder keyCoder) { @@ -122,7 +126,7 @@ private static class GroupByFn extends } @Override - public ByteBuffer apply(WindowedValue> wv) { + public ByteBuffer groupBy(WindowedValue> wv) { try { return ByteBuffer.wrap(CoderUtils.encodeToByteArray(keyCoder, wv.getValue().getKey())); } catch (CoderException e) { @@ -131,19 +135,9 @@ public ByteBuffer apply(WindowedValue> wv) { } } - private static class ValueToIterable - extends MapFunction>, WindowedValue>>> { - - @Override - public WindowedValue>> apply(WindowedValue> wv) { - Iterable values = Lists.newArrayList(wv.getValue().getValue()); - return wv.withValue(KV.of(wv.getValue().getKey(), values)); - } - } - private static class KeyedByTimestamp - extends MapFunction>>, - KV>>>> { + extends MapFunction>, + KV>>> { private final OutputTimeFn outputTimeFn; @@ -152,16 +146,17 @@ public KeyedByTimestamp(OutputTimeFn outputTimeFn) { } @Override - public KV>>> apply( - WindowedValue>> wv) { - org.joda.time.Instant timestamp = outputTimeFn.assignOutputTime(wv.getTimestamp(), + public KV>> map( + WindowedValue> wv) { + Instant timestamp = outputTimeFn.assignOutputTime(wv.getTimestamp(), Iterables.getOnlyElement(wv.getWindows())); return KV.of(timestamp, wv); } } private static class Merge extends - ReduceFunction>>>> { + FoldFunction>>, + KV>>>> { private final WindowFn, BoundedWindow> windowFn; private final OutputTimeFn outputTimeFn; @@ -173,14 +168,28 @@ private static class Merge extends } @Override - public KV>>> apply( - KV>>> kv1, - KV>>> kv2) { - org.joda.time.Instant t1 = kv1.getKey(); - org.joda.time.Instant t2 = kv2.getKey(); + public KV>>> init() { + return KV.of(null, null); + } + + @Override + public KV>>> fold( + KV>>> accum, + KV>> iter) { + if (accum.getKey() == null) { + WindowedValue> wv = iter.getValue(); + KV kv = wv.getValue(); + V v = kv.getValue(); + List nv = Lists.newArrayList(v); + return KV.of(iter.getKey(), wv.withValue(KV.of(kv.getKey(), nv))); + } + + Instant t1 = accum.getKey(); + Instant t2 = iter.getKey(); - final WindowedValue>> wv1 = kv1.getValue(); - final WindowedValue>> wv2 = kv2.getValue(); + final WindowedValue>> wv1 = accum.getValue(); + final WindowedValue> wv2 = iter.getValue(); + wv1.getValue().getValue().add(wv2.getValue().getValue()); final List mergedWindows = new ArrayList<>(); if (!windowFn.isNonMerging()) { @@ -208,23 +217,22 @@ public void merge(Collection toBeMerged, mergedWindows.addAll(wv1.getWindows()); } - org.joda.time.Instant timestamp = outputTimeFn.combine(t1, t2); + Instant timestamp = outputTimeFn.combine(t1, t2); return KV.of(timestamp, - WindowedValue.of(KV.of(wv1.getValue().getKey(), - Iterables.concat(wv1.getValue().getValue(), wv2.getValue().getValue())), timestamp, + WindowedValue.of(wv1.getValue(), timestamp, mergedWindows, wv1.getPane())); } } private static class Values extends - MapFunction>>>, - WindowedValue>>> { + MapFunction>>>, + WindowedValue>>> { @Override - public WindowedValue>> apply(KV>>> kv) { - org.joda.time.Instant timestamp = kv.getKey(); - WindowedValue>> wv = kv.getValue(); + public WindowedValue>> map(KV>>> kv) { + Instant timestamp = kv.getKey(); + WindowedValue>> wv = kv.getValue(); return WindowedValue.of(wv.getValue(), timestamp, wv.getWindows(), wv.getPane()); } } diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/ParDoBoundMultiTranslator.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/ParDoBoundMultiTranslator.java index bf7073b4430d7..8c57019e39518 100644 --- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/ParDoBoundMultiTranslator.java +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/ParDoBoundMultiTranslator.java @@ -18,158 +18,93 @@ package org.apache.beam.runners.gearpump.translators; + +import com.google.common.base.Predicate; import com.google.common.collect.Lists; +import com.google.common.collect.Sets; -import java.util.Iterator; +import java.util.Collection; import java.util.List; import java.util.Map; -import org.apache.beam.runners.core.DoFnRunner; -import org.apache.beam.runners.core.DoFnRunners; -import org.apache.beam.runners.gearpump.GearpumpPipelineOptions; -import org.apache.beam.runners.gearpump.translators.utils.DoFnRunnerFactory; -import org.apache.beam.runners.gearpump.translators.utils.NoOpAggregatorFactory; -import org.apache.beam.runners.gearpump.translators.utils.NoOpSideInputReader; -import org.apache.beam.runners.gearpump.translators.utils.NoOpStepContext; +import javax.annotation.Nullable; + +import org.apache.beam.runners.gearpump.translators.functions.DoFnFunction; +import org.apache.beam.runners.gearpump.translators.utils.TranslatorUtils; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.ParDo; -import org.apache.beam.sdk.transforms.reflect.DoFnInvokers; -import org.apache.beam.sdk.util.SideInputReader; import org.apache.beam.sdk.util.WindowedValue; -import org.apache.beam.sdk.util.WindowingStrategy; -import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TupleTag; -import org.apache.beam.sdk.values.TupleTagList; import org.apache.gearpump.streaming.dsl.api.functions.FilterFunction; -import org.apache.gearpump.streaming.dsl.api.functions.MapFunction; import org.apache.gearpump.streaming.dsl.javaapi.JavaStream; -import org.apache.gearpump.streaming.dsl.javaapi.functions.FlatMapFunction; /** * {@link ParDo.BoundMulti} is translated to Gearpump flatMap function - * with {@link DoFn} wrapped in {@link DoFnMultiFunction}. The outputs are + * with {@link DoFn} wrapped in {@link DoFnFunction}. The outputs are * further filtered with Gearpump filter function by output tag */ @SuppressWarnings({"rawtypes", "unchecked"}) public class ParDoBoundMultiTranslator implements TransformTranslator> { + private static final long serialVersionUID = -6023461558200028849L; + @Override public void translate(ParDo.BoundMulti transform, TranslationContext context) { PCollection inputT = (PCollection) context.getInput(transform); JavaStream> inputStream = context.getInputStream(inputT); - Map, PCollection> outputs = context.getOutput(transform).getAll(); + Collection> sideInputs = transform.getSideInputs(); + Map> tagsToSideInputs = + TranslatorUtils.getTagsToSideInputs(sideInputs); - JavaStream, OutputT>>> outputStream = inputStream.flatMap( - new DoFnMultiFunction<>( - context.getPipelineOptions(), - transform.getFn(), - transform.getMainOutputTag(), - transform.getSideOutputTags(), - inputT.getWindowingStrategy(), - new NoOpSideInputReader() - ), transform.getName()); - for (Map.Entry, PCollection> output : outputs.entrySet()) { + Map, PCollection> outputs = context.getOutput(transform).getAll(); + final TupleTag mainOutput = transform.getMainOutputTag(); + List> sideOutputs = Lists.newLinkedList(Sets.filter(outputs.keySet(), + new Predicate>() { + @Override + public boolean apply(@Nullable TupleTag tupleTag) { + return tupleTag != null && !tupleTag.getId().equals(mainOutput.getId()); + } + })); + + JavaStream unionStream = TranslatorUtils.withSideInputStream( + context, inputStream, tagsToSideInputs); + + JavaStream outputStream = + TranslatorUtils.toList(unionStream).flatMap( + new DoFnFunction<>( + context.getPipelineOptions(), + transform.getFn(), + inputT.getWindowingStrategy(), + sideInputs, + tagsToSideInputs, + mainOutput, + sideOutputs), transform.getName()); + for (Map.Entry, PCollection> output: outputs.entrySet()) { + output.getValue().getCoder(); JavaStream> taggedStream = outputStream - .filter(new FilterByOutputTag<>((TupleTag) output.getKey()) - , "filter_by_output_tag") - .map(new ExtractOutput(), "extract output"); - + .filter(new FilterByOutputTag(output.getKey().getId()), + "filter_by_output_tag") + .map(new TranslatorUtils.FromRawUnionValue(), "from_RawUnionValue"); context.setOutputStream(output.getValue(), taggedStream); } } - /** - * Gearpump {@link FlatMapFunction} wrapper over Beam {@link DoFnMultiFunction}. - */ - private static class DoFnMultiFunction - extends FlatMapFunction, WindowedValue, OutputT>>> - implements DoFnRunners.OutputManager { - - private final DoFnRunnerFactory doFnRunnerFactory; - private DoFnRunner doFnRunner; - private final DoFn doFn; - private List, OutputT>>> outputs; - - public DoFnMultiFunction( - GearpumpPipelineOptions pipelineOptions, - DoFn doFn, - TupleTag mainOutputTag, - TupleTagList sideOutputTags, - WindowingStrategy windowingStrategy, - SideInputReader sideInputReader) { - this.doFn = doFn; - this.doFnRunnerFactory = new DoFnRunnerFactory<>( - pipelineOptions, - doFn, - sideInputReader, - this, - mainOutputTag, - sideOutputTags.getAll(), - new NoOpStepContext(), - new NoOpAggregatorFactory(), - windowingStrategy - ); - } - - @Override - public void setup() { - DoFnInvokers.invokerFor(doFn).invokeSetup(); - } - - @Override - public void teardown() { - DoFnInvokers.invokerFor(doFn).invokeTeardown(); - } - - @Override - public Iterator, OutputT>>> apply(WindowedValue wv) { - outputs = Lists.newArrayList(); - - if (null == doFnRunner) { - doFnRunner = doFnRunnerFactory.createRunner(); - } - doFnRunner.startBundle(); - doFnRunner.processElement(wv); - doFnRunner.finishBundle(); - - return outputs.iterator(); - } - - @Override - public void output(TupleTag tag, WindowedValue output) { - KV, OutputT> kv = KV.of((TupleTag) tag, - (OutputT) output.getValue()); - outputs.add(WindowedValue.of(kv, output.getTimestamp(), - output.getWindows(), output.getPane())); - } - } - - private static class FilterByOutputTag extends - FilterFunction, OutputT>>> { + private static class FilterByOutputTag extends FilterFunction { - private final TupleTag tupleTag; + private static final long serialVersionUID = 7276155265895637526L; + private final String tag; - public FilterByOutputTag(TupleTag tupleTag) { - this.tupleTag = tupleTag; + FilterByOutputTag(String tag) { + this.tag = tag; } @Override - public boolean apply(WindowedValue, OutputT>> wv) { - return wv.getValue().getKey().equals(tupleTag); - } - } - - private static class ExtractOutput extends - MapFunction, OutputT>>, WindowedValue> { - - @Override - public WindowedValue apply(WindowedValue, OutputT>> wv) { - // System.out.println(wv.getValue().getKey() + ":" + wv.getValue().getValue()); - return WindowedValue.of(wv.getValue().getValue(), wv.getTimestamp(), - wv.getWindows(), wv.getPane()); + public boolean filter(TranslatorUtils.RawUnionValue value) { + return value.getUnionTag().equals(tag); } } } diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/ParDoBoundTranslator.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/ParDoBoundTranslator.java index 689bc08e0f47e..efae9388d46e6 100644 --- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/ParDoBoundTranslator.java +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/ParDoBoundTranslator.java @@ -18,14 +18,21 @@ package org.apache.beam.runners.gearpump.translators; +import java.util.Collection; +import java.util.List; +import java.util.Map; + import org.apache.beam.runners.gearpump.translators.functions.DoFnFunction; -import org.apache.beam.runners.gearpump.translators.utils.NoOpSideInputReader; +import org.apache.beam.runners.gearpump.translators.utils.TranslatorUtils; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.ParDo; 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; import org.apache.gearpump.streaming.dsl.javaapi.JavaStream; @@ -36,18 +43,33 @@ public class ParDoBoundTranslator implements TransformTranslator> { + private static final long serialVersionUID = -3413205558160983784L; + private final TupleTag mainOutput = new TupleTag<>(); + private final List> sideOutputs = TupleTagList.empty().getAll(); + @Override public void translate(ParDo.Bound transform, TranslationContext context) { DoFn doFn = transform.getFn(); PCollection output = context.getOutput(transform); WindowingStrategy windowingStrategy = output.getWindowingStrategy(); + Collection> sideInputs = transform.getSideInputs(); + Map> tagsToSideInputs = + TranslatorUtils.getTagsToSideInputs(sideInputs); + JavaStream> inputStream = context.getInputStream( + context.getInput(transform)); + JavaStream unionStream = + TranslatorUtils.withSideInputStream(context, + inputStream, tagsToSideInputs); + DoFnFunction doFnFunction = new DoFnFunction<>(context.getPipelineOptions(), - doFn, windowingStrategy, new NoOpSideInputReader()); - JavaStream> inputStream = - context.getInputStream(context.getInput(transform)); + doFn, windowingStrategy, sideInputs, tagsToSideInputs, + mainOutput, sideOutputs); + JavaStream> outputStream = - inputStream.flatMap(doFnFunction, transform.getName()); + TranslatorUtils.toList(unionStream) + .flatMap(doFnFunction, transform.getName()) + .map(new TranslatorUtils.FromRawUnionValue(), "from_RawUnionValue"); context.setOutputStream(context.getOutput(transform), outputStream); } diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/WindowBoundTranslator.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/WindowBoundTranslator.java index c0de2df192507..81970e27b9945 100644 --- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/WindowBoundTranslator.java +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/WindowBoundTranslator.java @@ -20,7 +20,10 @@ import com.google.common.collect.Iterables; +import java.util.ArrayList; import java.util.Collection; +import java.util.Iterator; +import java.util.List; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.Window; @@ -28,8 +31,8 @@ import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.WindowingStrategy; import org.apache.beam.sdk.values.PCollection; -import org.apache.gearpump.streaming.dsl.api.functions.MapFunction; import org.apache.gearpump.streaming.dsl.javaapi.JavaStream; +import org.apache.gearpump.streaming.dsl.javaapi.functions.FlatMapFunction; import org.joda.time.Instant; /** @@ -38,6 +41,8 @@ @SuppressWarnings("unchecked") public class WindowBoundTranslator implements TransformTranslator> { + private static final long serialVersionUID = -964887482120489061L; + @Override public void translate(Window.Bound transform, TranslationContext context) { PCollection input = context.getInput(transform); @@ -47,14 +52,15 @@ public void translate(Window.Bound transform, TranslationContext context) { WindowFn windowFn = (WindowFn) outputStrategy.getWindowFn(); JavaStream> outputStream = inputStream - .map(new AssignWindows(windowFn), "assign_windows"); + .flatMap(new AssignWindows(windowFn), "assign_windows"); context.setOutputStream(context.getOutput(transform), outputStream); } private static class AssignWindows extends - MapFunction, WindowedValue> { + FlatMapFunction, WindowedValue> { + private static final long serialVersionUID = 7284565861938681360L; private final WindowFn windowFn; AssignWindows(WindowFn windowFn) { @@ -62,7 +68,7 @@ private static class AssignWindows extends } @Override - public WindowedValue apply(final WindowedValue value) { + public Iterator> flatMap(final WindowedValue value) { try { Collection windows = windowFn.assignWindows(windowFn.new AssignContext() { @Override @@ -80,7 +86,12 @@ public BoundedWindow window() { return Iterables.getOnlyElement(value.getWindows()); } }); - return WindowedValue.of(value.getValue(), value.getTimestamp(), windows, value.getPane()); + List> values = new ArrayList<>(windows.size()); + for (BoundedWindow win: windows) { + values.add( + WindowedValue.of(value.getValue(), value.getTimestamp(), win, value.getPane())); + } + return values.iterator(); } catch (Exception e) { throw new RuntimeException(e); } diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/functions/DoFnFunction.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/functions/DoFnFunction.java index a66d3a44c20e5..b2c68d6a8c07a 100644 --- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/functions/DoFnFunction.java +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/functions/DoFnFunction.java @@ -18,90 +18,190 @@ package org.apache.beam.runners.gearpump.translators.functions; +import com.google.common.collect.Iterables; + import com.google.common.collect.Lists; +import java.io.Serializable; +import java.util.Collection; +import java.util.HashMap; +import java.util.HashSet; import java.util.Iterator; +import java.util.LinkedList; import java.util.List; +import java.util.Map; +import java.util.Set; -import org.apache.beam.runners.core.DoFnRunner; import org.apache.beam.runners.core.DoFnRunners; +import org.apache.beam.runners.core.PushbackSideInputDoFnRunner; +import org.apache.beam.runners.core.SideInputHandler; import org.apache.beam.runners.gearpump.GearpumpPipelineOptions; import org.apache.beam.runners.gearpump.translators.utils.DoFnRunnerFactory; import org.apache.beam.runners.gearpump.translators.utils.NoOpAggregatorFactory; import org.apache.beam.runners.gearpump.translators.utils.NoOpStepContext; +import org.apache.beam.runners.gearpump.translators.utils.TranslatorUtils; +import org.apache.beam.runners.gearpump.translators.utils.TranslatorUtils.RawUnionValue; import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.transforms.reflect.DoFnInvoker; import org.apache.beam.sdk.transforms.reflect.DoFnInvokers; -import org.apache.beam.sdk.util.SideInputReader; +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.InMemoryStateInternals; +import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TupleTag; -import org.apache.beam.sdk.values.TupleTagList; import org.apache.gearpump.streaming.dsl.javaapi.functions.FlatMapFunction; /** * Gearpump {@link FlatMapFunction} wrapper over Beam {@link DoFn}. */ +@SuppressWarnings("unchecked") public class DoFnFunction extends - FlatMapFunction, WindowedValue> implements - DoFnRunners.OutputManager { + FlatMapFunction, RawUnionValue> { - private final TupleTag mainTag = new TupleTag() {}; - private List> outputs = Lists.newArrayList(); + private static final long serialVersionUID = -5701440128544343353L; private final DoFnRunnerFactory doFnRunnerFactory; - private DoFnRunner doFnRunner; private final DoFn doFn; + private transient DoFnInvoker doFnInvoker; + private transient PushbackSideInputDoFnRunner doFnRunner; + private transient SideInputHandler sideInputReader; + private transient List> pushedBackValues; + private transient Map, List>>> sideInputValues; + private final Collection> sideInputs; + private final Map> tagsToSideInputs; + private final TupleTag mainOutput; + private final List> sideOutputs; + private final DoFnOutputManager outputManager; public DoFnFunction( GearpumpPipelineOptions pipelineOptions, DoFn doFn, WindowingStrategy windowingStrategy, - SideInputReader sideInputReader) { + Collection> sideInputs, + Map> sideInputTagMapping, + TupleTag mainOutput, + List> sideOutputs) { this.doFn = doFn; + this.outputManager = new DoFnOutputManager(); this.doFnRunnerFactory = new DoFnRunnerFactory<>( pipelineOptions, doFn, - sideInputReader, - this, - mainTag, - TupleTagList.empty().getAll(), + sideInputs, + outputManager, + mainOutput, + sideOutputs, new NoOpStepContext(), new NoOpAggregatorFactory(), windowingStrategy ); + this.sideInputs = sideInputs; + this.tagsToSideInputs = sideInputTagMapping; + this.mainOutput = mainOutput; + this.sideOutputs = sideOutputs; } @Override public void setup() { - DoFnInvokers.invokerFor(doFn).invokeSetup(); + sideInputReader = new SideInputHandler(sideInputs, + InMemoryStateInternals.forKey(null)); + doFnInvoker = DoFnInvokers.invokerFor(doFn); + doFnInvoker.invokeSetup(); + + doFnRunner = doFnRunnerFactory.createRunner(sideInputReader); + + pushedBackValues = new LinkedList<>(); + sideInputValues = new HashMap<>(); + outputManager.setup(mainOutput, sideOutputs); } @Override public void teardown() { - DoFnInvokers.invokerFor(doFn).invokeTeardown(); + doFnInvoker.invokeTeardown(); } @Override - public Iterator> apply(WindowedValue value) { - outputs = Lists.newArrayList(); + public Iterator flatMap(List inputs) { + outputManager.clear(); + + doFnRunner.startBundle(); - if (null == doFnRunner) { - doFnRunner = doFnRunnerFactory.createRunner(); + for (RawUnionValue unionValue: inputs) { + final String tag = unionValue.getUnionTag(); + if (tag.equals("0")) { + // main input + pushedBackValues.add((WindowedValue) unionValue.getValue()); + } else { + // side input + PCollectionView sideInput = tagsToSideInputs.get(unionValue.getUnionTag()); + WindowedValue> sideInputValue = + (WindowedValue>) unionValue.getValue(); + if (!sideInputValues.containsKey(sideInput)) { + sideInputValues.put(sideInput, new LinkedList>>()); + } + sideInputValues.get(sideInput).add(sideInputValue); + } } - doFnRunner.startBundle(); - doFnRunner.processElement(value); + for (PCollectionView sideInput: sideInputs) { + if (sideInputValues.containsKey(sideInput)) { + for (WindowedValue> value: sideInputValues.get(sideInput)) { + sideInputReader.addSideInputValue(sideInput, value); + } + } + for (WindowedValue value : pushedBackValues) { + for (BoundedWindow win: value.getWindows()) { + BoundedWindow sideInputWindow = + sideInput.getWindowingStrategyInternal().getWindowFn().getSideInputWindow(win); + if (!sideInputReader.isReady(sideInput, sideInputWindow)) { + Object emptyValue = WindowedValue.of( + Lists.newArrayList(), value.getTimestamp(), sideInputWindow, value.getPane()); + sideInputReader.addSideInputValue(sideInput, (WindowedValue>) emptyValue); + } + } + } + } + + List> nextPushedBackValues = new LinkedList<>(); + for (WindowedValue value : pushedBackValues) { + Iterable> values = doFnRunner.processElementInReadyWindows(value); + Iterables.addAll(nextPushedBackValues, values); + } + pushedBackValues.clear(); + Iterables.addAll(pushedBackValues, nextPushedBackValues); + sideInputValues.clear(); + doFnRunner.finishBundle(); - return outputs.iterator(); + return outputManager.getOutputs(); } - @SuppressWarnings({"rawtypes", "unchecked"}) - @Override - public void output(TupleTag tag, WindowedValue output) { - if (mainTag.equals(tag)) { - outputs.add((WindowedValue) output); - } else { - throw new RuntimeException("output is not of main tag"); + private static class DoFnOutputManager implements DoFnRunners.OutputManager, Serializable { + + private static final long serialVersionUID = 4967375172737408160L; + private transient List outputs; + private transient Set> outputTags; + + @Override + public void output(TupleTag outputTag, WindowedValue output) { + if (outputTags.contains(outputTag)) { + outputs.add(new RawUnionValue(outputTag.getId(), output)); + } + } + + void setup(TupleTag mainOutput, List> sideOutputs) { + outputs = new LinkedList<>(); + outputTags = new HashSet<>(); + outputTags.add(mainOutput); + outputTags.addAll(sideOutputs); + } + + void clear() { + outputs.clear(); + } + + Iterator getOutputs() { + return outputs.iterator(); } } } diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/io/BoundedSourceWrapper.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/io/BoundedSourceWrapper.java index f8891017688a3..2c187355cbe55 100644 --- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/io/BoundedSourceWrapper.java +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/io/BoundedSourceWrapper.java @@ -29,6 +29,7 @@ */ public class BoundedSourceWrapper extends GearpumpSource { + private static final long serialVersionUID = 8199570485738786123L; private final BoundedSource source; public BoundedSourceWrapper(BoundedSource source, PipelineOptions options) { diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/io/GearpumpSource.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/io/GearpumpSource.java index 3d0d7c8905653..c079603429e20 100644 --- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/io/GearpumpSource.java +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/io/GearpumpSource.java @@ -28,10 +28,13 @@ 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.transforms.windowing.BoundedWindow; +import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.util.WindowedValue; import org.apache.gearpump.Message; import org.apache.gearpump.streaming.source.DataSource; +import org.apache.gearpump.streaming.source.Watermark; import org.apache.gearpump.streaming.task.TaskContext; /** @@ -74,11 +77,11 @@ public Message read() { if (available) { T data = reader.getCurrent(); org.joda.time.Instant timestamp = reader.getCurrentTimestamp(); - available = reader.advance(); message = Message.apply( - WindowedValue.valueInGlobalWindow(data), + WindowedValue.timestampedValueInGlobalWindow(data, timestamp), timestamp.getMillis()); } + available = reader.advance(); } catch (Exception e) { close(); throw new RuntimeException(e); @@ -100,11 +103,19 @@ public void close() { @Override public Instant getWatermark() { if (reader instanceof UnboundedSource.UnboundedReader) { - return TranslatorUtils.jodaTimeToJava8Time( - ((UnboundedSource.UnboundedReader) reader).getWatermark()); + org.joda.time.Instant watermark = + ((UnboundedSource.UnboundedReader) reader).getWatermark(); + if (watermark == BoundedWindow.TIMESTAMP_MAX_VALUE) { + return Watermark.MAX(); + } else { + return TranslatorUtils.jodaTimeToJava8Time(watermark); + } } else { - return Instant.now(); + if (available) { + return TranslatorUtils.jodaTimeToJava8Time(reader.getCurrentTimestamp()); + } else { + return Watermark.MAX(); + } } } - } diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/io/ValuesSource.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/io/ValuesSource.java index f5a5eb459a711..e0488cd571e18 100644 --- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/io/ValuesSource.java +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/io/ValuesSource.java @@ -33,6 +33,7 @@ import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; +import org.apache.beam.sdk.values.TimestampedValue; import org.joda.time.Instant; /** @@ -40,6 +41,7 @@ */ public class ValuesSource extends UnboundedSource { + private static final long serialVersionUID = 9113026175795235710L; private final byte[] values; private final IterableCoder iterableCoder; @@ -135,7 +137,7 @@ public T getCurrent() throws NoSuchElementException { @Override public Instant getCurrentTimestamp() throws NoSuchElementException { - return Instant.now(); + return getTimestamp(current); } @Override @@ -145,7 +147,7 @@ public void close() throws IOException { @Override public Instant getWatermark() { if (iterator.hasNext()) { - return Instant.now(); + return getTimestamp(current); } else { return BoundedWindow.TIMESTAMP_MAX_VALUE; } @@ -160,5 +162,13 @@ public CheckpointMark getCheckpointMark() { public UnboundedSource getCurrentSource() { return source; } + + private Instant getTimestamp(Object value) { + if (value instanceof TimestampedValue) { + return ((TimestampedValue) value).getTimestamp(); + } else { + return Instant.now(); + } + } } } diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/utils/DoFnRunnerFactory.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/utils/DoFnRunnerFactory.java index aaefb88a2748f..5db8320c1497f 100644 --- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/utils/DoFnRunnerFactory.java +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/utils/DoFnRunnerFactory.java @@ -19,18 +19,21 @@ package org.apache.beam.runners.gearpump.translators.utils; import java.io.Serializable; +import java.util.Collection; import java.util.List; 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.PushbackSideInputDoFnRunner; import org.apache.beam.runners.core.SimpleDoFnRunner; import org.apache.beam.runners.gearpump.GearpumpPipelineOptions; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.util.ExecutionContext; -import org.apache.beam.sdk.util.SideInputReader; +import org.apache.beam.sdk.util.ReadyCheckingSideInputReader; import org.apache.beam.sdk.util.WindowingStrategy; +import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TupleTag; /** @@ -38,10 +41,10 @@ */ public class DoFnRunnerFactory implements Serializable { - private static final long serialVersionUID = 1083167395296383469L; + private static final long serialVersionUID = -4109539010014189725L; private final DoFn fn; private final transient PipelineOptions options; - private final SideInputReader sideInputReader; + private final Collection> sideInputs; private final DoFnRunners.OutputManager outputManager; private final TupleTag mainOutputTag; private final List> sideOutputTags; @@ -52,7 +55,7 @@ public class DoFnRunnerFactory implements Serializable { public DoFnRunnerFactory( GearpumpPipelineOptions pipelineOptions, DoFn doFn, - SideInputReader sideInputReader, + Collection> sideInputs, DoFnRunners.OutputManager outputManager, TupleTag mainOutputTag, List> sideOutputTags, @@ -61,7 +64,7 @@ public DoFnRunnerFactory( WindowingStrategy windowingStrategy) { this.fn = doFn; this.options = pipelineOptions; - this.sideInputReader = sideInputReader; + this.sideInputs = sideInputs; this.outputManager = outputManager; this.mainOutputTag = mainOutputTag; this.sideOutputTags = sideOutputTags; @@ -70,9 +73,12 @@ public DoFnRunnerFactory( this.windowingStrategy = windowingStrategy; } - public DoFnRunner createRunner() { - return DoFnRunners.createDefault(options, fn, sideInputReader, outputManager, mainOutputTag, + public PushbackSideInputDoFnRunner createRunner( + ReadyCheckingSideInputReader sideInputReader) { + DoFnRunner underlying = DoFnRunners.createDefault( + options, fn, sideInputReader, outputManager, mainOutputTag, sideOutputTags, stepContext, aggregatorFactory, windowingStrategy); + return PushbackSideInputDoFnRunner.create(underlying, sideInputs, sideInputReader); } } diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/utils/TranslatorUtils.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/utils/TranslatorUtils.java index 656fc6a147d26..b8a5233ecb7fd 100644 --- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/utils/TranslatorUtils.java +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/utils/TranslatorUtils.java @@ -18,14 +18,28 @@ package org.apache.beam.runners.gearpump.translators.utils; +import com.google.common.collect.Lists; + import java.time.Instant; +import java.util.Collection; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import org.apache.beam.runners.gearpump.translators.TranslationContext; 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.WindowedValue; +import org.apache.beam.sdk.values.PCollectionView; + +import org.apache.gearpump.streaming.dsl.api.functions.FoldFunction; +import org.apache.gearpump.streaming.dsl.api.functions.MapFunction; +import org.apache.gearpump.streaming.dsl.javaapi.JavaStream; import org.apache.gearpump.streaming.dsl.window.impl.Window; + /** * Utility methods for translators. */ @@ -52,4 +66,137 @@ public static Window boundedWindowToGearpumpWindow(BoundedWindow window) { throw new RuntimeException("unknown window " + window.getClass().getName()); } } + + public static JavaStream withSideInputStream( + TranslationContext context, + JavaStream> inputStream, + Map> tagsToSideInputs) { + JavaStream mainStream = + inputStream.map(new ToRawUnionValue("0"), "map_to_RawUnionValue"); + + for (Map.Entry> tagToSideInput: tagsToSideInputs.entrySet()) { + // actually JavaStream>> + // check CreatePCollectionViewTranslator + JavaStream> sideInputStream = context.getInputStream( + tagToSideInput.getValue()); + mainStream = mainStream.merge(sideInputStream.map(new ToRawUnionValue<>( + tagToSideInput.getKey()), "map_to_RawUnionValue"), "merge_to_MainStream"); + } + return mainStream; + } + + public static Map> getTagsToSideInputs( + Collection> sideInputs) { + Map> tagsToSideInputs = new HashMap<>(); + // tag 0 is reserved for main input + int tag = 1; + for (PCollectionView sideInput: sideInputs) { + tagsToSideInputs.put(tag + "", sideInput); + tag++; + } + return tagsToSideInputs; + } + + public static JavaStream> toList(JavaStream stream) { + return stream.fold(new FoldFunction>() { + + @Override + public List init() { + return Lists.newArrayList(); + } + + @Override + public List fold(List accumulator, + RawUnionValue rawUnionValue) { + accumulator.add(rawUnionValue); + return accumulator; + } + }, "fold_to_iterable"); + } + + /** + * Converts @link{RawUnionValue} to @link{WindowedValue}. + */ + public static class FromRawUnionValue extends + MapFunction> { + + private static final long serialVersionUID = -4764968219713478955L; + + @Override + public WindowedValue map(RawUnionValue value) { + return (WindowedValue) value.getValue(); + } + } + + private static class ToRawUnionValue extends + MapFunction, RawUnionValue> { + + private static final long serialVersionUID = 8648852871014813583L; + private final String tag; + + ToRawUnionValue(String tag) { + this.tag = tag; + } + + @Override + public RawUnionValue map(WindowedValue windowedValue) { + return new RawUnionValue(tag, windowedValue); + } + } + + + /** + * This is copied from org.apache.beam.sdk.transforms.join.RawUnionValue. + */ + public static class RawUnionValue { + private final String unionTag; + private final Object value; + + /** + * Constructs a partial union from the given union tag and value. + */ + public RawUnionValue(String unionTag, Object value) { + this.unionTag = unionTag; + this.value = value; + } + + public String getUnionTag() { + return unionTag; + } + + public Object getValue() { + return value; + } + + @Override + public String toString() { + return unionTag + ":" + value; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + + RawUnionValue that = (RawUnionValue) o; + + if (unionTag != that.unionTag) { + return false; + } + return value != null ? value.equals(that.value) : that.value == null; + + } + + @Override + public int hashCode() { + int result = unionTag.hashCode(); + result = 31 * result + value.hashCode(); + return result; + } + } + } From 3dc8fc81d14d67318e26b99691f1583aacbd3509 Mon Sep 17 00:00:00 2001 From: manuzhang Date: Fri, 10 Mar 2017 16:50:01 +0800 Subject: [PATCH 147/346] enable ParDoTest --- runners/gearpump/pom.xml | 5 ----- 1 file changed, 5 deletions(-) diff --git a/runners/gearpump/pom.xml b/runners/gearpump/pom.xml index 6a41dc064a466..3efb1f6eb94b2 100644 --- a/runners/gearpump/pom.xml +++ b/runners/gearpump/pom.xml @@ -75,11 +75,6 @@ org.apache.beam:beam-sdks-java-core - - - org.apache.beam.sdk.transforms.ParDoTest - - [ From 3eab6a647e4761725680c8bc40589dfa5569d75b Mon Sep 17 00:00:00 2001 From: manuzhang Date: Tue, 14 Mar 2017 08:09:46 +0800 Subject: [PATCH 148/346] [BEAM-79] Fix gearpump-runner merge conflicts and test failure --- runners/gearpump/pom.xml | 51 ++- .../gearpump/GearpumpPipelineResult.java | 21 +- .../gearpump/GearpumpPipelineTranslator.java | 388 +++++++++++++++++- .../beam/runners/gearpump/GearpumpRunner.java | 376 +---------------- .../runners/gearpump/TestGearpumpRunner.java | 38 +- .../gearpump/examples/StreamingWordCount.java | 98 ----- .../examples/UnboundedTextSource.java | 139 ------- .../gearpump/examples/package-info.java | 22 - ...eateGearpumpPCollectionViewTranslator.java | 14 +- .../CreatePCollectionViewTranslator.java | 6 +- .../translators/CreateValuesTranslator.java | 51 --- ...ava => FlattenPCollectionsTranslator.java} | 15 +- .../translators/GroupByKeyTranslator.java | 4 +- .../ParDoBoundMultiTranslator.java | 32 +- .../translators/ParDoBoundTranslator.java | 7 +- .../translators/ReadBoundedTranslator.java | 4 +- .../translators/ReadUnboundedTranslator.java | 4 +- .../translators/TransformTranslator.java | 2 +- .../translators/TranslationContext.java | 29 +- ...lator.java => WindowAssignTranslator.java} | 12 +- .../translators/functions/DoFnFunction.java | 12 +- .../io/UnboundedSourceWrapper.java | 1 + .../translators/utils/DoFnRunnerFactory.java | 4 +- .../utils/NoOpAggregatorFactory.java | 2 +- .../translators/utils/NoOpStepContext.java | 6 +- .../translators/utils/TranslatorUtils.java | 2 - .../utils/TranslatorUtilsTest.java | 1 - 27 files changed, 533 insertions(+), 808 deletions(-) delete mode 100644 runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/examples/StreamingWordCount.java delete mode 100644 runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/examples/UnboundedTextSource.java delete mode 100644 runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/examples/package-info.java delete mode 100644 runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/CreateValuesTranslator.java rename runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/{FlattenPCollectionTranslator.java => FlattenPCollectionsTranslator.java} (86%) rename runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/{WindowBoundTranslator.java => WindowAssignTranslator.java} (88%) diff --git a/runners/gearpump/pom.xml b/runners/gearpump/pom.xml index 3efb1f6eb94b2..9a6a4323955ff 100644 --- a/runners/gearpump/pom.xml +++ b/runners/gearpump/pom.xml @@ -23,7 +23,7 @@ org.apache.beam beam-runners-parent - 0.5.0-incubating-SNAPSHOT + 0.7.0-SNAPSHOT ../pom.xml @@ -65,10 +65,12 @@ org.apache.beam.sdk.testing.RunnableOnService + org.apache.beam.sdk.testing.FlattenWithHeterogeneousCoders, org.apache.beam.sdk.testing.UsesStatefulParDo, org.apache.beam.sdk.testing.UsesTimersInParDo, org.apache.beam.sdk.testing.UsesSplittableParDo, - org.apache.beam.sdk.testing.UsesMetrics + org.apache.beam.sdk.testing.UsesAttemptedMetrics, + org.apache.beam.sdk.testing.UsesCommittedMetrics none true @@ -135,6 +137,16 @@ org.apache.beam beam-runners-core-java + + org.apache.beam + beam-runners-core-construction-java + + + org.slf4j + slf4j-jdk14 + + + joda-time joda-time @@ -181,6 +193,11 @@ + + com.fasterxml.jackson.dataformat + jackson-dataformat-yaml + test + org.mockito mockito-all @@ -210,8 +227,36 @@ - org.apache.maven.plugins maven-compiler-plugin + + 1.8 + 1.8 + 1.8 + 1.8 + + + + + org.apache.maven.plugins + maven-enforcer-plugin + + + enforce + + enforce + + + + + 1.8 + + + [1.8,) + + + + + diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpPipelineResult.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpPipelineResult.java index 8f908983ee3d0..d833cd69b532a 100644 --- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpPipelineResult.java +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpPipelineResult.java @@ -43,6 +43,7 @@ public class GearpumpPipelineResult implements PipelineResult { private final ClientContext client; private final RunningApplication app; + private boolean finished = false; public GearpumpPipelineResult(ClientContext client, RunningApplication app) { this.client = client; @@ -51,13 +52,22 @@ public GearpumpPipelineResult(ClientContext client, RunningApplication app) { @Override public State getState() { - return getGearpumpState(); + if (!finished) { + return getGearpumpState(); + } else { + return State.DONE; + } } @Override public State cancel() throws IOException { - app.shutDown(); - return State.CANCELLED; + if (!finished) { + app.shutDown(); + finished = true; + return State.CANCELLED; + } else { + return State.DONE; + } } @Override @@ -67,7 +77,10 @@ public State waitUntilFinish(Duration duration) { @Override public State waitUntilFinish() { - app.waitUntilFinish(); + if (!finished) { + app.waitUntilFinish(); + finished = true; + } return State.DONE; } diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpPipelineTranslator.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpPipelineTranslator.java index 4cc060c76fcda..1a36343fc3eb3 100644 --- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpPipelineTranslator.java +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpPipelineTranslator.java @@ -18,13 +18,19 @@ package org.apache.beam.runners.gearpump; +import com.google.common.collect.ImmutableMap; + +import java.util.ArrayList; +import java.util.Collections; import java.util.HashMap; +import java.util.List; import java.util.Map; +import org.apache.beam.runners.core.construction.PTransformMatchers; +import org.apache.beam.runners.core.construction.SingleInputOutputOverrideFactory; import org.apache.beam.runners.gearpump.translators.CreateGearpumpPCollectionViewTranslator; import org.apache.beam.runners.gearpump.translators.CreatePCollectionViewTranslator; -import org.apache.beam.runners.gearpump.translators.CreateValuesTranslator; -import org.apache.beam.runners.gearpump.translators.FlattenPCollectionTranslator; +import org.apache.beam.runners.gearpump.translators.FlattenPCollectionsTranslator; import org.apache.beam.runners.gearpump.translators.GroupByKeyTranslator; import org.apache.beam.runners.gearpump.translators.ParDoBoundMultiTranslator; import org.apache.beam.runners.gearpump.translators.ParDoBoundTranslator; @@ -32,17 +38,29 @@ import org.apache.beam.runners.gearpump.translators.ReadUnboundedTranslator; import org.apache.beam.runners.gearpump.translators.TransformTranslator; import org.apache.beam.runners.gearpump.translators.TranslationContext; -import org.apache.beam.runners.gearpump.translators.WindowBoundTranslator; +import org.apache.beam.runners.gearpump.translators.WindowAssignTranslator; import org.apache.beam.sdk.Pipeline; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.coders.CoderRegistry; +import org.apache.beam.sdk.coders.KvCoder; +import org.apache.beam.sdk.coders.ListCoder; import org.apache.beam.sdk.io.Read; +import org.apache.beam.sdk.runners.PTransformMatcher; +import org.apache.beam.sdk.runners.PTransformOverrideFactory; import org.apache.beam.sdk.runners.TransformHierarchy; -import org.apache.beam.sdk.transforms.Create; +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.PTransform; import org.apache.beam.sdk.transforms.ParDo; import org.apache.beam.sdk.transforms.View; import org.apache.beam.sdk.transforms.windowing.Window; +import org.apache.beam.sdk.util.InstanceBuilder; +import org.apache.beam.sdk.util.PCollectionViews; +import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.PValue; import org.apache.gearpump.util.Graph; @@ -74,14 +92,13 @@ public class GearpumpPipelineTranslator extends Pipeline.PipelineVisitor.Default registerTransformTranslator(Read.Unbounded.class, new ReadUnboundedTranslator()); registerTransformTranslator(Read.Bounded.class, new ReadBoundedTranslator()); registerTransformTranslator(GroupByKey.class, new GroupByKeyTranslator()); - registerTransformTranslator(Flatten.FlattenPCollectionList.class, - new FlattenPCollectionTranslator()); + registerTransformTranslator(Flatten.PCollections.class, + new FlattenPCollectionsTranslator()); registerTransformTranslator(ParDo.BoundMulti.class, new ParDoBoundMultiTranslator()); - registerTransformTranslator(Window.Bound.class, new WindowBoundTranslator()); - registerTransformTranslator(Create.Values.class, new CreateValuesTranslator()); + registerTransformTranslator(Window.Assign.class, new WindowAssignTranslator()); registerTransformTranslator(View.CreatePCollectionView.class, new CreatePCollectionViewTranslator()); - registerTransformTranslator(GearpumpRunner.CreateGearpumpPCollectionView.class, + registerTransformTranslator(CreateGearpumpPCollectionView.class, new CreateGearpumpPCollectionViewTranslator<>()); } @@ -90,6 +107,27 @@ public GearpumpPipelineTranslator(TranslationContext translationContext) { } public void translate(Pipeline pipeline) { + Map overrides = + ImmutableMap.builder() + .put(PTransformMatchers.classEqualTo(Combine.GloballyAsSingletonView.class), + new ReflectiveOneToOneOverrideFactory( + StreamingCombineGloballyAsSingletonView.class)) + .put(PTransformMatchers.classEqualTo(View.AsMap.class), + new ReflectiveOneToOneOverrideFactory(StreamingViewAsMap.class)) + .put(PTransformMatchers.classEqualTo(View.AsMultimap.class), + new ReflectiveOneToOneOverrideFactory(StreamingViewAsMultimap.class)) + .put(PTransformMatchers.classEqualTo(View.AsSingleton.class), + new ReflectiveOneToOneOverrideFactory(StreamingViewAsSingleton.class)) + .put(PTransformMatchers.classEqualTo(View.AsList.class), + new ReflectiveOneToOneOverrideFactory(StreamingViewAsList.class)) + .put(PTransformMatchers.classEqualTo(View.AsIterable.class), + new ReflectiveOneToOneOverrideFactory(StreamingViewAsIterable.class)) + .build(); + + for (Map.Entry override : + overrides.entrySet()) { + pipeline.replace(override.getKey(), override.getValue()); + } pipeline.traverseTopologically(this); } @@ -145,5 +183,337 @@ TransformTranslator getTransformTranslator(Class transfo return transformTranslators.get(transformClass); } + // The following codes are forked from DataflowRunner for View translator + private static class ReflectiveOneToOneOverrideFactory< + InputT extends PValue, + OutputT extends PValue, + TransformT extends PTransform> + extends SingleInputOutputOverrideFactory { + private final Class> replacement; + + private ReflectiveOneToOneOverrideFactory( + Class> replacement) { + this.replacement = replacement; + } + + @Override + public PTransform getReplacementTransform(TransformT transform) { + return InstanceBuilder.ofType(replacement) + .withArg((Class>) transform.getClass(), transform) + .build(); + } + } + + /** + * Specialized implementation for + * {@link org.apache.beam.sdk.transforms.View.AsMap View.AsMap} + * for the Gearpump runner. + */ + private static class StreamingViewAsMap + extends PTransform>, PCollectionView>> { + + private static final long serialVersionUID = 4791080760092950304L; + + public StreamingViewAsMap(View.AsMap transform) {} + + @Override + public PCollectionView> expand(PCollection> input) { + PCollectionView> view = + PCollectionViews.mapView( + input.getPipeline(), + input.getWindowingStrategy(), + input.getCoder()); + + @SuppressWarnings({"rawtypes", "unchecked"}) + KvCoder inputCoder = (KvCoder) input.getCoder(); + try { + inputCoder.getKeyCoder().verifyDeterministic(); + } catch (Coder.NonDeterministicException e) { + // throw new RuntimeException(e); + } + + return input + .apply(Combine.globally(new Concatenate>()).withoutDefaults()) + .apply(CreateGearpumpPCollectionView., Map>of(view)); + } + + @Override + protected String getKindString() { + return "StreamingViewAsMap"; + } + } + + /** + * Specialized expansion for {@link + * org.apache.beam.sdk.transforms.View.AsMultimap View.AsMultimap} for the + * Gearpump runner. + */ + private static class StreamingViewAsMultimap + extends PTransform>, PCollectionView>>> { + + private static final long serialVersionUID = 5854899081751333352L; + + public StreamingViewAsMultimap(View.AsMultimap transform) {} + + @Override + public PCollectionView>> expand(PCollection> input) { + PCollectionView>> view = + PCollectionViews.multimapView( + input.getPipeline(), + input.getWindowingStrategy(), + input.getCoder()); + + @SuppressWarnings({"rawtypes", "unchecked"}) + KvCoder inputCoder = (KvCoder) input.getCoder(); + try { + inputCoder.getKeyCoder().verifyDeterministic(); + } catch (Coder.NonDeterministicException e) { + // throw new RuntimeException(e); + } + + return input + .apply(Combine.globally(new Concatenate>()).withoutDefaults()) + .apply(CreateGearpumpPCollectionView., Map>>of(view)); + } + + @Override + protected String getKindString() { + return "StreamingViewAsMultimap"; + } + } + + /** + * Specialized implementation for + * {@link org.apache.beam.sdk.transforms.View.AsIterable View.AsIterable} for the + * Gearpump runner. + */ + private static class StreamingViewAsIterable + extends PTransform, PCollectionView>> { + + private static final long serialVersionUID = -3399860618995613421L; + + public StreamingViewAsIterable(View.AsIterable transform) {} + + @Override + public PCollectionView> expand(PCollection input) { + PCollectionView> view = + PCollectionViews.iterableView( + input.getPipeline(), + input.getWindowingStrategy(), + input.getCoder()); + + return input.apply(Combine.globally(new Concatenate()).withoutDefaults()) + .apply(CreateGearpumpPCollectionView.>of(view)); + } + + @Override + protected String getKindString() { + return "StreamingViewAsIterable"; + } + } + + /** + * Specialized implementation for + * {@link org.apache.beam.sdk.transforms.View.AsList View.AsList} for the + * Gearpump runner. + */ + private static class StreamingViewAsList + extends PTransform, PCollectionView>> { + + private static final long serialVersionUID = -5018631473886330629L; + + public StreamingViewAsList(View.AsList transform) {} + + @Override + public PCollectionView> expand(PCollection input) { + PCollectionView> view = + PCollectionViews.listView( + input.getPipeline(), + input.getWindowingStrategy(), + input.getCoder()); + + return input.apply(Combine.globally(new Concatenate()).withoutDefaults()) + .apply(CreateGearpumpPCollectionView.>of(view)); + } + + @Override + protected String getKindString() { + return "StreamingViewAsList"; + } + } + private static class StreamingCombineGloballyAsSingletonView + extends PTransform, PCollectionView> { + + private static final long serialVersionUID = 9064900748869035738L; + private final Combine.GloballyAsSingletonView transform; + + public StreamingCombineGloballyAsSingletonView( + Combine.GloballyAsSingletonView transform) { + this.transform = transform; + } + + @Override + public PCollectionView expand(PCollection input) { + PCollection combined = + input.apply(Combine.globally(transform.getCombineFn()) + .withoutDefaults() + .withFanout(transform.getFanout())); + + PCollectionView view = PCollectionViews.singletonView( + combined.getPipeline(), + combined.getWindowingStrategy(), + transform.getInsertDefault(), + transform.getInsertDefault() + ? transform.getCombineFn().defaultValue() : null, + combined.getCoder()); + return combined + .apply(ParDo.of(new WrapAsList())) + .apply(CreateGearpumpPCollectionView.of(view)); + } + + @Override + protected String getKindString() { + return "StreamingCombineGloballyAsSingletonView"; + } + } + + private static class StreamingViewAsSingleton + extends PTransform, PCollectionView> { + + private static final long serialVersionUID = 5870455965625071546L; + private final View.AsSingleton transform; + + public StreamingViewAsSingleton(View.AsSingleton transform) { + this.transform = transform; + } + + @Override + public PCollectionView expand(PCollection input) { + Combine.Globally combine = Combine.globally( + new SingletonCombine<>(transform.hasDefaultValue(), transform.defaultValue())); + if (!transform.hasDefaultValue()) { + combine = combine.withoutDefaults(); + } + return input.apply(combine.asSingletonView()); + } + + @Override + protected String getKindString() { + return "StreamingViewAsSingleton"; + } + + private static class SingletonCombine extends Combine.BinaryCombineFn { + private boolean hasDefaultValue; + private T defaultValue; + + SingletonCombine(boolean hasDefaultValue, T defaultValue) { + this.hasDefaultValue = hasDefaultValue; + this.defaultValue = defaultValue; + } + + @Override + public T apply(T left, T right) { + throw new IllegalArgumentException("PCollection with more than one element " + + "accessed as a singleton view. Consider using Combine.globally().asSingleton() to " + + "combine the PCollection into a single value"); + } + + @Override + public T identity() { + if (hasDefaultValue) { + return defaultValue; + } else { + throw new IllegalArgumentException( + "Empty PCollection accessed as a singleton view. " + + "Consider setting withDefault to provide a default value"); + } + } + } + } + + private static class WrapAsList extends DoFn> { + @ProcessElement + public void processElement(ProcessContext c) { + c.output(Collections.singletonList(c.element())); + } + } + /** + * Creates a primitive {@link PCollectionView}. + * + *

              For internal use only by runner implementors. + * + * @param The type of the elements of the input PCollection + * @param The type associated with the {@link PCollectionView} used as a side input + */ + public static class CreateGearpumpPCollectionView + extends PTransform>, PCollectionView> { + private static final long serialVersionUID = -2637073020800540542L; + private PCollectionView view; + + private CreateGearpumpPCollectionView(PCollectionView view) { + this.view = view; + } + + public static CreateGearpumpPCollectionView of( + PCollectionView view) { + return new CreateGearpumpPCollectionView<>(view); + } + + public PCollectionView getView() { + return view; + } + + @Override + public PCollectionView expand(PCollection> input) { + return view; + } + } + + /** + * Combiner that combines {@code T}s into a single {@code List} containing all inputs. + * + *

              For internal use by {@link StreamingViewAsMap}, {@link StreamingViewAsMultimap}, + * {@link StreamingViewAsList}, {@link StreamingViewAsIterable}. + * They require the input {@link PCollection} fits in memory. + * For a large {@link PCollection} this is expected to crash! + * + * @param the type of elements to concatenate. + */ + private static class Concatenate extends Combine.CombineFn, List> { + @Override + public List createAccumulator() { + return new ArrayList<>(); + } + + @Override + public List addInput(List accumulator, T input) { + accumulator.add(input); + return accumulator; + } + + @Override + public List mergeAccumulators(Iterable> accumulators) { + List result = createAccumulator(); + for (List accumulator : accumulators) { + result.addAll(accumulator); + } + return result; + } + + @Override + public List extractOutput(List accumulator) { + return accumulator; + } + + @Override + public Coder> getAccumulatorCoder(CoderRegistry registry, Coder inputCoder) { + return ListCoder.of(inputCoder); + } + + @Override + public Coder> getDefaultOutputCoder(CoderRegistry registry, Coder inputCoder) { + return ListCoder.of(inputCoder); + } + } } diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpRunner.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpRunner.java index 72f21260c8e8c..897467a6374ab 100644 --- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpRunner.java +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpRunner.java @@ -17,40 +17,18 @@ */ package org.apache.beam.runners.gearpump; -import com.google.common.collect.ImmutableMap; import com.typesafe.config.Config; import com.typesafe.config.ConfigValueFactory; -import java.util.ArrayList; -import java.util.Collections; import java.util.HashMap; -import java.util.List; import java.util.Map; import org.apache.beam.runners.gearpump.translators.TranslationContext; + import org.apache.beam.sdk.Pipeline; -import org.apache.beam.sdk.coders.Coder; -import org.apache.beam.sdk.coders.CoderRegistry; -import org.apache.beam.sdk.coders.KvCoder; -import org.apache.beam.sdk.coders.ListCoder; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.options.PipelineOptionsValidator; import org.apache.beam.sdk.runners.PipelineRunner; -import org.apache.beam.sdk.transforms.Combine; -import org.apache.beam.sdk.transforms.Create; -import org.apache.beam.sdk.transforms.DoFn; -import org.apache.beam.sdk.transforms.Flatten; -import org.apache.beam.sdk.transforms.PTransform; -import org.apache.beam.sdk.transforms.ParDo; -import org.apache.beam.sdk.transforms.View; -import org.apache.beam.sdk.util.InstanceBuilder; -import org.apache.beam.sdk.util.PCollectionViews; -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.PCollectionView; -import org.apache.beam.sdk.values.PInput; -import org.apache.beam.sdk.values.POutput; import org.apache.gearpump.cluster.ClusterConfig; import org.apache.gearpump.cluster.UserConfig; @@ -72,21 +50,8 @@ public class GearpumpRunner extends PipelineRunner { private static final String GEARPUMP_SERIALIZERS = "gearpump.serializers"; private static final String DEFAULT_APPNAME = "beam_gearpump_app"; - /** Custom transforms implementations. */ - private final Map, Class> overrides; - public GearpumpRunner(GearpumpPipelineOptions options) { this.options = options; - - ImmutableMap.Builder, Class> builder = ImmutableMap.builder(); - builder.put(Combine.GloballyAsSingletonView.class, - StreamingCombineGloballyAsSingletonView.class); - builder.put(View.AsMap.class, StreamingViewAsMap.class); - builder.put(View.AsMultimap.class, StreamingViewAsMultimap.class); - builder.put(View.AsSingleton.class, StreamingViewAsSingleton.class); - builder.put(View.AsList.class, StreamingViewAsList.class); - builder.put(View.AsIterable.class, StreamingViewAsIterable.class); - overrides = builder.build(); } public static GearpumpRunner fromOptions(PipelineOptions options) { @@ -95,31 +60,6 @@ public static GearpumpRunner fromOptions(PipelineOptions options) { return new GearpumpRunner(pipelineOptions); } - - public OutputT apply( - PTransform transform, InputT input) { - if (overrides.containsKey(transform.getClass())) { - - Class> transformClass = - (Class>) transform.getClass(); - - Class> customTransformClass = - (Class>) overrides.get(transform.getClass()); - - PTransform customTransform = - InstanceBuilder.ofType(customTransformClass) - .withArg(transformClass, transform) - .build(); - - return Pipeline.applyTransform(input, customTransform); - } else if (Flatten.FlattenPCollectionList.class.equals(transform.getClass()) - && ((PCollectionList) input).size() == 0) { - return (OutputT) Pipeline.applyTransform(input.getPipeline().begin(), Create.of()); - } else { - return super.apply(transform, input); - } - } - @Override public GearpumpPipelineResult run(Pipeline pipeline) { String appName = options.getApplicationName(); @@ -170,318 +110,4 @@ private Config registerSerializers(Config config, Map userSerial - // The following codes are forked from DataflowRunner for View translator - /** - * Specialized implementation for - * {@link org.apache.beam.sdk.transforms.View.AsMap View.AsMap} - * for the Gearpump runner. - */ - private static class StreamingViewAsMap - extends PTransform>, PCollectionView>> { - - private static final long serialVersionUID = 4791080760092950304L; - - public StreamingViewAsMap(View.AsMap transform) {} - - @Override - public PCollectionView> expand(PCollection> input) { - PCollectionView> view = - PCollectionViews.mapView( - input.getPipeline(), - input.getWindowingStrategy(), - input.getCoder()); - - @SuppressWarnings({"rawtypes", "unchecked"}) - KvCoder inputCoder = (KvCoder) input.getCoder(); - try { - inputCoder.getKeyCoder().verifyDeterministic(); - } catch (Coder.NonDeterministicException e) { - // throw new RuntimeException(e); - } - - return input - .apply(Combine.globally(new Concatenate>()).withoutDefaults()) - .apply(CreateGearpumpPCollectionView., Map>of(view)); - } - - @Override - protected String getKindString() { - return "StreamingViewAsMap"; - } - } - - /** - * Specialized expansion for {@link - * org.apache.beam.sdk.transforms.View.AsMultimap View.AsMultimap} for the - * Gearpump runner. - */ - private static class StreamingViewAsMultimap - extends PTransform>, PCollectionView>>> { - - private static final long serialVersionUID = 5854899081751333352L; - - public StreamingViewAsMultimap(View.AsMultimap transform) {} - - @Override - public PCollectionView>> expand(PCollection> input) { - PCollectionView>> view = - PCollectionViews.multimapView( - input.getPipeline(), - input.getWindowingStrategy(), - input.getCoder()); - - @SuppressWarnings({"rawtypes", "unchecked"}) - KvCoder inputCoder = (KvCoder) input.getCoder(); - try { - inputCoder.getKeyCoder().verifyDeterministic(); - } catch (Coder.NonDeterministicException e) { - // throw new RuntimeException(e); - } - - return input - .apply(Combine.globally(new Concatenate>()).withoutDefaults()) - .apply(CreateGearpumpPCollectionView., Map>>of(view)); - } - - @Override - protected String getKindString() { - return "StreamingViewAsMultimap"; - } - } - - /** - * Specialized implementation for - * {@link org.apache.beam.sdk.transforms.View.AsIterable View.AsIterable} for the - * Gearpump runner. - */ - private static class StreamingViewAsIterable - extends PTransform, PCollectionView>> { - - private static final long serialVersionUID = -3399860618995613421L; - - public StreamingViewAsIterable(View.AsIterable transform) {} - - @Override - public PCollectionView> expand(PCollection input) { - PCollectionView> view = - PCollectionViews.iterableView( - input.getPipeline(), - input.getWindowingStrategy(), - input.getCoder()); - - return input.apply(Combine.globally(new Concatenate()).withoutDefaults()) - .apply(CreateGearpumpPCollectionView.>of(view)); - } - - @Override - protected String getKindString() { - return "StreamingViewAsIterable"; - } - } - - /** - * Specialized implementation for - * {@link org.apache.beam.sdk.transforms.View.AsList View.AsList} for the - * Gearpump runner. - */ - private static class StreamingViewAsList - extends PTransform, PCollectionView>> { - - private static final long serialVersionUID = -5018631473886330629L; - - public StreamingViewAsList(View.AsList transform) {} - - @Override - public PCollectionView> expand(PCollection input) { - PCollectionView> view = - PCollectionViews.listView( - input.getPipeline(), - input.getWindowingStrategy(), - input.getCoder()); - - return input.apply(Combine.globally(new Concatenate()).withoutDefaults()) - .apply(CreateGearpumpPCollectionView.>of(view)); - } - - @Override - protected String getKindString() { - return "StreamingViewAsList"; - } - } - private static class StreamingCombineGloballyAsSingletonView - extends PTransform, PCollectionView> { - - private static final long serialVersionUID = 9064900748869035738L; - private final Combine.GloballyAsSingletonView transform; - - public StreamingCombineGloballyAsSingletonView( - Combine.GloballyAsSingletonView transform) { - this.transform = transform; - } - - @Override - public PCollectionView expand(PCollection input) { - PCollection combined = - input.apply(Combine.globally(transform.getCombineFn()) - .withoutDefaults() - .withFanout(transform.getFanout())); - - PCollectionView view = PCollectionViews.singletonView( - combined.getPipeline(), - combined.getWindowingStrategy(), - transform.getInsertDefault(), - transform.getInsertDefault() - ? transform.getCombineFn().defaultValue() : null, - combined.getCoder()); - return combined - .apply(ParDo.of(new WrapAsList())) - .apply(CreateGearpumpPCollectionView.of(view)); - } - - @Override - protected String getKindString() { - return "StreamingCombineGloballyAsSingletonView"; - } - } - - private static class StreamingViewAsSingleton - extends PTransform, PCollectionView> { - - private static final long serialVersionUID = 5870455965625071546L; - private final View.AsSingleton transform; - - public StreamingViewAsSingleton(View.AsSingleton transform) { - this.transform = transform; - } - - @Override - public PCollectionView expand(PCollection input) { - Combine.Globally combine = Combine.globally( - new SingletonCombine<>(transform.hasDefaultValue(), transform.defaultValue())); - if (!transform.hasDefaultValue()) { - combine = combine.withoutDefaults(); - } - return input.apply(combine.asSingletonView()); - } - - @Override - protected String getKindString() { - return "StreamingViewAsSingleton"; - } - - private static class SingletonCombine extends Combine.BinaryCombineFn { - private boolean hasDefaultValue; - private T defaultValue; - - SingletonCombine(boolean hasDefaultValue, T defaultValue) { - this.hasDefaultValue = hasDefaultValue; - this.defaultValue = defaultValue; - } - - @Override - public T apply(T left, T right) { - throw new IllegalArgumentException("PCollection with more than one element " - + "accessed as a singleton view. Consider using Combine.globally().asSingleton() to " - + "combine the PCollection into a single value"); - } - - @Override - public T identity() { - if (hasDefaultValue) { - return defaultValue; - } else { - throw new IllegalArgumentException( - "Empty PCollection accessed as a singleton view. " - + "Consider setting withDefault to provide a default value"); - } - } - } - } - - private static class WrapAsList extends DoFn> { - @ProcessElement - public void processElement(ProcessContext c) { - c.output(Collections.singletonList(c.element())); - } - } - - /** - * Creates a primitive {@link PCollectionView}. - * - *

              For internal use only by runner implementors. - * - * @param The type of the elements of the input PCollection - * @param The type associated with the {@link PCollectionView} used as a side input - */ - public static class CreateGearpumpPCollectionView - extends PTransform>, PCollectionView> { - private static final long serialVersionUID = -2637073020800540542L; - private PCollectionView view; - - private CreateGearpumpPCollectionView(PCollectionView view) { - this.view = view; - } - - public static CreateGearpumpPCollectionView of( - PCollectionView view) { - return new CreateGearpumpPCollectionView<>(view); - } - - public PCollectionView getView() { - return view; - } - - @Override - public PCollectionView expand(PCollection> input) { - return view; - } - } - - /** - * Combiner that combines {@code T}s into a single {@code List} containing all inputs. - * - *

              For internal use by {@link StreamingViewAsMap}, {@link StreamingViewAsMultimap}, - * {@link StreamingViewAsList}, {@link StreamingViewAsIterable}. - * They require the input {@link PCollection} fits in memory. - * For a large {@link PCollection} this is expected to crash! - * - * @param the type of elements to concatenate. - */ - private static class Concatenate extends Combine.CombineFn, List> { - @Override - public List createAccumulator() { - return new ArrayList<>(); - } - - @Override - public List addInput(List accumulator, T input) { - accumulator.add(input); - return accumulator; - } - - @Override - public List mergeAccumulators(Iterable> accumulators) { - List result = createAccumulator(); - for (List accumulator : accumulators) { - result.addAll(accumulator); - } - return result; - } - - @Override - public List extractOutput(List accumulator) { - return accumulator; - } - - @Override - public Coder> getAccumulatorCoder(CoderRegistry registry, Coder inputCoder) { - return ListCoder.of(inputCoder); - } - - @Override - public Coder> getDefaultOutputCoder(CoderRegistry registry, Coder inputCoder) { - return ListCoder.of(inputCoder); - } - } - } diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/TestGearpumpRunner.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/TestGearpumpRunner.java index c96bcb1f1df4f..ea7dd26a234d2 100644 --- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/TestGearpumpRunner.java +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/TestGearpumpRunner.java @@ -24,9 +24,6 @@ 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.transforms.PTransform; -import org.apache.beam.sdk.values.PInput; -import org.apache.beam.sdk.values.POutput; import org.apache.gearpump.cluster.ClusterConfig; import org.apache.gearpump.cluster.embedded.EmbeddedCluster; @@ -58,36 +55,9 @@ public static TestGearpumpRunner fromOptions(PipelineOptions options) { @Override public GearpumpPipelineResult run(Pipeline pipeline) { - try { - GearpumpPipelineResult result = delegate.run(pipeline); - result.waitUntilFinish(); - cluster.stop(); - return result; - } catch (Throwable e) { - // copied from TestFlinkRunner to pull out AssertionError - // which is wrapped in UserCodeException - Throwable cause = e; - Throwable oldCause; - do { - if (cause.getCause() == null) { - break; - } - - oldCause = cause; - cause = cause.getCause(); - - } while (!oldCause.equals(cause)); - if (cause instanceof AssertionError) { - throw (AssertionError) cause; - } else { - throw e; - } - } - } - - @Override - public - OutputT apply(PTransform transform, InputT input) { - return delegate.apply(transform, input); + GearpumpPipelineResult result = delegate.run(pipeline); + result.waitUntilFinish(); + cluster.stop(); + return result; } } diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/examples/StreamingWordCount.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/examples/StreamingWordCount.java deleted file mode 100644 index b2d762a200eef..0000000000000 --- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/examples/StreamingWordCount.java +++ /dev/null @@ -1,98 +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.gearpump.examples; - -import org.apache.beam.runners.gearpump.GearpumpPipelineOptions; -import org.apache.beam.runners.gearpump.GearpumpRunner; -import org.apache.beam.sdk.Pipeline; -import org.apache.beam.sdk.io.Read; -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.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.apache.gearpump.cluster.client.ClientContext; -import org.joda.time.Duration; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - - -/** - * streaming word count example on Gearpump runner. - */ -public class StreamingWordCount { - - static class ExtractWordsFn extends DoFn { - - @ProcessElement - public void process(ProcessContext c) { - // Split the line into words. - String[] words = c.element().split("[^a-zA-Z']+"); - - // Output each word encountered into the output PCollection. - for (String word : words) { - if (!word.isEmpty()) { - c.output(word); - } - } - } - } - - static class FormatAsStringFn extends DoFn, String> { - private static final Logger LOG = LoggerFactory.getLogger(FormatAsStringFn.class); - - @ProcessElement - public void process(ProcessContext c) { - String row = c.element().getKey() - + " - " + c.element().getValue() - + " @ " + c.timestamp().toString(); - LOG.debug("output {}", row); - c.output(row); - } - } - - - public static void main(String[] args) { - GearpumpPipelineOptions options = PipelineOptionsFactory - .fromArgs(args).as(GearpumpPipelineOptions.class); - options.setRunner(GearpumpRunner.class); - options.setApplicationName("StreamingWordCount"); - options.setParallelism(1); - - Pipeline p = Pipeline.create(options); - - PCollection> wordCounts = - p.apply(Read.from(new UnboundedTextSource())) - .apply(ParDo.of(new ExtractWordsFn())) - .apply(Window.into(FixedWindows.of(Duration.standardSeconds(10)))) - .apply(Count.perElement()); - - wordCounts.apply(ParDo.of(new FormatAsStringFn())); - - p.run(); - - ClientContext clientContext = options.getClientContext(); - clientContext.close(); - - } -} diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/examples/UnboundedTextSource.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/examples/UnboundedTextSource.java deleted file mode 100644 index b01443273287f..0000000000000 --- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/examples/UnboundedTextSource.java +++ /dev/null @@ -1,139 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.beam.runners.gearpump.examples; - -import java.io.IOException; -import java.io.Serializable; -import java.util.Collections; -import java.util.List; -import java.util.NoSuchElementException; - -import javax.annotation.Nullable; - -import org.apache.beam.sdk.coders.Coder; -import org.apache.beam.sdk.coders.StringUtf8Coder; -import org.apache.beam.sdk.io.UnboundedSource; -import org.apache.beam.sdk.options.PipelineOptions; - -import org.joda.time.Instant; - - -/** - * unbounded source that reads from text. - */ -public class UnboundedTextSource extends UnboundedSource { - - @Override - public List> generateInitialSplits( - int desiredNumSplits, PipelineOptions options) throws Exception { - return Collections.>singletonList(this); - } - - @Override - public UnboundedReader createReader(PipelineOptions options, - @Nullable CheckpointMark checkpointMark) { - return new UnboundedTextReader(this); - } - - @Nullable - @Override - public Coder getCheckpointMarkCoder() { - return null; - } - - @Override - public void validate() { - } - - @Override - public Coder getDefaultOutputCoder() { - return StringUtf8Coder.of(); - } - - /** - * reads from text. - */ - public static class UnboundedTextReader extends UnboundedReader implements Serializable { - - private static final long serialVersionUID = 7526472295622776147L; - - private final UnboundedTextSource source; - - private final String[] texts = new String[]{"foo foo foo bar bar", "foo foo bar bar bar"}; - private long index = 0; - - private String currentRecord; - - private Instant currentTimestamp; - - public UnboundedTextReader(UnboundedTextSource source) { - this.source = source; - } - - @Override - public boolean start() throws IOException { - currentRecord = texts[0]; - currentTimestamp = new Instant(0); - return true; - } - - @Override - public boolean advance() throws IOException { - index++; - currentRecord = texts[(int) index % (texts.length)]; - currentTimestamp = new Instant(index * 1000); - - return true; - } - - @Override - public byte[] getCurrentRecordId() throws NoSuchElementException { - return new byte[0]; - } - - @Override - public String getCurrent() throws NoSuchElementException { - return this.currentRecord; - } - - @Override - public Instant getCurrentTimestamp() throws NoSuchElementException { - return currentTimestamp; - } - - @Override - public void close() throws IOException { - } - - @Override - public Instant getWatermark() { - return currentTimestamp; - } - - @Override - public CheckpointMark getCheckpointMark() { - return null; - } - - @Override - public UnboundedSource getCurrentSource() { - return this.source; - } - } -} diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/examples/package-info.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/examples/package-info.java deleted file mode 100644 index a62a6c08a2582..0000000000000 --- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/examples/package-info.java +++ /dev/null @@ -1,22 +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. - */ - -/** - * Examples showcase Beam application over Gearpump runner. - */ -package org.apache.beam.runners.gearpump.examples; diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/CreateGearpumpPCollectionViewTranslator.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/CreateGearpumpPCollectionViewTranslator.java index d05c89dc3ab96..c7f24a86ec51e 100644 --- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/CreateGearpumpPCollectionViewTranslator.java +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/CreateGearpumpPCollectionViewTranslator.java @@ -20,25 +20,27 @@ import java.util.List; -import org.apache.beam.runners.gearpump.GearpumpRunner; +import org.apache.beam.runners.gearpump.GearpumpPipelineTranslator; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.PCollectionView; import org.apache.gearpump.streaming.dsl.javaapi.JavaStream; - /** * CreateGearpumpPCollectionView bridges input stream to down stream * transforms. */ public class CreateGearpumpPCollectionViewTranslator implements - TransformTranslator> { + TransformTranslator> { + + private static final long serialVersionUID = -3955521308055056034L; @Override - public void translate(GearpumpRunner.CreateGearpumpPCollectionView transform, + public void translate( + GearpumpPipelineTranslator.CreateGearpumpPCollectionView transform, TranslationContext context) { JavaStream>> inputStream = - context.getInputStream(context.getInput(transform)); - PCollectionView view = transform.getView(); + context.getInputStream(context.getInput()); + PCollectionView view = (PCollectionView) context.getOutput(); context.setOutputStream(view, inputStream); } } diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/CreatePCollectionViewTranslator.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/CreatePCollectionViewTranslator.java index e9e2e5d0c58fe..da55d705a18fc 100644 --- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/CreatePCollectionViewTranslator.java +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/CreatePCollectionViewTranslator.java @@ -32,12 +32,14 @@ public class CreatePCollectionViewTranslator implements TransformTranslator> { + private static final long serialVersionUID = -2394386873317515748L; + @Override public void translate(View.CreatePCollectionView transform, TranslationContext context) { JavaStream>> inputStream = - context.getInputStream(context.getInput(transform)); - PCollectionView view = transform.getView(); + context.getInputStream(context.getInput()); + PCollectionView view = (PCollectionView) context.getOutput(); context.setOutputStream(view, inputStream); } } diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/CreateValuesTranslator.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/CreateValuesTranslator.java deleted file mode 100644 index e5dc6dd096c20..0000000000000 --- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/CreateValuesTranslator.java +++ /dev/null @@ -1,51 +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.gearpump.translators; - -import org.apache.beam.runners.gearpump.translators.io.UnboundedSourceWrapper; -import org.apache.beam.runners.gearpump.translators.io.ValuesSource; -import org.apache.beam.sdk.coders.CannotProvideCoderException; -import org.apache.beam.sdk.io.UnboundedSource; -import org.apache.beam.sdk.transforms.Create; -import org.apache.beam.sdk.util.WindowedValue; - -import org.apache.gearpump.streaming.dsl.javaapi.JavaStream; - -/** - * Wraps elements from Create.Values into an {@link UnboundedSource}. - * mainly used for test - */ -public class CreateValuesTranslator implements TransformTranslator> { - - private static final long serialVersionUID = 5411841848199229738L; - - @Override - public void translate(Create.Values transform, TranslationContext context) { - try { - UnboundedSourceWrapper unboundedSourceWrapper = new UnboundedSourceWrapper<>( - new ValuesSource<>(transform.getElements(), - transform.getDefaultOutputCoder(context.getInput(transform))), - context.getPipelineOptions()); - JavaStream> sourceStream = context.getSourceStream(unboundedSourceWrapper); - context.setOutputStream(context.getOutput(transform), sourceStream); - } catch (CannotProvideCoderException e) { - throw new RuntimeException(e); - } - } -} diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/FlattenPCollectionTranslator.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/FlattenPCollectionsTranslator.java similarity index 86% rename from runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/FlattenPCollectionTranslator.java rename to runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/FlattenPCollectionsTranslator.java index 27e54b8f436fd..3a465cbef84ec 100644 --- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/FlattenPCollectionTranslator.java +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/FlattenPCollectionsTranslator.java @@ -29,25 +29,24 @@ import org.apache.beam.sdk.transforms.Flatten; import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.TaggedPValue; import org.apache.gearpump.streaming.dsl.api.functions.MapFunction; import org.apache.gearpump.streaming.dsl.javaapi.JavaStream; - - /** * Flatten.FlattenPCollectionList is translated to Gearpump merge function. - * Note only two-way merge is working now */ -public class FlattenPCollectionTranslator implements - TransformTranslator> { +public class FlattenPCollectionsTranslator implements + TransformTranslator> { private static final long serialVersionUID = -5552148802472944759L; @Override - public void translate(Flatten.FlattenPCollectionList transform, TranslationContext context) { + public void translate(Flatten.PCollections transform, TranslationContext context) { JavaStream merged = null; Set> unique = new HashSet<>(); - for (PCollection collection : context.getInput(transform).getAll()) { + for (TaggedPValue input: context.getInputs()) { + PCollection collection = (PCollection) input.getValue(); unique.add(collection); JavaStream inputStream = context.getInputStream(collection); if (null == merged) { @@ -69,7 +68,7 @@ public void translate(Flatten.FlattenPCollectionList transform, TranslationCo StringUtf8Coder.of()), context.getPipelineOptions()); merged = context.getSourceStream(unboundedSourceWrapper); } - context.setOutputStream(context.getOutput(transform), merged); + context.setOutputStream(context.getOutput(), merged); } private static class DummyFunction extends MapFunction { diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/GroupByKeyTranslator.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/GroupByKeyTranslator.java index df8bfe95988c8..5dfd3e9d1794f 100644 --- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/GroupByKeyTranslator.java +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/GroupByKeyTranslator.java @@ -61,7 +61,7 @@ public class GroupByKeyTranslator implements TransformTranslator transform, TranslationContext context) { - PCollection> input = context.getInput(transform); + PCollection> input = (PCollection>) context.getInput(); Coder inputKeyCoder = ((KvCoder) input.getCoder()).getKeyCoder(); JavaStream>> inputStream = context.getInputStream(input); @@ -80,7 +80,7 @@ public void translate(GroupByKey transform, TranslationContext context) { .fold(new Merge<>(windowFn, outputTimeFn), "merge") .map(new Values(), "values"); - context.setOutputStream(context.getOutput(transform), outputStream); + context.setOutputStream(context.getOutput(), outputStream); } private static class GearpumpWindowFn diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/ParDoBoundMultiTranslator.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/ParDoBoundMultiTranslator.java index 8c57019e39518..e88cb7364d5fc 100644 --- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/ParDoBoundMultiTranslator.java +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/ParDoBoundMultiTranslator.java @@ -18,17 +18,11 @@ package org.apache.beam.runners.gearpump.translators; - -import com.google.common.base.Predicate; -import com.google.common.collect.Lists; -import com.google.common.collect.Sets; - +import java.util.ArrayList; import java.util.Collection; import java.util.List; import java.util.Map; -import javax.annotation.Nullable; - import org.apache.beam.runners.gearpump.translators.functions.DoFnFunction; import org.apache.beam.runners.gearpump.translators.utils.TranslatorUtils; import org.apache.beam.sdk.transforms.DoFn; @@ -36,6 +30,7 @@ import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionView; +import org.apache.beam.sdk.values.TaggedPValue; import org.apache.beam.sdk.values.TupleTag; import org.apache.gearpump.streaming.dsl.api.functions.FilterFunction; @@ -54,21 +49,21 @@ public class ParDoBoundMultiTranslator implements @Override public void translate(ParDo.BoundMulti transform, TranslationContext context) { - PCollection inputT = (PCollection) context.getInput(transform); + PCollection inputT = (PCollection) context.getInput(); JavaStream> inputStream = context.getInputStream(inputT); Collection> sideInputs = transform.getSideInputs(); Map> tagsToSideInputs = TranslatorUtils.getTagsToSideInputs(sideInputs); - Map, PCollection> outputs = context.getOutput(transform).getAll(); + List outputs = context.getOutputs(); final TupleTag mainOutput = transform.getMainOutputTag(); - List> sideOutputs = Lists.newLinkedList(Sets.filter(outputs.keySet(), - new Predicate>() { - @Override - public boolean apply(@Nullable TupleTag tupleTag) { - return tupleTag != null && !tupleTag.getId().equals(mainOutput.getId()); - } - })); + List> sideOutputs = new ArrayList<>(outputs.size() - 1); + for (TaggedPValue output: outputs) { + TupleTag tag = output.getTag(); + if (tag != null && !tag.getId().equals(mainOutput.getId())) { + sideOutputs.add(tag); + } + } JavaStream unionStream = TranslatorUtils.withSideInputStream( context, inputStream, tagsToSideInputs); @@ -83,10 +78,9 @@ public boolean apply(@Nullable TupleTag tupleTag) { tagsToSideInputs, mainOutput, sideOutputs), transform.getName()); - for (Map.Entry, PCollection> output: outputs.entrySet()) { - output.getValue().getCoder(); + for (TaggedPValue output: outputs) { JavaStream> taggedStream = outputStream - .filter(new FilterByOutputTag(output.getKey().getId()), + .filter(new FilterByOutputTag(output.getTag().getId()), "filter_by_output_tag") .map(new TranslatorUtils.FromRawUnionValue(), "from_RawUnionValue"); context.setOutputStream(output.getValue(), taggedStream); diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/ParDoBoundTranslator.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/ParDoBoundTranslator.java index efae9388d46e6..dc32b8c8a3bb7 100644 --- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/ParDoBoundTranslator.java +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/ParDoBoundTranslator.java @@ -35,7 +35,6 @@ import org.apache.beam.sdk.values.TupleTagList; import org.apache.gearpump.streaming.dsl.javaapi.JavaStream; - /** * {@link ParDo.Bound} is translated to Gearpump flatMap function * with {@link DoFn} wrapped in {@link DoFnFunction}. @@ -50,14 +49,14 @@ public class ParDoBoundTranslator implements @Override public void translate(ParDo.Bound transform, TranslationContext context) { DoFn doFn = transform.getFn(); - PCollection output = context.getOutput(transform); + PCollection output = (PCollection) context.getOutput(); WindowingStrategy windowingStrategy = output.getWindowingStrategy(); Collection> sideInputs = transform.getSideInputs(); Map> tagsToSideInputs = TranslatorUtils.getTagsToSideInputs(sideInputs); JavaStream> inputStream = context.getInputStream( - context.getInput(transform)); + context.getInput()); JavaStream unionStream = TranslatorUtils.withSideInputStream(context, inputStream, tagsToSideInputs); @@ -71,6 +70,6 @@ public void translate(ParDo.Bound transform, TranslationContext .flatMap(doFnFunction, transform.getName()) .map(new TranslatorUtils.FromRawUnionValue(), "from_RawUnionValue"); - context.setOutputStream(context.getOutput(transform), outputStream); + context.setOutputStream(context.getOutput(), outputStream); } } diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/ReadBoundedTranslator.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/ReadBoundedTranslator.java index 478d58f87b1ee..8f71a8e8e52c0 100644 --- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/ReadBoundedTranslator.java +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/ReadBoundedTranslator.java @@ -31,6 +31,8 @@ */ public class ReadBoundedTranslator implements TransformTranslator> { + private static final long serialVersionUID = -3899020490896998330L; + @Override public void translate(Read.Bounded transform, TranslationContext context) { BoundedSource boundedSource = transform.getSource(); @@ -38,7 +40,7 @@ public void translate(Read.Bounded transform, TranslationContext context) { context.getPipelineOptions()); JavaStream> sourceStream = context.getSourceStream(sourceWrapper); - context.setOutputStream(context.getOutput(transform), sourceStream); + context.setOutputStream(context.getOutput(), sourceStream); } } diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/ReadUnboundedTranslator.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/ReadUnboundedTranslator.java index 7e12a9c629fdb..0462c57e1f01e 100644 --- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/ReadUnboundedTranslator.java +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/ReadUnboundedTranslator.java @@ -33,6 +33,8 @@ public class ReadUnboundedTranslator implements TransformTranslator> { + private static final long serialVersionUID = 3529494817859948619L; + @Override public void translate(Read.Unbounded transform, TranslationContext context) { UnboundedSource unboundedSource = transform.getSource(); @@ -40,7 +42,7 @@ public void translate(Read.Unbounded transform, TranslationContext context) { unboundedSource, context.getPipelineOptions()); JavaStream> sourceStream = context.getSourceStream(unboundedSourceWrapper); - context.setOutputStream(context.getOutput(transform), sourceStream); + context.setOutputStream(context.getOutput(), sourceStream); } } diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/TransformTranslator.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/TransformTranslator.java index c8587d3914298..c7becadc9eb13 100644 --- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/TransformTranslator.java +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/TransformTranslator.java @@ -23,7 +23,7 @@ import org.apache.beam.sdk.transforms.PTransform; /** - * translates {@link PTransform} to Gearpump functions. + * Translates {@link PTransform} to Gearpump functions. */ public interface TransformTranslator extends Serializable { void translate(T transform, TranslationContext context); diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/TranslationContext.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/TranslationContext.java index b2cff8a9b184b..e88bb7409bf87 100644 --- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/TranslationContext.java +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/TranslationContext.java @@ -20,17 +20,18 @@ import static com.google.common.base.Preconditions.checkArgument; +import com.google.common.collect.Iterables; + import java.util.HashMap; +import java.util.List; import java.util.Map; import org.apache.beam.runners.gearpump.GearpumpPipelineOptions; 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.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.gearpump.cluster.UserConfig; import org.apache.gearpump.streaming.dsl.javaapi.JavaStream; import org.apache.gearpump.streaming.dsl.javaapi.JavaStreamApp; @@ -70,18 +71,26 @@ public void setOutputStream(PValue output, JavaStream outputS } } - public InputT getInput(PTransform transform) { - return (InputT) getCurrentTransform(transform).getInput(); + public List getInputs() { + return getCurrentTransform().getInputs(); + } + + public PValue getInput() { + return Iterables.getOnlyElement(getInputs()).getValue(); + } + + public List getOutputs() { + return getCurrentTransform().getOutputs(); } - public OutputT getOutput(PTransform transform) { - return (OutputT) getCurrentTransform(transform).getOutput(); + public PValue getOutput() { + return Iterables.getOnlyElement(getOutputs()).getValue(); } - private AppliedPTransform getCurrentTransform(PTransform transform) { + private AppliedPTransform getCurrentTransform() { checkArgument( - currentTransform != null && currentTransform.getTransform() == transform, - "can only be called with current transform"); + currentTransform != null, + "current transform not set"); return currentTransform; } diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/WindowBoundTranslator.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/WindowAssignTranslator.java similarity index 88% rename from runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/WindowBoundTranslator.java rename to runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/WindowAssignTranslator.java index 81970e27b9945..fe6015ae4a4b6 100644 --- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/WindowBoundTranslator.java +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/WindowAssignTranslator.java @@ -39,22 +39,22 @@ * {@link Window.Bound} is translated to Gearpump flatMap function. */ @SuppressWarnings("unchecked") -public class WindowBoundTranslator implements TransformTranslator> { +public class WindowAssignTranslator implements TransformTranslator> { private static final long serialVersionUID = -964887482120489061L; @Override - public void translate(Window.Bound transform, TranslationContext context) { - PCollection input = context.getInput(transform); + public void translate(Window.Assign transform, TranslationContext context) { + PCollection input = (PCollection) context.getInput(); + PCollection output = (PCollection) context.getOutput(); JavaStream> inputStream = context.getInputStream(input); - WindowingStrategy outputStrategy = - transform.getOutputStrategyInternal(input.getWindowingStrategy()); + WindowingStrategy outputStrategy = output.getWindowingStrategy(); WindowFn windowFn = (WindowFn) outputStrategy.getWindowFn(); JavaStream> outputStream = inputStream .flatMap(new AssignWindows(windowFn), "assign_windows"); - context.setOutputStream(context.getOutput(transform), outputStream); + context.setOutputStream(output, outputStream); } private static class AssignWindows extends diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/functions/DoFnFunction.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/functions/DoFnFunction.java index b2c68d6a8c07a..9941e71338d77 100644 --- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/functions/DoFnFunction.java +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/functions/DoFnFunction.java @@ -33,6 +33,7 @@ import java.util.Set; import org.apache.beam.runners.core.DoFnRunners; +import org.apache.beam.runners.core.InMemoryStateInternals; import org.apache.beam.runners.core.PushbackSideInputDoFnRunner; import org.apache.beam.runners.core.SideInputHandler; import org.apache.beam.runners.gearpump.GearpumpPipelineOptions; @@ -48,7 +49,6 @@ import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.WindowingStrategy; -import org.apache.beam.sdk.util.state.InMemoryStateInternals; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TupleTag; import org.apache.gearpump.streaming.dsl.javaapi.functions.FlatMapFunction; @@ -134,12 +134,16 @@ public Iterator flatMap(List input } else { // side input PCollectionView sideInput = tagsToSideInputs.get(unionValue.getUnionTag()); - WindowedValue> sideInputValue = - (WindowedValue>) unionValue.getValue(); + WindowedValue sideInputValue = + (WindowedValue) unionValue.getValue(); + Object value = sideInputValue.getValue(); + if (!(value instanceof Iterable)) { + sideInputValue = sideInputValue.withValue(Lists.newArrayList(value)); + } if (!sideInputValues.containsKey(sideInput)) { sideInputValues.put(sideInput, new LinkedList>>()); } - sideInputValues.get(sideInput).add(sideInputValue); + sideInputValues.get(sideInput).add((WindowedValue>) sideInputValue); } } diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/io/UnboundedSourceWrapper.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/io/UnboundedSourceWrapper.java index dfdecb2a21b41..cb912c12d9c2b 100644 --- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/io/UnboundedSourceWrapper.java +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/io/UnboundedSourceWrapper.java @@ -30,6 +30,7 @@ public class UnboundedSourceWrapper extends GearpumpSource { + private static final long serialVersionUID = -2453956849834747150L; private final UnboundedSource source; public UnboundedSourceWrapper(UnboundedSource source, diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/utils/DoFnRunnerFactory.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/utils/DoFnRunnerFactory.java index 5db8320c1497f..bdfc33679d07a 100644 --- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/utils/DoFnRunnerFactory.java +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/utils/DoFnRunnerFactory.java @@ -25,12 +25,12 @@ 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.SimpleDoFnRunner; import org.apache.beam.runners.gearpump.GearpumpPipelineOptions; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.DoFn; -import org.apache.beam.sdk.util.ExecutionContext; import org.apache.beam.sdk.util.ReadyCheckingSideInputReader; import org.apache.beam.sdk.util.WindowingStrategy; import org.apache.beam.sdk.values.PCollectionView; @@ -75,7 +75,7 @@ public DoFnRunnerFactory( public PushbackSideInputDoFnRunner createRunner( ReadyCheckingSideInputReader sideInputReader) { - DoFnRunner underlying = DoFnRunners.createDefault( + DoFnRunner underlying = DoFnRunners.simpleRunner( options, fn, sideInputReader, outputManager, mainOutputTag, sideOutputTags, stepContext, aggregatorFactory, windowingStrategy); return PushbackSideInputDoFnRunner.create(underlying, sideInputs, sideInputReader); diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/utils/NoOpAggregatorFactory.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/utils/NoOpAggregatorFactory.java index 22ffc4d926b09..343693088af8c 100644 --- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/utils/NoOpAggregatorFactory.java +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/utils/NoOpAggregatorFactory.java @@ -21,9 +21,9 @@ import java.io.Serializable; 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; -import org.apache.beam.sdk.util.ExecutionContext; /** * no-op aggregator factory. diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/utils/NoOpStepContext.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/utils/NoOpStepContext.java index 45f146b7d9650..140df2ad7c476 100644 --- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/utils/NoOpStepContext.java +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/utils/NoOpStepContext.java @@ -21,12 +21,12 @@ import java.io.IOException; import java.io.Serializable; +import org.apache.beam.runners.core.ExecutionContext; +import org.apache.beam.runners.core.StateInternals; +import org.apache.beam.runners.core.TimerInternals; 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; import org.apache.beam.sdk.values.TupleTag; /** diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/utils/TranslatorUtils.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/utils/TranslatorUtils.java index b8a5233ecb7fd..b8f0ccb41e7ea 100644 --- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/utils/TranslatorUtils.java +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/utils/TranslatorUtils.java @@ -38,8 +38,6 @@ import org.apache.gearpump.streaming.dsl.javaapi.JavaStream; import org.apache.gearpump.streaming.dsl.window.impl.Window; - - /** * Utility methods for translators. */ diff --git a/runners/gearpump/src/test/java/org/apache/beam/runners/gearpump/translators/utils/TranslatorUtilsTest.java b/runners/gearpump/src/test/java/org/apache/beam/runners/gearpump/translators/utils/TranslatorUtilsTest.java index 10976e8e96622..524887dd2057c 100644 --- a/runners/gearpump/src/test/java/org/apache/beam/runners/gearpump/translators/utils/TranslatorUtilsTest.java +++ b/runners/gearpump/src/test/java/org/apache/beam/runners/gearpump/translators/utils/TranslatorUtilsTest.java @@ -34,7 +34,6 @@ import org.apache.gearpump.streaming.dsl.window.impl.Window; import org.junit.Test; - /** * Tests for {@link TranslatorUtils}. */ From eb0d333df23624f54aae2abb8d7c7873f8ed2a7a Mon Sep 17 00:00:00 2001 From: huafengw Date: Tue, 21 Mar 2017 19:45:10 +0800 Subject: [PATCH 149/346] [BEAM-972] Add unit tests to Gearpump runner --- examples/java/pom.xml | 12 +++ pom.xml | 6 ++ runners/gearpump/README.md | 41 ++++++++- runners/gearpump/pom.xml | 2 - .../gearpump/GearpumpRunnerRegistrar.java | 4 +- .../translators/WindowAssignTranslator.java | 2 +- .../gearpump/translators/io/ValuesSource.java | 2 - .../gearpump/GearpumpRunnerRegistrarTest.java | 55 ++++++++++++ .../runners/gearpump/PipelineOptionsTest.java | 73 +++++++++++++++ .../translators/io/GearpumpSourceTest.java | 90 +++++++++++++++++++ .../translators/io/ValueSoureTest.java | 82 +++++++++++++++++ 11 files changed, 362 insertions(+), 7 deletions(-) create mode 100644 runners/gearpump/src/test/java/org/apache/beam/runners/gearpump/GearpumpRunnerRegistrarTest.java create mode 100644 runners/gearpump/src/test/java/org/apache/beam/runners/gearpump/PipelineOptionsTest.java create mode 100644 runners/gearpump/src/test/java/org/apache/beam/runners/gearpump/translators/io/GearpumpSourceTest.java create mode 100644 runners/gearpump/src/test/java/org/apache/beam/runners/gearpump/translators/io/ValueSoureTest.java diff --git a/examples/java/pom.xml b/examples/java/pom.xml index ed4a1d461650b..0a6d8fecd1904 100644 --- a/examples/java/pom.xml +++ b/examples/java/pom.xml @@ -87,6 +87,18 @@ + + + gearpump-runner + + + org.apache.beam + beam-runners-gearpump + runtime + + + + flink-runner diff --git a/pom.xml b/pom.xml index c3b847648f76c..2cdb09d68090e 100644 --- a/pom.xml +++ b/pom.xml @@ -473,6 +473,12 @@ ${project.version} + + org.apache.beam + beam-runners-gearpump + ${project.version} + + org.apache.beam beam-examples-java diff --git a/runners/gearpump/README.md b/runners/gearpump/README.md index ad043faea4c7e..e8ce79487d4af 100644 --- a/runners/gearpump/README.md +++ b/runners/gearpump/README.md @@ -19,4 +19,43 @@ ## Gearpump Beam Runner -The Gearpump Beam runner allows users to execute pipelines written using the Apache Beam programming API with Apache Gearpump (incubating) as an execution engine. \ No newline at end of file +The Gearpump Beam runner allows users to execute pipelines written using the Apache Beam programming API with Apache Gearpump (incubating) as an execution engine. + +##Getting Started + +The following shows how to run the WordCount example that is provided with the source code on Beam. + +###Installing Beam + +To get the latest version of Beam with Gearpump-Runner, first clone the Beam repository: + +``` +git clone https://github.com/apache/beam +git checkout gearpump-runner +``` + +Then switch to the newly created directory and run Maven to build the Apache Beam: + +``` +cd beam +mvn clean install -DskipTests +``` + +Now Apache Beam and the Gearpump Runner are installed in your local Maven repository. + +###Running Wordcount Example + +Download something to count: + +``` +curl http://www.gutenberg.org/cache/epub/1128/pg1128.txt > /tmp/kinglear.txt +``` + +Run the pipeline, using the Gearpump runner: + +``` +cd examples/java +mvn exec:java -Dexec.mainClass=org.apache.beam.examples.WordCount -Dexec.args="--inputFile=/tmp/kinglear.txt --output=/tmp/wordcounts.txt --runner=TestGearpumpRunner" -Pgearpump-runner +``` + +Once completed, check the output file /tmp/wordcounts.txt-00000-of-00001 diff --git a/runners/gearpump/pom.xml b/runners/gearpump/pom.xml index 9a6a4323955ff..a691801c19369 100644 --- a/runners/gearpump/pom.xml +++ b/runners/gearpump/pom.xml @@ -99,13 +99,11 @@ org.apache.gearpump gearpump-streaming_2.11 ${gearpump.version} - provided org.apache.gearpump gearpump-core_2.11 ${gearpump.version} - provided com.typesafe diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpRunnerRegistrar.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpRunnerRegistrar.java index b77e1e3b7dbaa..3183d4523719b 100644 --- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpRunnerRegistrar.java +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpRunnerRegistrar.java @@ -44,7 +44,9 @@ public static class Runner implements PipelineRunnerRegistrar { @Override public Iterable>> getPipelineRunners() { - return ImmutableList.>>of(TestGearpumpRunner.class); + return ImmutableList.>>of( + GearpumpRunner.class, + TestGearpumpRunner.class); } } diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/WindowAssignTranslator.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/WindowAssignTranslator.java index fe6015ae4a4b6..29d8f02901165 100644 --- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/WindowAssignTranslator.java +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/WindowAssignTranslator.java @@ -39,7 +39,7 @@ * {@link Window.Bound} is translated to Gearpump flatMap function. */ @SuppressWarnings("unchecked") -public class WindowAssignTranslator implements TransformTranslator> { +public class WindowAssignTranslator implements TransformTranslator> { private static final long serialVersionUID = -964887482120489061L; diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/io/ValuesSource.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/io/ValuesSource.java index e0488cd571e18..ccd5cdf7893c8 100644 --- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/io/ValuesSource.java +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/io/ValuesSource.java @@ -110,8 +110,6 @@ private static class ValuesReader extends UnboundedReader { this.source = source; } - - @Override public boolean start() throws IOException { if (null == iterator) { diff --git a/runners/gearpump/src/test/java/org/apache/beam/runners/gearpump/GearpumpRunnerRegistrarTest.java b/runners/gearpump/src/test/java/org/apache/beam/runners/gearpump/GearpumpRunnerRegistrarTest.java new file mode 100644 index 0000000000000..9a01d20b5168a --- /dev/null +++ b/runners/gearpump/src/test/java/org/apache/beam/runners/gearpump/GearpumpRunnerRegistrarTest.java @@ -0,0 +1,55 @@ +/* + * 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.gearpump; + +import static org.junit.Assert.assertEquals; + +import com.google.common.collect.ImmutableList; +import org.apache.beam.sdk.options.PipelineOptions; +import org.apache.beam.sdk.options.PipelineOptionsFactory; +import org.junit.Test; + +/** + * Tests for {@link GearpumpRunnerRegistrar}. + */ +public class GearpumpRunnerRegistrarTest { + + @Test + public void testFullName() { + String[] args = + new String[] {String.format("--runner=%s", GearpumpRunner.class.getName())}; + PipelineOptions opts = PipelineOptionsFactory.fromArgs(args).create(); + assertEquals(opts.getRunner(), GearpumpRunner.class); + } + + @Test + public void testClassName() { + String[] args = + new String[] {String.format("--runner=%s", GearpumpRunner.class.getSimpleName())}; + PipelineOptions opts = PipelineOptionsFactory.fromArgs(args).create(); + assertEquals(opts.getRunner(), GearpumpRunner.class); + } + + @Test + public void testOptions() { + assertEquals( + ImmutableList.of(GearpumpPipelineOptions.class), + new GearpumpRunnerRegistrar.Options().getPipelineOptions()); + } +} diff --git a/runners/gearpump/src/test/java/org/apache/beam/runners/gearpump/PipelineOptionsTest.java b/runners/gearpump/src/test/java/org/apache/beam/runners/gearpump/PipelineOptionsTest.java new file mode 100644 index 0000000000000..994856be4a589 --- /dev/null +++ b/runners/gearpump/src/test/java/org/apache/beam/runners/gearpump/PipelineOptionsTest.java @@ -0,0 +1,73 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.runners.gearpump; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNull; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.collect.Maps; +import com.typesafe.config.Config; + +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.util.Map; + +import org.apache.beam.sdk.options.PipelineOptions; +import org.apache.beam.sdk.options.PipelineOptionsFactory; +import org.apache.gearpump.cluster.ClusterConfig; +import org.apache.gearpump.cluster.embedded.EmbeddedCluster; +import org.junit.Test; + +/** + * Tests for {@link GearpumpPipelineOptions}. + */ +public class PipelineOptionsTest { + + @Test + public void testIgnoredFieldSerialization() throws IOException { + String appName = "forTest"; + Map serializers = Maps.newHashMap(); + serializers.put("classA", "SerializerA"); + GearpumpPipelineOptions options = PipelineOptionsFactory.create() + .as(GearpumpPipelineOptions.class); + Config config = ClusterConfig.master(null); + EmbeddedCluster cluster = new EmbeddedCluster(config); + options.setSerializers(serializers); + options.setApplicationName(appName); + options.setEmbeddedCluster(cluster); + options.setParallelism(10); + + byte[] serializedOptions = serialize(options); + GearpumpPipelineOptions deserializedOptions = new ObjectMapper() + .readValue(serializedOptions, PipelineOptions.class).as(GearpumpPipelineOptions.class); + + assertNull(deserializedOptions.getEmbeddedCluster()); + assertNull(deserializedOptions.getSerializers()); + assertEquals(10, deserializedOptions.getParallelism()); + assertEquals(appName, deserializedOptions.getApplicationName()); + } + + private byte[] serialize(Object obj) { + try (ByteArrayOutputStream baos = new ByteArrayOutputStream()) { + new ObjectMapper().writeValue(baos, obj); + return baos.toByteArray(); + } catch (Exception e) { + throw new RuntimeException("Couldn't serialize PipelineOptions.", e); + } + } +} diff --git a/runners/gearpump/src/test/java/org/apache/beam/runners/gearpump/translators/io/GearpumpSourceTest.java b/runners/gearpump/src/test/java/org/apache/beam/runners/gearpump/translators/io/GearpumpSourceTest.java new file mode 100644 index 0000000000000..af5a1d2c01649 --- /dev/null +++ b/runners/gearpump/src/test/java/org/apache/beam/runners/gearpump/translators/io/GearpumpSourceTest.java @@ -0,0 +1,90 @@ +/* + * 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.gearpump.translators.io; + +import com.google.common.collect.Lists; + +import java.io.IOException; +import java.time.Instant; +import java.util.List; + +import org.apache.beam.runners.gearpump.GearpumpPipelineOptions; +import org.apache.beam.runners.gearpump.translators.utils.TranslatorUtils; +import org.apache.beam.sdk.coders.StringUtf8Coder; +import org.apache.beam.sdk.io.Source; +import org.apache.beam.sdk.options.PipelineOptions; +import org.apache.beam.sdk.options.PipelineOptionsFactory; +import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.TimestampedValue; +import org.apache.gearpump.Message; +import org.apache.gearpump.streaming.source.Watermark; +import org.junit.Assert; +import org.junit.Test; + +/** + * Tests for {@link GearpumpSource}. + */ +public class GearpumpSourceTest { + private static final List> TEST_VALUES = Lists.newArrayList( + TimestampedValue.of("a", new org.joda.time.Instant(Long.MIN_VALUE)), + TimestampedValue.of("b", new org.joda.time.Instant(0)), + TimestampedValue.of("c", new org.joda.time.Instant(53)), + TimestampedValue.of("d", new org.joda.time.Instant(Long.MAX_VALUE - 1)) + ); + + private static class SourceForTest extends GearpumpSource { + private ValuesSource valuesSource; + + SourceForTest(PipelineOptions options, ValuesSource valuesSource) { + super(options); + this.valuesSource = valuesSource; + } + + @Override + protected Source.Reader createReader(PipelineOptions options) throws IOException { + return this.valuesSource.createReader(options, null); + } + } + + @Test + public void testGearpumpSource() { + GearpumpPipelineOptions options = PipelineOptionsFactory.create() + .as(GearpumpPipelineOptions.class); + ValuesSource> valuesSource = new ValuesSource<>(TEST_VALUES, + TimestampedValue.TimestampedValueCoder.of(StringUtf8Coder.of())); + SourceForTest> sourceForTest = + new SourceForTest<>(options, valuesSource); + sourceForTest.open(null, Instant.EPOCH); + + for (TimestampedValue value: TEST_VALUES) { + // Check the watermark first since the Source will advance when it's opened + Instant expectedWaterMark = TranslatorUtils.jodaTimeToJava8Time(value.getTimestamp()); + Assert.assertEquals(expectedWaterMark, sourceForTest.getWatermark()); + + Message expectedMsg = Message.apply( + WindowedValue.timestampedValueInGlobalWindow(value, value.getTimestamp()), + value.getTimestamp().getMillis()); + Message message = sourceForTest.read(); + Assert.assertEquals(expectedMsg, message); + } + + Assert.assertNull(sourceForTest.read()); + Assert.assertEquals(Watermark.MAX(), sourceForTest.getWatermark()); + } +} diff --git a/runners/gearpump/src/test/java/org/apache/beam/runners/gearpump/translators/io/ValueSoureTest.java b/runners/gearpump/src/test/java/org/apache/beam/runners/gearpump/translators/io/ValueSoureTest.java new file mode 100644 index 0000000000000..8c50703c3c327 --- /dev/null +++ b/runners/gearpump/src/test/java/org/apache/beam/runners/gearpump/translators/io/ValueSoureTest.java @@ -0,0 +1,82 @@ +/* + * 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.gearpump.translators.io; + +import com.google.common.collect.Lists; +import com.google.common.collect.Sets; +import com.typesafe.config.Config; +import com.typesafe.config.ConfigValueFactory; + +import java.util.Collections; +import java.util.HashSet; +import java.util.List; +import java.util.Set; + +import org.apache.beam.runners.gearpump.GearpumpPipelineOptions; +import org.apache.beam.runners.gearpump.GearpumpRunner; +import org.apache.beam.sdk.Pipeline; +import org.apache.beam.sdk.coders.StringUtf8Coder; +import org.apache.beam.sdk.io.Read; +import org.apache.beam.sdk.options.PipelineOptionsFactory; +import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.transforms.ParDo; +import org.apache.gearpump.cluster.ClusterConfig; +import org.apache.gearpump.cluster.embedded.EmbeddedCluster; +import org.apache.gearpump.util.Constants; +import org.junit.Assert; +import org.junit.Test; + +/** + * Tests for {@link ValuesSource}. + */ +public class ValueSoureTest { + + @Test + public void testValueSource() { + GearpumpPipelineOptions options = PipelineOptionsFactory.create() + .as(GearpumpPipelineOptions.class); + Config config = ClusterConfig.master(null); + config = config.withValue(Constants.APPLICATION_TOTAL_RETRIES(), + ConfigValueFactory.fromAnyRef(0)); + EmbeddedCluster cluster = new EmbeddedCluster(config); + cluster.start(); + + options.setEmbeddedCluster(cluster); + options.setRunner(GearpumpRunner.class); + options.setParallelism(1); + Pipeline p = Pipeline.create(options); + List values = Lists.newArrayList("1", "2", "3", "4", "5"); + ValuesSource source = new ValuesSource<>(values, StringUtf8Coder.of()); + p.apply(Read.from(source)) + .apply(ParDo.of(new ResultCollector())); + + p.run().waitUntilFinish(); + cluster.stop(); + + Assert.assertEquals(Sets.newHashSet(values), ResultCollector.RESULTS); + } + + private static class ResultCollector extends DoFn { + private static final Set RESULTS = Collections.synchronizedSet(new HashSet<>()); + + @ProcessElement + public void processElement(ProcessContext c) throws Exception { + RESULTS.add(c.element()); + } + } +} From f3138dde12b0c6a5cfdb8fefab916a6060b7f5ea Mon Sep 17 00:00:00 2001 From: huafengw Date: Wed, 12 Apr 2017 19:11:09 +0800 Subject: [PATCH 150/346] [BEAM-972] Add more unit test to Gearpump runner --- .../FlattenPCollectionsTranslator.java | 2 +- .../translators/GroupByKeyTranslator.java | 20 ++- .../translators/WindowAssignTranslator.java | 5 +- ...GearpumpPCollectionViewTranslatorTest.java | 57 +++++++ .../CreatePCollectionViewTranslatorTest.java | 55 +++++++ .../FlattenPCollectionsTranslatorTest.java | 137 ++++++++++++++++ .../translators/GroupByKeyTranslatorTest.java | 151 ++++++++++++++++++ .../ReadBoundedTranslatorTest.java | 70 ++++++++ .../ReadUnboundedTranslatorTest.java | 70 ++++++++ .../WindowAssignTranslatorTest.java | 110 +++++++++++++ .../translators/io/GearpumpSourceTest.java | 36 ++--- .../translators/io/ValueSoureTest.java | 15 +- 12 files changed, 695 insertions(+), 33 deletions(-) create mode 100644 runners/gearpump/src/test/java/org/apache/beam/runners/gearpump/translators/CreateGearpumpPCollectionViewTranslatorTest.java create mode 100644 runners/gearpump/src/test/java/org/apache/beam/runners/gearpump/translators/CreatePCollectionViewTranslatorTest.java create mode 100644 runners/gearpump/src/test/java/org/apache/beam/runners/gearpump/translators/FlattenPCollectionsTranslatorTest.java create mode 100644 runners/gearpump/src/test/java/org/apache/beam/runners/gearpump/translators/GroupByKeyTranslatorTest.java create mode 100644 runners/gearpump/src/test/java/org/apache/beam/runners/gearpump/translators/ReadBoundedTranslatorTest.java create mode 100644 runners/gearpump/src/test/java/org/apache/beam/runners/gearpump/translators/ReadUnboundedTranslatorTest.java create mode 100644 runners/gearpump/src/test/java/org/apache/beam/runners/gearpump/translators/WindowAssignTranslatorTest.java diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/FlattenPCollectionsTranslator.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/FlattenPCollectionsTranslator.java index 3a465cbef84ec..56f7d1a67d068 100644 --- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/FlattenPCollectionsTranslator.java +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/FlattenPCollectionsTranslator.java @@ -47,7 +47,6 @@ public void translate(Flatten.PCollections transform, TranslationContext cont Set> unique = new HashSet<>(); for (TaggedPValue input: context.getInputs()) { PCollection collection = (PCollection) input.getValue(); - unique.add(collection); JavaStream inputStream = context.getInputStream(collection); if (null == merged) { merged = inputStream; @@ -60,6 +59,7 @@ public void translate(Flatten.PCollections transform, TranslationContext cont merged = merged.merge(inputStream, transform.getName()); } + unique.add(collection); } if (null == merged) { diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/GroupByKeyTranslator.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/GroupByKeyTranslator.java index 5dfd3e9d1794f..54c87373d9b4c 100644 --- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/GroupByKeyTranslator.java +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/GroupByKeyTranslator.java @@ -83,7 +83,10 @@ public void translate(GroupByKey transform, TranslationContext context) { context.setOutputStream(context.getOutput(), outputStream); } - private static class GearpumpWindowFn + /** + * A transform used internally to translate Beam's Window to Gearpump's Window. + */ + protected static class GearpumpWindowFn implements WindowFunction>, Serializable { private final boolean isNonMerging; @@ -115,7 +118,10 @@ private Window[] toGearpumpWindows(BoundedWindow[] windows) { } } - private static class GroupByFn extends + /** + * A transform used internally to group KV message by its key. + */ + protected static class GroupByFn extends GroupByFunction>, ByteBuffer> { private static final long serialVersionUID = -807905402490735530L; @@ -135,7 +141,10 @@ public ByteBuffer groupBy(WindowedValue> wv) { } } - private static class KeyedByTimestamp + /** + * A transform used internally to transform WindowedValue to KV. + */ + protected static class KeyedByTimestamp extends MapFunction>, KV>>> { @@ -154,7 +163,10 @@ public KV>> map( } } - private static class Merge extends + /** + * A transform used internally by Gearpump which encapsulates the merge logic. + */ + protected static class Merge extends FoldFunction>>, KV>>>> { diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/WindowAssignTranslator.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/WindowAssignTranslator.java index 29d8f02901165..2d70b63e24158 100644 --- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/WindowAssignTranslator.java +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/WindowAssignTranslator.java @@ -57,7 +57,10 @@ public void translate(Window.Assign transform, TranslationContext context) { context.setOutputStream(output, outputStream); } - private static class AssignWindows extends + /** + * A Function used internally by Gearpump to wrap the actual Beam's WindowFn. + */ + protected static class AssignWindows extends FlatMapFunction, WindowedValue> { private static final long serialVersionUID = 7284565861938681360L; diff --git a/runners/gearpump/src/test/java/org/apache/beam/runners/gearpump/translators/CreateGearpumpPCollectionViewTranslatorTest.java b/runners/gearpump/src/test/java/org/apache/beam/runners/gearpump/translators/CreateGearpumpPCollectionViewTranslatorTest.java new file mode 100644 index 0000000000000..b23b0c603ece8 --- /dev/null +++ b/runners/gearpump/src/test/java/org/apache/beam/runners/gearpump/translators/CreateGearpumpPCollectionViewTranslatorTest.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.gearpump.translators; + +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +import org.apache.beam.runners.gearpump.GearpumpPipelineTranslator; +import org.apache.beam.sdk.values.PCollectionView; +import org.apache.beam.sdk.values.PValue; +import org.apache.gearpump.streaming.dsl.javaapi.JavaStream; +import org.junit.Test; + +/** Tests for {@link CreateGearpumpPCollectionViewTranslator}. */ +public class CreateGearpumpPCollectionViewTranslatorTest { + + @Test + @SuppressWarnings({"rawtypes", "unchecked"}) + public void testTranslate() { + CreateGearpumpPCollectionViewTranslator translator = + new CreateGearpumpPCollectionViewTranslator(); + + GearpumpPipelineTranslator.CreateGearpumpPCollectionView pCollectionView = + mock(GearpumpPipelineTranslator.CreateGearpumpPCollectionView.class); + + JavaStream javaStream = mock(JavaStream.class); + TranslationContext translationContext = mock(TranslationContext.class); + + PValue mockInput = mock(PValue.class); + when(translationContext.getInput()).thenReturn(mockInput); + when(translationContext.getInputStream(mockInput)).thenReturn(javaStream); + + PCollectionView view = mock(PCollectionView.class); + when(translationContext.getOutput()).thenReturn(view); + + translator.translate(pCollectionView, translationContext); + verify(translationContext, times(1)).setOutputStream(view, javaStream); + } +} diff --git a/runners/gearpump/src/test/java/org/apache/beam/runners/gearpump/translators/CreatePCollectionViewTranslatorTest.java b/runners/gearpump/src/test/java/org/apache/beam/runners/gearpump/translators/CreatePCollectionViewTranslatorTest.java new file mode 100644 index 0000000000000..42ff14e60adcf --- /dev/null +++ b/runners/gearpump/src/test/java/org/apache/beam/runners/gearpump/translators/CreatePCollectionViewTranslatorTest.java @@ -0,0 +1,55 @@ +/* + * 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.gearpump.translators; + +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +import org.apache.beam.sdk.transforms.View; +import org.apache.beam.sdk.values.PCollectionView; +import org.apache.beam.sdk.values.PValue; +import org.apache.gearpump.streaming.dsl.javaapi.JavaStream; +import org.junit.Test; + +/** Tests for {@link CreatePCollectionViewTranslator}. */ +public class CreatePCollectionViewTranslatorTest { + + @Test + @SuppressWarnings({"rawtypes", "unchecked"}) + public void testTranslate() { + CreatePCollectionViewTranslator translator = new CreatePCollectionViewTranslator(); + View.CreatePCollectionView> createView = + mock(View.CreatePCollectionView.class); + + JavaStream javaStream = mock(JavaStream.class); + TranslationContext translationContext = mock(TranslationContext.class); + + PValue mockInput = mock(PValue.class); + when(translationContext.getInput()).thenReturn(mockInput); + when(translationContext.getInputStream(mockInput)).thenReturn(javaStream); + + PCollectionView view = mock(PCollectionView.class); + when(translationContext.getOutput()).thenReturn(view); + + translator.translate(createView, translationContext); + verify(translationContext, times(1)).setOutputStream(view, javaStream); + } +} diff --git a/runners/gearpump/src/test/java/org/apache/beam/runners/gearpump/translators/FlattenPCollectionsTranslatorTest.java b/runners/gearpump/src/test/java/org/apache/beam/runners/gearpump/translators/FlattenPCollectionsTranslatorTest.java new file mode 100644 index 0000000000000..fa89d4a480306 --- /dev/null +++ b/runners/gearpump/src/test/java/org/apache/beam/runners/gearpump/translators/FlattenPCollectionsTranslatorTest.java @@ -0,0 +1,137 @@ +/* + * 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.gearpump.translators; + +import static org.mockito.Matchers.argThat; +import static org.mockito.Mockito.any; +import static org.mockito.Mockito.eq; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +import com.google.common.collect.Lists; +import java.util.Collections; +import org.apache.beam.runners.gearpump.translators.io.UnboundedSourceWrapper; +import org.apache.beam.sdk.transforms.Flatten; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.PValue; +import org.apache.beam.sdk.values.TaggedPValue; +import org.apache.gearpump.streaming.dsl.api.functions.MapFunction; +import org.apache.gearpump.streaming.dsl.javaapi.JavaStream; +import org.apache.gearpump.streaming.source.DataSource; +import org.junit.Test; +import org.mockito.ArgumentMatcher; + +/** Tests for {@link FlattenPCollectionsTranslator}. */ +public class FlattenPCollectionsTranslatorTest { + + private FlattenPCollectionsTranslator translator = new FlattenPCollectionsTranslator(); + private Flatten.PCollections transform = mock(Flatten.PCollections.class); + + class UnboundedSourceWrapperMatcher extends ArgumentMatcher { + @Override + public boolean matches(Object o) { + return o instanceof UnboundedSourceWrapper; + } + } + + @Test + @SuppressWarnings({"rawtypes", "unchecked"}) + public void testTranslateWithEmptyCollection() { + PValue mockOutput = mock(PValue.class); + TranslationContext translationContext = mock(TranslationContext.class); + + when(translationContext.getInputs()).thenReturn(Collections.EMPTY_LIST); + when(translationContext.getOutput()).thenReturn(mockOutput); + + translator.translate(transform, translationContext); + verify(translationContext).getSourceStream(argThat(new UnboundedSourceWrapperMatcher())); + } + + @Test + @SuppressWarnings({"rawtypes", "unchecked"}) + public void testTranslateWithOneCollection() { + JavaStream javaStream = mock(JavaStream.class); + TranslationContext translationContext = mock(TranslationContext.class); + + TaggedPValue mockInput = mock(TaggedPValue.class); + PCollection mockCollection = mock(PCollection.class); + when(mockInput.getValue()).thenReturn(mockCollection); + + when(translationContext.getInputs()).thenReturn(Lists.newArrayList(mockInput)); + when(translationContext.getInputStream(mockCollection)).thenReturn(javaStream); + + PValue mockOutput = mock(PValue.class); + when(translationContext.getOutput()).thenReturn(mockOutput); + + translator.translate(transform, translationContext); + verify(translationContext, times(1)).setOutputStream(mockOutput, javaStream); + } + + @Test + @SuppressWarnings({"rawtypes", "unchecked"}) + public void testWithMoreThanOneCollections() { + String transformName = "transform"; + when(transform.getName()).thenReturn(transformName); + + JavaStream javaStream1 = mock(JavaStream.class); + JavaStream javaStream2 = mock(JavaStream.class); + TranslationContext translationContext = mock(TranslationContext.class); + + TaggedPValue mockInput1 = mock(TaggedPValue.class); + PCollection mockCollection1 = mock(PCollection.class); + when(mockInput1.getValue()).thenReturn(mockCollection1); + + TaggedPValue mockInput2 = mock(TaggedPValue.class); + PCollection mockCollection2 = mock(PCollection.class); + when(mockInput2.getValue()).thenReturn(mockCollection2); + + when(translationContext.getInputs()).thenReturn(Lists.newArrayList(mockInput1, mockInput2)); + when(translationContext.getInputStream(mockCollection1)).thenReturn(javaStream1); + when(translationContext.getInputStream(mockCollection2)).thenReturn(javaStream2); + + translator.translate(transform, translationContext); + verify(javaStream1).merge(javaStream2, transformName); + } + + @Test + @SuppressWarnings({"rawtypes", "unchecked"}) + public void testWithDuplicatedCollections() { + String transformName = "transform"; + when(transform.getName()).thenReturn(transformName); + + JavaStream javaStream1 = mock(JavaStream.class); + TranslationContext translationContext = mock(TranslationContext.class); + + PCollection mockCollection1 = mock(PCollection.class); + TaggedPValue mockInput1 = mock(TaggedPValue.class); + when(mockInput1.getValue()).thenReturn(mockCollection1); + + TaggedPValue mockInput2 = mock(TaggedPValue.class); + when(mockInput2.getValue()).thenReturn(mockCollection1); + + when(translationContext.getInputs()).thenReturn(Lists.newArrayList(mockInput1, mockInput2)); + when(translationContext.getInputStream(mockCollection1)).thenReturn(javaStream1); + + translator.translate(transform, translationContext); + verify(javaStream1).map(any(MapFunction.class), eq("dummy")); + verify(javaStream1).merge(any(JavaStream.class), eq(transformName)); + } +} diff --git a/runners/gearpump/src/test/java/org/apache/beam/runners/gearpump/translators/GroupByKeyTranslatorTest.java b/runners/gearpump/src/test/java/org/apache/beam/runners/gearpump/translators/GroupByKeyTranslatorTest.java new file mode 100644 index 0000000000000..9135022a5605c --- /dev/null +++ b/runners/gearpump/src/test/java/org/apache/beam/runners/gearpump/translators/GroupByKeyTranslatorTest.java @@ -0,0 +1,151 @@ +/* + * 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.gearpump.translators; + +import static org.hamcrest.Matchers.equalTo; +import static org.junit.Assert.assertThat; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.Iterables; +import com.google.common.collect.Lists; + +import java.time.Instant; +import java.util.Collection; +import java.util.List; + +import org.apache.beam.runners.gearpump.translators.GroupByKeyTranslator.GearpumpWindowFn; +import org.apache.beam.runners.gearpump.translators.utils.TranslatorUtils; +import org.apache.beam.sdk.transforms.windowing.BoundedWindow; +import org.apache.beam.sdk.transforms.windowing.IntervalWindow; +import org.apache.beam.sdk.transforms.windowing.OutputTimeFn; +import org.apache.beam.sdk.transforms.windowing.OutputTimeFns; +import org.apache.beam.sdk.transforms.windowing.PaneInfo; +import org.apache.beam.sdk.transforms.windowing.Sessions; +import org.apache.beam.sdk.transforms.windowing.WindowFn; +import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.KV; +import org.apache.gearpump.streaming.dsl.window.api.WindowFunction; +import org.apache.gearpump.streaming.dsl.window.impl.Window; +import org.joda.time.Duration; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +/** Tests for {@link GroupByKeyTranslator}. */ +@RunWith(Parameterized.class) +public class GroupByKeyTranslatorTest { + + @Test + @SuppressWarnings({"rawtypes", "unchecked"}) + public void testGearpumpWindowFn() { + GearpumpWindowFn windowFn = new GearpumpWindowFn(true); + List windows = + Lists.newArrayList( + new IntervalWindow(new org.joda.time.Instant(0), new org.joda.time.Instant(10)), + new IntervalWindow(new org.joda.time.Instant(5), new org.joda.time.Instant(15))); + + WindowFunction.Context> context = + new WindowFunction.Context>() { + @Override + public Instant timestamp() { + return Instant.EPOCH; + } + + @Override + public WindowedValue element() { + return WindowedValue.of( + "v1", new org.joda.time.Instant(6), windows, PaneInfo.NO_FIRING); + } + }; + + Window[] result = windowFn.apply(context); + List expected = Lists.newArrayList(); + for (BoundedWindow w : windows) { + expected.add(TranslatorUtils.boundedWindowToGearpumpWindow(w)); + } + assertThat(result, equalTo(expected.toArray())); + } + + @Parameterized.Parameters(name = "{index}: {0}") + public static Iterable> data() { + return ImmutableList.of( + OutputTimeFns.outputAtEarliestInputTimestamp(), + OutputTimeFns.outputAtLatestInputTimestamp(), + OutputTimeFns.outputAtEndOfWindow()); + } + + @Parameterized.Parameter(0) + public OutputTimeFn outputTimeFn; + + @Test + @SuppressWarnings({"rawtypes", "unchecked"}) + public void testKeyedByTimestamp() { + BoundedWindow window = + new IntervalWindow(new org.joda.time.Instant(0), new org.joda.time.Instant(10)); + GroupByKeyTranslator.KeyedByTimestamp keyedByTimestamp = + new GroupByKeyTranslator.KeyedByTimestamp(outputTimeFn); + WindowedValue> value = + WindowedValue.of( + KV.of("key", "val"), org.joda.time.Instant.now(), window, PaneInfo.NO_FIRING); + KV>> result = + keyedByTimestamp.map(value); + org.joda.time.Instant time = + outputTimeFn.assignOutputTime( + value.getTimestamp(), Iterables.getOnlyElement(value.getWindows())); + assertThat(result, equalTo(KV.of(time, value))); + } + + @Test + @SuppressWarnings({"rawtypes", "unchecked"}) + public void testMerge() { + WindowFn slidingWindows = Sessions.withGapDuration(Duration.millis(10)); + GroupByKeyTranslator.Merge merge = new GroupByKeyTranslator.Merge(slidingWindows, outputTimeFn); + org.joda.time.Instant key1 = new org.joda.time.Instant(5); + WindowedValue> value1 = + WindowedValue.of( + KV.of("key1", "value1"), + key1, + new IntervalWindow(new org.joda.time.Instant(5), new org.joda.time.Instant(10)), + PaneInfo.NO_FIRING); + + org.joda.time.Instant key2 = new org.joda.time.Instant(10); + WindowedValue> value2 = + WindowedValue.of( + KV.of("key2", "value2"), + key2, + new IntervalWindow(new org.joda.time.Instant(9), new org.joda.time.Instant(14)), + PaneInfo.NO_FIRING); + + KV>>> result1 = + merge.fold(KV.of(null, null), KV.of(key1, value1)); + assertThat(result1.getKey(), equalTo(key1)); + assertThat(result1.getValue().getValue().getValue(), equalTo(Lists.newArrayList("value1"))); + + KV>>> result2 = + merge.fold(result1, KV.of(key2, value2)); + assertThat(result2.getKey(), equalTo(outputTimeFn.combine(key1, key2))); + Collection resultWindows = result2.getValue().getWindows(); + assertThat(resultWindows.size(), equalTo(1)); + IntervalWindow expectedWindow = + new IntervalWindow(new org.joda.time.Instant(5), new org.joda.time.Instant(14)); + assertThat(resultWindows.toArray()[0], equalTo(expectedWindow)); + assertThat( + result2.getValue().getValue().getValue(), equalTo(Lists.newArrayList("value1", "value2"))); + } +} diff --git a/runners/gearpump/src/test/java/org/apache/beam/runners/gearpump/translators/ReadBoundedTranslatorTest.java b/runners/gearpump/src/test/java/org/apache/beam/runners/gearpump/translators/ReadBoundedTranslatorTest.java new file mode 100644 index 0000000000000..20ee1a24a37a8 --- /dev/null +++ b/runners/gearpump/src/test/java/org/apache/beam/runners/gearpump/translators/ReadBoundedTranslatorTest.java @@ -0,0 +1,70 @@ +/* + * 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.gearpump.translators; + +import static org.mockito.Matchers.any; +import static org.mockito.Matchers.argThat; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +import org.apache.beam.runners.gearpump.GearpumpPipelineOptions; +import org.apache.beam.runners.gearpump.translators.io.BoundedSourceWrapper; +import org.apache.beam.sdk.io.BoundedSource; +import org.apache.beam.sdk.io.Read; +import org.apache.beam.sdk.options.PipelineOptionsFactory; +import org.apache.beam.sdk.values.PValue; +import org.apache.gearpump.streaming.dsl.javaapi.JavaStream; +import org.apache.gearpump.streaming.source.DataSource; +import org.junit.Test; +import org.mockito.ArgumentMatcher; + +/** Tests for {@link ReadBoundedTranslator}. */ +public class ReadBoundedTranslatorTest { + + class BoundedSourceWrapperMatcher extends ArgumentMatcher { + @Override + public boolean matches(Object o) { + return o instanceof BoundedSourceWrapper; + } + } + + @Test + @SuppressWarnings({"rawtypes", "unchecked"}) + public void testTranslate() { + ReadBoundedTranslator translator = new ReadBoundedTranslator(); + GearpumpPipelineOptions options = + PipelineOptionsFactory.create().as(GearpumpPipelineOptions.class); + Read.Bounded transform = mock(Read.Bounded.class); + BoundedSource source = mock(BoundedSource.class); + when(transform.getSource()).thenReturn(source); + + TranslationContext translationContext = mock(TranslationContext.class); + when(translationContext.getPipelineOptions()).thenReturn(options); + + JavaStream stream = mock(JavaStream.class); + PValue mockOutput = mock(PValue.class); + when(translationContext.getOutput()).thenReturn(mockOutput); + when(translationContext.getSourceStream(any(DataSource.class))).thenReturn(stream); + + translator.translate(transform, translationContext); + verify(translationContext).getSourceStream(argThat(new BoundedSourceWrapperMatcher())); + verify(translationContext).setOutputStream(mockOutput, stream); + } +} diff --git a/runners/gearpump/src/test/java/org/apache/beam/runners/gearpump/translators/ReadUnboundedTranslatorTest.java b/runners/gearpump/src/test/java/org/apache/beam/runners/gearpump/translators/ReadUnboundedTranslatorTest.java new file mode 100644 index 0000000000000..f27b568374ab9 --- /dev/null +++ b/runners/gearpump/src/test/java/org/apache/beam/runners/gearpump/translators/ReadUnboundedTranslatorTest.java @@ -0,0 +1,70 @@ +/* + * 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.gearpump.translators; + +import static org.mockito.Matchers.any; +import static org.mockito.Matchers.argThat; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +import org.apache.beam.runners.gearpump.GearpumpPipelineOptions; +import org.apache.beam.runners.gearpump.translators.io.UnboundedSourceWrapper; +import org.apache.beam.sdk.io.Read; +import org.apache.beam.sdk.io.UnboundedSource; +import org.apache.beam.sdk.options.PipelineOptionsFactory; +import org.apache.beam.sdk.values.PValue; +import org.apache.gearpump.streaming.dsl.javaapi.JavaStream; +import org.apache.gearpump.streaming.source.DataSource; +import org.junit.Test; +import org.mockito.ArgumentMatcher; + +/** Tests for {@link ReadUnboundedTranslator}. */ +public class ReadUnboundedTranslatorTest { + + class UnboundedSourceWrapperMatcher extends ArgumentMatcher { + @Override + public boolean matches(Object o) { + return o instanceof UnboundedSourceWrapper; + } + } + + @Test + @SuppressWarnings({"rawtypes", "unchecked"}) + public void testTranslate() { + ReadUnboundedTranslator translator = new ReadUnboundedTranslator(); + GearpumpPipelineOptions options = + PipelineOptionsFactory.create().as(GearpumpPipelineOptions.class); + Read.Unbounded transform = mock(Read.Unbounded.class); + UnboundedSource source = mock(UnboundedSource.class); + when(transform.getSource()).thenReturn(source); + + TranslationContext translationContext = mock(TranslationContext.class); + when(translationContext.getPipelineOptions()).thenReturn(options); + + JavaStream stream = mock(JavaStream.class); + PValue mockOutput = mock(PValue.class); + when(translationContext.getOutput()).thenReturn(mockOutput); + when(translationContext.getSourceStream(any(DataSource.class))).thenReturn(stream); + + translator.translate(transform, translationContext); + verify(translationContext).getSourceStream(argThat(new UnboundedSourceWrapperMatcher())); + verify(translationContext).setOutputStream(mockOutput, stream); + } +} diff --git a/runners/gearpump/src/test/java/org/apache/beam/runners/gearpump/translators/WindowAssignTranslatorTest.java b/runners/gearpump/src/test/java/org/apache/beam/runners/gearpump/translators/WindowAssignTranslatorTest.java new file mode 100644 index 0000000000000..06ccaaf761713 --- /dev/null +++ b/runners/gearpump/src/test/java/org/apache/beam/runners/gearpump/translators/WindowAssignTranslatorTest.java @@ -0,0 +1,110 @@ +/* + * 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.gearpump.translators; + +import static org.hamcrest.Matchers.equalTo; +import static org.junit.Assert.assertThat; + +import com.google.common.collect.Lists; +import java.util.ArrayList; +import java.util.Iterator; +import org.apache.beam.sdk.transforms.windowing.GlobalWindows; +import org.apache.beam.sdk.transforms.windowing.IntervalWindow; +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.transforms.windowing.WindowFn; +import org.apache.beam.sdk.util.WindowedValue; +import org.joda.time.Duration; +import org.joda.time.Instant; +import org.junit.Test; + +/** Tests for {@link WindowAssignTranslator}. */ +public class WindowAssignTranslatorTest { + + @Test + @SuppressWarnings({"rawtypes", "unchecked"}) + public void testAssignWindowsWithSlidingWindow() { + WindowFn slidingWindows = SlidingWindows.of(Duration.millis(10)).every(Duration.millis(5)); + WindowAssignTranslator.AssignWindows assignWindows = + new WindowAssignTranslator.AssignWindows(slidingWindows); + + String value = "v1"; + Instant timestamp = new Instant(1); + WindowedValue windowedValue = + WindowedValue.timestampedValueInGlobalWindow(value, timestamp); + ArrayList> expected = new ArrayList<>(); + expected.add( + WindowedValue.of( + value, + timestamp, + new IntervalWindow(new Instant(0), new Instant(10)), + PaneInfo.NO_FIRING)); + expected.add( + WindowedValue.of( + value, + timestamp, + new IntervalWindow(new Instant(-5), new Instant(5)), + PaneInfo.NO_FIRING)); + + Iterator> result = assignWindows.flatMap(windowedValue); + assertThat(expected, equalTo(Lists.newArrayList(result))); + } + + @Test + @SuppressWarnings({"rawtypes", "unchecked"}) + public void testAssignWindowsWithSessions() { + WindowFn slidingWindows = Sessions.withGapDuration(Duration.millis(10)); + WindowAssignTranslator.AssignWindows assignWindows = + new WindowAssignTranslator.AssignWindows(slidingWindows); + + String value = "v1"; + Instant timestamp = new Instant(1); + WindowedValue windowedValue = + WindowedValue.timestampedValueInGlobalWindow(value, timestamp); + ArrayList> expected = new ArrayList<>(); + expected.add( + WindowedValue.of( + value, + timestamp, + new IntervalWindow(new Instant(1), new Instant(11)), + PaneInfo.NO_FIRING)); + + Iterator> result = assignWindows.flatMap(windowedValue); + assertThat(expected, equalTo(Lists.newArrayList(result))); + } + + @Test + @SuppressWarnings({"rawtypes", "unchecked"}) + public void testAssignWindowsGlobal() { + WindowFn slidingWindows = new GlobalWindows(); + WindowAssignTranslator.AssignWindows assignWindows = + new WindowAssignTranslator.AssignWindows(slidingWindows); + + String value = "v1"; + Instant timestamp = new Instant(1); + WindowedValue windowedValue = + WindowedValue.timestampedValueInGlobalWindow(value, timestamp); + ArrayList> expected = new ArrayList<>(); + expected.add(WindowedValue.timestampedValueInGlobalWindow(value, timestamp)); + + Iterator> result = assignWindows.flatMap(windowedValue); + assertThat(expected, equalTo(Lists.newArrayList(result))); + } +} diff --git a/runners/gearpump/src/test/java/org/apache/beam/runners/gearpump/translators/io/GearpumpSourceTest.java b/runners/gearpump/src/test/java/org/apache/beam/runners/gearpump/translators/io/GearpumpSourceTest.java index af5a1d2c01649..b244484cbeb82 100644 --- a/runners/gearpump/src/test/java/org/apache/beam/runners/gearpump/translators/io/GearpumpSourceTest.java +++ b/runners/gearpump/src/test/java/org/apache/beam/runners/gearpump/translators/io/GearpumpSourceTest.java @@ -37,16 +37,14 @@ import org.junit.Assert; import org.junit.Test; -/** - * Tests for {@link GearpumpSource}. - */ +/** Tests for {@link GearpumpSource}. */ public class GearpumpSourceTest { - private static final List> TEST_VALUES = Lists.newArrayList( - TimestampedValue.of("a", new org.joda.time.Instant(Long.MIN_VALUE)), - TimestampedValue.of("b", new org.joda.time.Instant(0)), - TimestampedValue.of("c", new org.joda.time.Instant(53)), - TimestampedValue.of("d", new org.joda.time.Instant(Long.MAX_VALUE - 1)) - ); + private static final List> TEST_VALUES = + Lists.newArrayList( + TimestampedValue.of("a", new org.joda.time.Instant(Long.MIN_VALUE)), + TimestampedValue.of("b", new org.joda.time.Instant(0)), + TimestampedValue.of("c", new org.joda.time.Instant(53)), + TimestampedValue.of("d", new org.joda.time.Instant(Long.MAX_VALUE - 1))); private static class SourceForTest extends GearpumpSource { private ValuesSource valuesSource; @@ -64,22 +62,24 @@ protected Source.Reader createReader(PipelineOptions options) throws IOExcept @Test public void testGearpumpSource() { - GearpumpPipelineOptions options = PipelineOptionsFactory.create() - .as(GearpumpPipelineOptions.class); - ValuesSource> valuesSource = new ValuesSource<>(TEST_VALUES, - TimestampedValue.TimestampedValueCoder.of(StringUtf8Coder.of())); + GearpumpPipelineOptions options = + PipelineOptionsFactory.create().as(GearpumpPipelineOptions.class); + ValuesSource> valuesSource = + new ValuesSource<>( + TEST_VALUES, TimestampedValue.TimestampedValueCoder.of(StringUtf8Coder.of())); SourceForTest> sourceForTest = - new SourceForTest<>(options, valuesSource); + new SourceForTest<>(options, valuesSource); sourceForTest.open(null, Instant.EPOCH); - for (TimestampedValue value: TEST_VALUES) { + for (TimestampedValue value : TEST_VALUES) { // Check the watermark first since the Source will advance when it's opened Instant expectedWaterMark = TranslatorUtils.jodaTimeToJava8Time(value.getTimestamp()); Assert.assertEquals(expectedWaterMark, sourceForTest.getWatermark()); - Message expectedMsg = Message.apply( - WindowedValue.timestampedValueInGlobalWindow(value, value.getTimestamp()), - value.getTimestamp().getMillis()); + Message expectedMsg = + Message.apply( + WindowedValue.timestampedValueInGlobalWindow(value, value.getTimestamp()), + value.getTimestamp().getMillis()); Message message = sourceForTest.read(); Assert.assertEquals(expectedMsg, message); } diff --git a/runners/gearpump/src/test/java/org/apache/beam/runners/gearpump/translators/io/ValueSoureTest.java b/runners/gearpump/src/test/java/org/apache/beam/runners/gearpump/translators/io/ValueSoureTest.java index 8c50703c3c327..439e1b18a0b81 100644 --- a/runners/gearpump/src/test/java/org/apache/beam/runners/gearpump/translators/io/ValueSoureTest.java +++ b/runners/gearpump/src/test/java/org/apache/beam/runners/gearpump/translators/io/ValueSoureTest.java @@ -41,18 +41,16 @@ import org.junit.Assert; import org.junit.Test; -/** - * Tests for {@link ValuesSource}. - */ +/** Tests for {@link ValuesSource}. */ public class ValueSoureTest { @Test public void testValueSource() { - GearpumpPipelineOptions options = PipelineOptionsFactory.create() - .as(GearpumpPipelineOptions.class); + GearpumpPipelineOptions options = + PipelineOptionsFactory.create().as(GearpumpPipelineOptions.class); Config config = ClusterConfig.master(null); - config = config.withValue(Constants.APPLICATION_TOTAL_RETRIES(), - ConfigValueFactory.fromAnyRef(0)); + config = + config.withValue(Constants.APPLICATION_TOTAL_RETRIES(), ConfigValueFactory.fromAnyRef(0)); EmbeddedCluster cluster = new EmbeddedCluster(config); cluster.start(); @@ -62,8 +60,7 @@ public void testValueSource() { Pipeline p = Pipeline.create(options); List values = Lists.newArrayList("1", "2", "3", "4", "5"); ValuesSource source = new ValuesSource<>(values, StringUtf8Coder.of()); - p.apply(Read.from(source)) - .apply(ParDo.of(new ResultCollector())); + p.apply(Read.from(source)).apply(ParDo.of(new ResultCollector())); p.run().waitUntilFinish(); cluster.stop(); From 44d21ac662e263c09caf2dd3b93b1c325bdfea15 Mon Sep 17 00:00:00 2001 From: manuzhang Date: Thu, 20 Apr 2017 20:59:47 +0800 Subject: [PATCH 151/346] Update gearpump-runner against master changes. --- runners/gearpump/pom.xml | 11 ++- .../gearpump/GearpumpPipelineTranslator.java | 92 ++++++++++--------- .../FlattenPCollectionsTranslator.java | 6 +- .../ParDoMultiOutputTranslator.java | 11 +-- .../translators/TranslationContext.java | 11 +-- .../translators/WindowAssignTranslator.java | 2 +- .../translators/functions/DoFnFunction.java | 2 +- .../translators/io/GearpumpSource.java | 3 +- .../gearpump/translators/io/ValuesSource.java | 3 +- .../translators/utils/DoFnRunnerFactory.java | 3 +- .../translators/utils/NoOpStepContext.java | 3 +- .../FlattenPCollectionsTranslatorTest.java | 48 ++++++---- .../translators/GroupByKeyTranslatorTest.java | 3 +- sdks/java/pom.xml | 2 +- 14 files changed, 113 insertions(+), 87 deletions(-) diff --git a/runners/gearpump/pom.xml b/runners/gearpump/pom.xml index a691801c19369..dcfa3905a8958 100644 --- a/runners/gearpump/pom.xml +++ b/runners/gearpump/pom.xml @@ -43,13 +43,13 @@ UTF-8 UTF-8 - 0.8.3-SNAPSHOT + 0.8.3 - local-runnable-on-service-tests - true + local-validates-runner-tests + false @@ -63,14 +63,15 @@ test - org.apache.beam.sdk.testing.RunnableOnService + org.apache.beam.sdk.testing.ValidatesRunner org.apache.beam.sdk.testing.FlattenWithHeterogeneousCoders, org.apache.beam.sdk.testing.UsesStatefulParDo, org.apache.beam.sdk.testing.UsesTimersInParDo, org.apache.beam.sdk.testing.UsesSplittableParDo, org.apache.beam.sdk.testing.UsesAttemptedMetrics, - org.apache.beam.sdk.testing.UsesCommittedMetrics + org.apache.beam.sdk.testing.UsesCommittedMetrics, + org.apache.beam.sdk.testing.UsesTestStream none true diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpPipelineTranslator.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpPipelineTranslator.java index 1a36343fc3eb3..f5f5e70e14c8f 100644 --- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpPipelineTranslator.java +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpPipelineTranslator.java @@ -18,7 +18,7 @@ package org.apache.beam.runners.gearpump; -import com.google.common.collect.ImmutableMap; +import com.google.common.collect.ImmutableList; import java.util.ArrayList; import java.util.Collections; @@ -27,13 +27,14 @@ import java.util.Map; import org.apache.beam.runners.core.construction.PTransformMatchers; +import org.apache.beam.runners.core.construction.PTransformReplacements; import org.apache.beam.runners.core.construction.SingleInputOutputOverrideFactory; import org.apache.beam.runners.gearpump.translators.CreateGearpumpPCollectionViewTranslator; import org.apache.beam.runners.gearpump.translators.CreatePCollectionViewTranslator; import org.apache.beam.runners.gearpump.translators.FlattenPCollectionsTranslator; import org.apache.beam.runners.gearpump.translators.GroupByKeyTranslator; -import org.apache.beam.runners.gearpump.translators.ParDoBoundMultiTranslator; -import org.apache.beam.runners.gearpump.translators.ParDoBoundTranslator; +import org.apache.beam.runners.gearpump.translators.ParDoMultiOutputTranslator; +import org.apache.beam.runners.gearpump.translators.ParDoSingleOutputTranslator; import org.apache.beam.runners.gearpump.translators.ReadBoundedTranslator; import org.apache.beam.runners.gearpump.translators.ReadUnboundedTranslator; import org.apache.beam.runners.gearpump.translators.TransformTranslator; @@ -45,9 +46,9 @@ import org.apache.beam.sdk.coders.KvCoder; import org.apache.beam.sdk.coders.ListCoder; import org.apache.beam.sdk.io.Read; -import org.apache.beam.sdk.runners.PTransformMatcher; -import org.apache.beam.sdk.runners.PTransformOverrideFactory; +import org.apache.beam.sdk.runners.PTransformOverride; 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; import org.apache.beam.sdk.transforms.Flatten; @@ -72,7 +73,7 @@ * into Gearpump {@link Graph}. */ @SuppressWarnings({"rawtypes", "unchecked"}) -public class GearpumpPipelineTranslator extends Pipeline.PipelineVisitor.Defaults { +public class GearpumpPipelineTranslator implements Pipeline.PipelineVisitor { private static final Logger LOG = LoggerFactory.getLogger( GearpumpPipelineTranslator.class); @@ -88,13 +89,13 @@ public class GearpumpPipelineTranslator extends Pipeline.PipelineVisitor.Default static { // register TransformTranslators - registerTransformTranslator(ParDo.Bound.class, new ParDoBoundTranslator()); + registerTransformTranslator(ParDo.SingleOutput.class, new ParDoSingleOutputTranslator()); registerTransformTranslator(Read.Unbounded.class, new ReadUnboundedTranslator()); registerTransformTranslator(Read.Bounded.class, new ReadBoundedTranslator()); registerTransformTranslator(GroupByKey.class, new GroupByKeyTranslator()); registerTransformTranslator(Flatten.PCollections.class, new FlattenPCollectionsTranslator()); - registerTransformTranslator(ParDo.BoundMulti.class, new ParDoBoundMultiTranslator()); + registerTransformTranslator(ParDo.MultiOutput.class, new ParDoMultiOutputTranslator()); registerTransformTranslator(Window.Assign.class, new WindowAssignTranslator()); registerTransformTranslator(View.CreatePCollectionView.class, new CreatePCollectionViewTranslator()); @@ -107,27 +108,30 @@ public GearpumpPipelineTranslator(TranslationContext translationContext) { } public void translate(Pipeline pipeline) { - Map overrides = - ImmutableMap.builder() - .put(PTransformMatchers.classEqualTo(Combine.GloballyAsSingletonView.class), + List overrides = + ImmutableList.builder() + .add(PTransformOverride.of( + PTransformMatchers.classEqualTo(View.AsMap.class), + new ReflectiveOneToOneOverrideFactory(StreamingViewAsMap.class))) + .add(PTransformOverride.of( + PTransformMatchers.classEqualTo(View.AsMultimap.class), + new ReflectiveOneToOneOverrideFactory(StreamingViewAsMultimap.class))) + .add(PTransformOverride.of( + PTransformMatchers.classEqualTo(View.AsSingleton.class), + new ReflectiveOneToOneOverrideFactory(StreamingViewAsSingleton.class))) + .add(PTransformOverride.of( + PTransformMatchers.classEqualTo(View.AsList.class), + new ReflectiveOneToOneOverrideFactory(StreamingViewAsList.class))) + .add(PTransformOverride.of( + PTransformMatchers.classEqualTo(View.AsIterable.class), + new ReflectiveOneToOneOverrideFactory(StreamingViewAsIterable.class))) + .add(PTransformOverride.of( + PTransformMatchers.classEqualTo(Combine.GloballyAsSingletonView.class), new ReflectiveOneToOneOverrideFactory( - StreamingCombineGloballyAsSingletonView.class)) - .put(PTransformMatchers.classEqualTo(View.AsMap.class), - new ReflectiveOneToOneOverrideFactory(StreamingViewAsMap.class)) - .put(PTransformMatchers.classEqualTo(View.AsMultimap.class), - new ReflectiveOneToOneOverrideFactory(StreamingViewAsMultimap.class)) - .put(PTransformMatchers.classEqualTo(View.AsSingleton.class), - new ReflectiveOneToOneOverrideFactory(StreamingViewAsSingleton.class)) - .put(PTransformMatchers.classEqualTo(View.AsList.class), - new ReflectiveOneToOneOverrideFactory(StreamingViewAsList.class)) - .put(PTransformMatchers.classEqualTo(View.AsIterable.class), - new ReflectiveOneToOneOverrideFactory(StreamingViewAsIterable.class)) + StreamingCombineGloballyAsSingletonView.class))) .build(); - for (Map.Entry override : - overrides.entrySet()) { - pipeline.replace(override.getKey(), override.getValue()); - } + pipeline.replaceAll(overrides); pipeline.traverseTopologically(this); } @@ -185,22 +189,27 @@ TransformTranslator getTransformTranslator(Class transfo // The following codes are forked from DataflowRunner for View translator private static class ReflectiveOneToOneOverrideFactory< - InputT extends PValue, - OutputT extends PValue, - TransformT extends PTransform> - extends SingleInputOutputOverrideFactory { - private final Class> replacement; + InputT, OutputT, TransformT extends PTransform, PCollection>> + extends SingleInputOutputOverrideFactory< + PCollection, PCollection, TransformT> { + private final Class, PCollection>> replacement; private ReflectiveOneToOneOverrideFactory( - Class> replacement) { + Class, PCollection>> replacement) { this.replacement = replacement; } @Override - public PTransform getReplacementTransform(TransformT transform) { - return InstanceBuilder.ofType(replacement) - .withArg((Class>) transform.getClass(), transform) - .build(); + public PTransformReplacement, PCollection> getReplacementTransform( + AppliedPTransform, PCollection, TransformT> transform) { + return PTransformReplacement.of( + PTransformReplacements.getSingletonMainInput(transform), + InstanceBuilder.ofType(replacement) + .withArg( + (Class, PCollection>>) + transform.getTransform().getClass(), + transform.getTransform()) + .build()); } } @@ -220,7 +229,7 @@ public StreamingViewAsMap(View.AsMap transform) {} public PCollectionView> expand(PCollection> input) { PCollectionView> view = PCollectionViews.mapView( - input.getPipeline(), + input, input.getWindowingStrategy(), input.getCoder()); @@ -259,7 +268,7 @@ public StreamingViewAsMultimap(View.AsMultimap transform) {} public PCollectionView>> expand(PCollection> input) { PCollectionView>> view = PCollectionViews.multimapView( - input.getPipeline(), + input, input.getWindowingStrategy(), input.getCoder()); @@ -298,7 +307,7 @@ public StreamingViewAsIterable(View.AsIterable transform) {} public PCollectionView> expand(PCollection input) { PCollectionView> view = PCollectionViews.iterableView( - input.getPipeline(), + input, input.getWindowingStrategy(), input.getCoder()); @@ -328,7 +337,7 @@ public StreamingViewAsList(View.AsList transform) {} public PCollectionView> expand(PCollection input) { PCollectionView> view = PCollectionViews.listView( - input.getPipeline(), + input, input.getWindowingStrategy(), input.getCoder()); @@ -341,6 +350,7 @@ protected String getKindString() { return "StreamingViewAsList"; } } + private static class StreamingCombineGloballyAsSingletonView extends PTransform, PCollectionView> { @@ -360,7 +370,7 @@ public PCollectionView expand(PCollection input) { .withFanout(transform.getFanout())); PCollectionView view = PCollectionViews.singletonView( - combined.getPipeline(), + combined, combined.getWindowingStrategy(), transform.getInsertDefault(), transform.getInsertDefault() diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/FlattenPCollectionsTranslator.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/FlattenPCollectionsTranslator.java index 56f7d1a67d068..5ca05d88f79b1 100644 --- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/FlattenPCollectionsTranslator.java +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/FlattenPCollectionsTranslator.java @@ -29,7 +29,7 @@ import org.apache.beam.sdk.transforms.Flatten; import org.apache.beam.sdk.values.PCollection; -import org.apache.beam.sdk.values.TaggedPValue; +import org.apache.beam.sdk.values.PValue; import org.apache.gearpump.streaming.dsl.api.functions.MapFunction; import org.apache.gearpump.streaming.dsl.javaapi.JavaStream; @@ -45,8 +45,8 @@ public class FlattenPCollectionsTranslator implements public void translate(Flatten.PCollections transform, TranslationContext context) { JavaStream merged = null; Set> unique = new HashSet<>(); - for (TaggedPValue input: context.getInputs()) { - PCollection collection = (PCollection) input.getValue(); + for (PValue input: context.getInputs().values()) { + PCollection collection = (PCollection) input; JavaStream inputStream = context.getInputStream(collection); if (null == merged) { merged = inputStream; diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/ParDoMultiOutputTranslator.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/ParDoMultiOutputTranslator.java index e78568dbd0661..d92979b91d717 100644 --- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/ParDoMultiOutputTranslator.java +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/ParDoMultiOutputTranslator.java @@ -30,7 +30,7 @@ import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionView; -import org.apache.beam.sdk.values.TaggedPValue; +import org.apache.beam.sdk.values.PValue; import org.apache.beam.sdk.values.TupleTag; import org.apache.gearpump.streaming.dsl.api.functions.FilterFunction; @@ -55,11 +55,10 @@ public void translate(ParDo.MultiOutput transform, TranslationC Map> tagsToSideInputs = TranslatorUtils.getTagsToSideInputs(sideInputs); - List outputs = context.getOutputs(); + Map, PValue> outputs = context.getOutputs(); final TupleTag mainOutput = transform.getMainOutputTag(); List> sideOutputs = new ArrayList<>(outputs.size() - 1); - for (TaggedPValue output: outputs) { - TupleTag tag = output.getTag(); + for (TupleTag tag: outputs.keySet()) { if (tag != null && !tag.getId().equals(mainOutput.getId())) { sideOutputs.add(tag); } @@ -78,9 +77,9 @@ public void translate(ParDo.MultiOutput transform, TranslationC tagsToSideInputs, mainOutput, sideOutputs), transform.getName()); - for (TaggedPValue output: outputs) { + for (Map.Entry, PValue> output: outputs.entrySet()) { JavaStream> taggedStream = outputStream - .filter(new FilterByOutputTag(output.getTag().getId()), + .filter(new FilterByOutputTag(output.getKey().getId()), "filter_by_output_tag") .map(new TranslatorUtils.FromRawUnionValue(), "from_RawUnionValue"); context.setOutputStream(output.getValue(), taggedStream); diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/TranslationContext.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/TranslationContext.java index e88bb7409bf87..eb6bc181cc0db 100644 --- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/TranslationContext.java +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/TranslationContext.java @@ -23,7 +23,6 @@ import com.google.common.collect.Iterables; import java.util.HashMap; -import java.util.List; import java.util.Map; import org.apache.beam.runners.gearpump.GearpumpPipelineOptions; @@ -31,7 +30,7 @@ import org.apache.beam.sdk.transforms.AppliedPTransform; import org.apache.beam.sdk.values.PValue; -import org.apache.beam.sdk.values.TaggedPValue; +import org.apache.beam.sdk.values.TupleTag; import org.apache.gearpump.cluster.UserConfig; import org.apache.gearpump.streaming.dsl.javaapi.JavaStream; import org.apache.gearpump.streaming.dsl.javaapi.JavaStreamApp; @@ -71,20 +70,20 @@ public void setOutputStream(PValue output, JavaStream outputS } } - public List getInputs() { + public Map, PValue> getInputs() { return getCurrentTransform().getInputs(); } public PValue getInput() { - return Iterables.getOnlyElement(getInputs()).getValue(); + return Iterables.getOnlyElement(getInputs().values()); } - public List getOutputs() { + public Map, PValue> getOutputs() { return getCurrentTransform().getOutputs(); } public PValue getOutput() { - return Iterables.getOnlyElement(getOutputs()).getValue(); + return Iterables.getOnlyElement(getOutputs().values()); } private AppliedPTransform getCurrentTransform() { diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/WindowAssignTranslator.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/WindowAssignTranslator.java index 2d70b63e24158..149f80ce8cfb7 100644 --- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/WindowAssignTranslator.java +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/WindowAssignTranslator.java @@ -36,7 +36,7 @@ import org.joda.time.Instant; /** - * {@link Window.Bound} is translated to Gearpump flatMap function. + * {@link Window.Assign} is translated to Gearpump flatMap function. */ @SuppressWarnings("unchecked") public class WindowAssignTranslator implements TransformTranslator> { diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/functions/DoFnFunction.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/functions/DoFnFunction.java index 9941e71338d77..3473f530db47d 100644 --- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/functions/DoFnFunction.java +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/functions/DoFnFunction.java @@ -156,7 +156,7 @@ public Iterator flatMap(List input for (WindowedValue value : pushedBackValues) { for (BoundedWindow win: value.getWindows()) { BoundedWindow sideInputWindow = - sideInput.getWindowingStrategyInternal().getWindowFn().getSideInputWindow(win); + sideInput.getWindowMappingFn().getSideInputWindow(win); if (!sideInputReader.isReady(sideInput, sideInputWindow)) { Object emptyValue = WindowedValue.of( Lists.newArrayList(), value.getTimestamp(), sideInputWindow, value.getPane()); diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/io/GearpumpSource.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/io/GearpumpSource.java index c079603429e20..5e79151281629 100644 --- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/io/GearpumpSource.java +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/io/GearpumpSource.java @@ -46,6 +46,7 @@ public abstract class GearpumpSource implements DataSource { private Source.Reader reader; private boolean available = false; + private long count = 0L; GearpumpSource(PipelineOptions options) { try { @@ -112,7 +113,7 @@ public Instant getWatermark() { } } else { if (available) { - return TranslatorUtils.jodaTimeToJava8Time(reader.getCurrentTimestamp()); + return Watermark.MIN(); } else { return Watermark.MAX(); } diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/io/ValuesSource.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/io/ValuesSource.java index ccd5cdf7893c8..b62da192ae6b3 100644 --- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/io/ValuesSource.java +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/io/ValuesSource.java @@ -23,6 +23,7 @@ import java.io.IOException; import java.util.Collections; import java.util.Iterator; +import java.util.List; import java.util.NoSuchElementException; import javax.annotation.Nullable; @@ -68,7 +69,7 @@ private Iterable decode(byte[] bytes) throws IOException{ } @Override - public java.util.List> generateInitialSplits( + public java.util.List> split( int desiredNumSplits, PipelineOptions options) throws Exception { return Collections.singletonList(this); } diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/utils/DoFnRunnerFactory.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/utils/DoFnRunnerFactory.java index bdfc33679d07a..70b4271c24ade 100644 --- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/utils/DoFnRunnerFactory.java +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/utils/DoFnRunnerFactory.java @@ -28,6 +28,7 @@ import org.apache.beam.runners.core.ExecutionContext; import org.apache.beam.runners.core.PushbackSideInputDoFnRunner; import org.apache.beam.runners.core.SimpleDoFnRunner; +import org.apache.beam.runners.core.SimplePushbackSideInputDoFnRunner; import org.apache.beam.runners.gearpump.GearpumpPipelineOptions; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.DoFn; @@ -78,7 +79,7 @@ public PushbackSideInputDoFnRunner createRunner( DoFnRunner underlying = DoFnRunners.simpleRunner( options, fn, sideInputReader, outputManager, mainOutputTag, sideOutputTags, stepContext, aggregatorFactory, windowingStrategy); - return PushbackSideInputDoFnRunner.create(underlying, sideInputs, sideInputReader); + return SimplePushbackSideInputDoFnRunner.create(underlying, sideInputs, sideInputReader); } } diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/utils/NoOpStepContext.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/utils/NoOpStepContext.java index 140df2ad7c476..4e0a74cd6496a 100644 --- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/utils/NoOpStepContext.java +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/utils/NoOpStepContext.java @@ -49,7 +49,8 @@ public void noteOutput(WindowedValue output) { } @Override - public void noteSideOutput(TupleTag tag, WindowedValue output) { + public void noteOutput(TupleTag tag, WindowedValue output) { + } @Override diff --git a/runners/gearpump/src/test/java/org/apache/beam/runners/gearpump/translators/FlattenPCollectionsTranslatorTest.java b/runners/gearpump/src/test/java/org/apache/beam/runners/gearpump/translators/FlattenPCollectionsTranslatorTest.java index fa89d4a480306..ac12fa4151061 100644 --- a/runners/gearpump/src/test/java/org/apache/beam/runners/gearpump/translators/FlattenPCollectionsTranslatorTest.java +++ b/runners/gearpump/src/test/java/org/apache/beam/runners/gearpump/translators/FlattenPCollectionsTranslatorTest.java @@ -26,13 +26,15 @@ import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; -import com.google.common.collect.Lists; import java.util.Collections; +import java.util.HashMap; +import java.util.Map; + import org.apache.beam.runners.gearpump.translators.io.UnboundedSourceWrapper; import org.apache.beam.sdk.transforms.Flatten; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PValue; -import org.apache.beam.sdk.values.TaggedPValue; +import org.apache.beam.sdk.values.TupleTag; import org.apache.gearpump.streaming.dsl.api.functions.MapFunction; import org.apache.gearpump.streaming.dsl.javaapi.JavaStream; import org.apache.gearpump.streaming.source.DataSource; @@ -55,10 +57,10 @@ public boolean matches(Object o) { @Test @SuppressWarnings({"rawtypes", "unchecked"}) public void testTranslateWithEmptyCollection() { - PValue mockOutput = mock(PValue.class); + PCollection mockOutput = mock(PCollection.class); TranslationContext translationContext = mock(TranslationContext.class); - when(translationContext.getInputs()).thenReturn(Collections.EMPTY_LIST); + when(translationContext.getInputs()).thenReturn(Collections.EMPTY_MAP); when(translationContext.getOutput()).thenReturn(mockOutput); translator.translate(transform, translationContext); @@ -71,11 +73,12 @@ public void testTranslateWithOneCollection() { JavaStream javaStream = mock(JavaStream.class); TranslationContext translationContext = mock(TranslationContext.class); - TaggedPValue mockInput = mock(TaggedPValue.class); + Map, PValue> inputs = new HashMap<>(); + TupleTag tag = mock(TupleTag.class); PCollection mockCollection = mock(PCollection.class); - when(mockInput.getValue()).thenReturn(mockCollection); + inputs.put(tag, mockCollection); - when(translationContext.getInputs()).thenReturn(Lists.newArrayList(mockInput)); + when(translationContext.getInputs()).thenReturn(inputs); when(translationContext.getInputStream(mockCollection)).thenReturn(javaStream); PValue mockOutput = mock(PValue.class); @@ -93,22 +96,30 @@ public void testWithMoreThanOneCollections() { JavaStream javaStream1 = mock(JavaStream.class); JavaStream javaStream2 = mock(JavaStream.class); + JavaStream mergedStream = mock(JavaStream.class); TranslationContext translationContext = mock(TranslationContext.class); - TaggedPValue mockInput1 = mock(TaggedPValue.class); + Map, PValue> inputs = new HashMap<>(); + TupleTag tag1 = mock(TupleTag.class); PCollection mockCollection1 = mock(PCollection.class); - when(mockInput1.getValue()).thenReturn(mockCollection1); + inputs.put(tag1, mockCollection1); - TaggedPValue mockInput2 = mock(TaggedPValue.class); + TupleTag tag2 = mock(TupleTag.class); PCollection mockCollection2 = mock(PCollection.class); - when(mockInput2.getValue()).thenReturn(mockCollection2); + inputs.put(tag2, mockCollection2); - when(translationContext.getInputs()).thenReturn(Lists.newArrayList(mockInput1, mockInput2)); + PCollection output = mock(PCollection.class); + + when(translationContext.getInputs()).thenReturn(inputs); when(translationContext.getInputStream(mockCollection1)).thenReturn(javaStream1); when(translationContext.getInputStream(mockCollection2)).thenReturn(javaStream2); + when(javaStream1.merge(javaStream2, transformName)).thenReturn(mergedStream); + when(javaStream2.merge(javaStream1, transformName)).thenReturn(mergedStream); + + when(translationContext.getOutput()).thenReturn(output); translator.translate(transform, translationContext); - verify(javaStream1).merge(javaStream2, transformName); + verify(translationContext).setOutputStream(output, mergedStream); } @Test @@ -120,14 +131,15 @@ public void testWithDuplicatedCollections() { JavaStream javaStream1 = mock(JavaStream.class); TranslationContext translationContext = mock(TranslationContext.class); + Map, PValue> inputs = new HashMap<>(); + TupleTag tag1 = mock(TupleTag.class); PCollection mockCollection1 = mock(PCollection.class); - TaggedPValue mockInput1 = mock(TaggedPValue.class); - when(mockInput1.getValue()).thenReturn(mockCollection1); + inputs.put(tag1, mockCollection1); - TaggedPValue mockInput2 = mock(TaggedPValue.class); - when(mockInput2.getValue()).thenReturn(mockCollection1); + TupleTag tag2 = mock(TupleTag.class); + inputs.put(tag2, mockCollection1); - when(translationContext.getInputs()).thenReturn(Lists.newArrayList(mockInput1, mockInput2)); + when(translationContext.getInputs()).thenReturn(inputs); when(translationContext.getInputStream(mockCollection1)).thenReturn(javaStream1); translator.translate(transform, translationContext); diff --git a/runners/gearpump/src/test/java/org/apache/beam/runners/gearpump/translators/GroupByKeyTranslatorTest.java b/runners/gearpump/src/test/java/org/apache/beam/runners/gearpump/translators/GroupByKeyTranslatorTest.java index 9135022a5605c..4e66ba9b8f6c6 100644 --- a/runners/gearpump/src/test/java/org/apache/beam/runners/gearpump/translators/GroupByKeyTranslatorTest.java +++ b/runners/gearpump/src/test/java/org/apache/beam/runners/gearpump/translators/GroupByKeyTranslatorTest.java @@ -133,7 +133,8 @@ public void testMerge() { PaneInfo.NO_FIRING); KV>>> result1 = - merge.fold(KV.of(null, null), KV.of(key1, value1)); + merge.fold(KV.>>>of( + null, null), KV.of(key1, value1)); assertThat(result1.getKey(), equalTo(key1)); assertThat(result1.getValue().getValue().getValue(), equalTo(Lists.newArrayList("value1"))); diff --git a/sdks/java/pom.xml b/sdks/java/pom.xml index 7ca61090e7b21..21b5841c34830 100644 --- a/sdks/java/pom.xml +++ b/sdks/java/pom.xml @@ -38,7 +38,7 @@ build-tools --> core io - maven-archetypes + extensions + true From 98854d4d01ca526ea4a44dc077d2cfb4cddf9914 Mon Sep 17 00:00:00 2001 From: manuzhang Date: Fri, 19 May 2017 09:19:42 +0800 Subject: [PATCH 156/346] Respect WindowFn#getOutputTime in gearpump-runner --- .../gearpump/translators/GroupByKeyTranslator.java | 12 ++++++++---- .../translators/GroupByKeyTranslatorTest.java | 8 ++++---- 2 files changed, 12 insertions(+), 8 deletions(-) diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/GroupByKeyTranslator.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/GroupByKeyTranslator.java index 521f665fca5b2..7d944a41580c5 100644 --- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/GroupByKeyTranslator.java +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/GroupByKeyTranslator.java @@ -74,7 +74,7 @@ public void translate(GroupByKey transform, TranslationContext context) { new GearpumpWindowFn(windowFn.isNonMerging()), EventTimeTrigger$.MODULE$, Discarding$.MODULE$), "assign_window") .groupBy(new GroupByFn(inputKeyCoder), parallelism, "group_by_Key_and_Window") - .map(new KeyedByTimestamp(timestampCombiner), "keyed_by_timestamp") + .map(new KeyedByTimestamp(windowFn, timestampCombiner), "keyed_by_timestamp") .fold(new Merge<>(windowFn, timestampCombiner), "merge") .map(new Values(), "values"); @@ -146,17 +146,21 @@ protected static class KeyedByTimestamp extends MapFunction>, KV>>> { + private final WindowFn, BoundedWindow> windowFn; private final TimestampCombiner timestampCombiner; - public KeyedByTimestamp(TimestampCombiner timestampCombiner) { + public KeyedByTimestamp(WindowFn, BoundedWindow> windowFn, + TimestampCombiner timestampCombiner) { + this.windowFn = windowFn; this.timestampCombiner = timestampCombiner; } @Override public KV>> map( WindowedValue> wv) { - Instant timestamp = timestampCombiner.assign( - Iterables.getOnlyElement(wv.getWindows()), wv.getTimestamp()); + BoundedWindow window = Iterables.getOnlyElement(wv.getWindows()); + Instant timestamp = timestampCombiner.assign(window + , windowFn.getOutputTime(wv.getTimestamp(), window)); return KV.of(timestamp, wv); } } diff --git a/runners/gearpump/src/test/java/org/apache/beam/runners/gearpump/translators/GroupByKeyTranslatorTest.java b/runners/gearpump/src/test/java/org/apache/beam/runners/gearpump/translators/GroupByKeyTranslatorTest.java index 86b60aa385156..d5b931b78868b 100644 --- a/runners/gearpump/src/test/java/org/apache/beam/runners/gearpump/translators/GroupByKeyTranslatorTest.java +++ b/runners/gearpump/src/test/java/org/apache/beam/runners/gearpump/translators/GroupByKeyTranslatorTest.java @@ -22,7 +22,6 @@ import static org.junit.Assert.assertThat; import com.google.common.collect.ImmutableList; -import com.google.common.collect.Iterables; import com.google.common.collect.Lists; import java.time.Instant; @@ -95,18 +94,19 @@ public static Iterable data() { @Test @SuppressWarnings({"rawtypes", "unchecked"}) public void testKeyedByTimestamp() { + WindowFn slidingWindows = Sessions.withGapDuration(Duration.millis(10)); BoundedWindow window = new IntervalWindow(new org.joda.time.Instant(0), new org.joda.time.Instant(10)); GroupByKeyTranslator.KeyedByTimestamp keyedByTimestamp = - new GroupByKeyTranslator.KeyedByTimestamp(timestampCombiner); + new GroupByKeyTranslator.KeyedByTimestamp(slidingWindows, timestampCombiner); WindowedValue> value = WindowedValue.of( KV.of("key", "val"), org.joda.time.Instant.now(), window, PaneInfo.NO_FIRING); KV>> result = keyedByTimestamp.map(value); org.joda.time.Instant time = - timestampCombiner.assign(Iterables.getOnlyElement(value.getWindows()), - value.getTimestamp()); + timestampCombiner.assign(window, + slidingWindows.getOutputTime(value.getTimestamp(), window)); assertThat(result, equalTo(KV.of(time, value))); } From 7653e7ed6de3d9db822dcd390d2bf70819954fa5 Mon Sep 17 00:00:00 2001 From: manuzhang Date: Wed, 7 Jun 2017 14:08:04 +0800 Subject: [PATCH 157/346] Fix side input handling in DoFnFunction --- .../translators/TranslationContext.java | 2 ++ .../translators/functions/DoFnFunction.java | 23 ++++--------------- .../translators/io/GearpumpSource.java | 1 - .../translators/utils/TranslatorUtils.java | 5 ++-- 4 files changed, 8 insertions(+), 23 deletions(-) diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/TranslationContext.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/TranslationContext.java index 4090354b03564..64a1e0dd8196d 100644 --- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/TranslationContext.java +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/TranslationContext.java @@ -68,6 +68,8 @@ public JavaStream getInputStream(PValue input) { public void setOutputStream(PValue output, JavaStream outputStream) { if (!streams.containsKey(output)) { streams.put(output, outputStream); + } else { + throw new RuntimeException("set stream for duplicated output " + output); } } diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/functions/DoFnFunction.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/functions/DoFnFunction.java index f521d7b0579fa..6e4fbebd8892a 100644 --- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/functions/DoFnFunction.java +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/functions/DoFnFunction.java @@ -24,7 +24,6 @@ import java.io.Serializable; import java.util.Collection; -import java.util.HashMap; import java.util.HashSet; import java.util.Iterator; import java.util.LinkedList; @@ -66,7 +65,6 @@ public class DoFnFunction extends private transient PushbackSideInputDoFnRunner doFnRunner; private transient SideInputHandler sideInputReader; private transient List> pushedBackValues; - private transient Map, List>>> sideInputValues; private final Collection> sideInputs; private final Map> tagsToSideInputs; private final TupleTag mainOutput; @@ -109,7 +107,6 @@ public void setup() { doFnRunner = doFnRunnerFactory.createRunner(sideInputReader); pushedBackValues = new LinkedList<>(); - sideInputValues = new HashMap<>(); outputManager.setup(mainOutput, sideOutputs); } @@ -132,25 +129,14 @@ public Iterator flatMap(List input } else { // side input PCollectionView sideInput = tagsToSideInputs.get(unionValue.getUnionTag()); - WindowedValue sideInputValue = - (WindowedValue) unionValue.getValue(); - Object value = sideInputValue.getValue(); - if (!(value instanceof Iterable)) { - sideInputValue = sideInputValue.withValue(Lists.newArrayList(value)); - } - if (!sideInputValues.containsKey(sideInput)) { - sideInputValues.put(sideInput, new LinkedList>>()); - } - sideInputValues.get(sideInput).add((WindowedValue>) sideInputValue); + WindowedValue> sideInputValue = + (WindowedValue>) unionValue.getValue(); + sideInputReader.addSideInputValue(sideInput, sideInputValue); } } + for (PCollectionView sideInput: sideInputs) { - if (sideInputValues.containsKey(sideInput)) { - for (WindowedValue> value: sideInputValues.get(sideInput)) { - sideInputReader.addSideInputValue(sideInput, value); - } - } for (WindowedValue value : pushedBackValues) { for (BoundedWindow win: value.getWindows()) { BoundedWindow sideInputWindow = @@ -171,7 +157,6 @@ public Iterator flatMap(List input } pushedBackValues.clear(); Iterables.addAll(pushedBackValues, nextPushedBackValues); - sideInputValues.clear(); doFnRunner.finishBundle(); diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/io/GearpumpSource.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/io/GearpumpSource.java index 5e79151281629..60f319dccf8ce 100644 --- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/io/GearpumpSource.java +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/io/GearpumpSource.java @@ -28,7 +28,6 @@ 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.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.util.WindowedValue; diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/utils/TranslatorUtils.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/utils/TranslatorUtils.java index 999afae8a7f8a..282f2613cc942 100644 --- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/utils/TranslatorUtils.java +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/utils/TranslatorUtils.java @@ -70,11 +70,10 @@ public static JavaStream withSideInputStream( JavaStream> inputStream, Map> tagsToSideInputs) { JavaStream mainStream = - inputStream.map(new ToRawUnionValue("0"), "map_to_RawUnionValue"); + inputStream.map(new ToRawUnionValue<>("0"), "map_to_RawUnionValue"); for (Map.Entry> tagToSideInput: tagsToSideInputs.entrySet()) { - // actually JavaStream>> - JavaStream> sideInputStream = context.getInputStream( + JavaStream>> sideInputStream = context.getInputStream( tagToSideInput.getValue()); mainStream = mainStream.merge(sideInputStream.map(new ToRawUnionValue<>( tagToSideInput.getKey()), "map_to_RawUnionValue"), "merge_to_MainStream"); From f61822d41653def8332cb3cc55140685c3dd75a2 Mon Sep 17 00:00:00 2001 From: manuzhang Date: Wed, 7 Jun 2017 14:06:43 +0800 Subject: [PATCH 158/346] upgrade to gearpump 0.8.4-SNAPSHOT --- runners/gearpump/pom.xml | 2 +- .../gearpump/GearpumpPipelineTranslator.java | 4 +++- .../CreateGearpumpPCollectionViewTranslator.java | 2 +- .../translators/FlattenPCollectionsTranslator.java | 2 +- .../gearpump/translators/GroupByKeyTranslator.java | 13 +++++++++---- .../translators/functions/DoFnFunction.java | 1 + .../gearpump/translators/io/GearpumpSource.java | 5 +++-- .../gearpump/translators/utils/TranslatorUtils.java | 2 +- .../FlattenPCollectionsTranslatorTest.java | 6 +++--- .../gearpump/translators/io/GearpumpSourceTest.java | 3 ++- 10 files changed, 25 insertions(+), 15 deletions(-) diff --git a/runners/gearpump/pom.xml b/runners/gearpump/pom.xml index 7e39a484db4c0..3c98d5e5fa235 100644 --- a/runners/gearpump/pom.xml +++ b/runners/gearpump/pom.xml @@ -43,7 +43,7 @@ UTF-8 UTF-8 - 0.8.3 + 0.8.4-SNAPSHOT diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpPipelineTranslator.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpPipelineTranslator.java index daf65d9406c76..58b44a3e9b6c7 100644 --- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpPipelineTranslator.java +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpPipelineTranslator.java @@ -54,6 +54,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.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.Window; import org.apache.beam.sdk.util.InstanceBuilder; import org.apache.beam.sdk.values.KV; @@ -143,7 +144,7 @@ public void leaveCompositeTransform(TransformHierarchy.Node node) { @Override public void visitPrimitiveTransform(TransformHierarchy.Node node) { - LOG.debug("visiting transform {}", node.getTransform()); + LOG.info("visiting transform {}", node.getTransform()); PTransform transform = node.getTransform(); TransformTranslator translator = getTransformTranslator(transform.getClass()); if (null == translator) { @@ -346,6 +347,7 @@ protected String getKindString() { } } + private static class StreamingCombineGloballyAsSingletonView extends PTransform, PCollectionView> { diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/CreateGearpumpPCollectionViewTranslator.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/CreateGearpumpPCollectionViewTranslator.java index c7f24a86ec51e..d7588c21c4aa2 100644 --- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/CreateGearpumpPCollectionViewTranslator.java +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/CreateGearpumpPCollectionViewTranslator.java @@ -41,6 +41,6 @@ public void translate( JavaStream>> inputStream = context.getInputStream(context.getInput()); PCollectionView view = (PCollectionView) context.getOutput(); - context.setOutputStream(view, inputStream); + context.setOutputStream(view.getPCollection(), inputStream); } } diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/FlattenPCollectionsTranslator.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/FlattenPCollectionsTranslator.java index 5ca05d88f79b1..8cc0058bcc90b 100644 --- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/FlattenPCollectionsTranslator.java +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/FlattenPCollectionsTranslator.java @@ -57,7 +57,7 @@ public void translate(Flatten.PCollections transform, TranslationContext cont inputStream = inputStream.map(new DummyFunction(), "dummy"); } - merged = merged.merge(inputStream, transform.getName()); + merged = merged.merge(inputStream, 1, transform.getName()); } unique.add(collection); } diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/GroupByKeyTranslator.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/GroupByKeyTranslator.java index 7d944a41580c5..8409beb93575e 100644 --- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/GroupByKeyTranslator.java +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/GroupByKeyTranslator.java @@ -72,7 +72,7 @@ public void translate(GroupByKey transform, TranslationContext context) { JavaStream>>> outputStream = inputStream .window(Windows.apply( new GearpumpWindowFn(windowFn.isNonMerging()), - EventTimeTrigger$.MODULE$, Discarding$.MODULE$), "assign_window") + EventTimeTrigger$.MODULE$, Discarding$.MODULE$, windowFn.toString())) .groupBy(new GroupByFn(inputKeyCoder), parallelism, "group_by_Key_and_Window") .map(new KeyedByTimestamp(windowFn, timestampCombiner), "keyed_by_timestamp") .fold(new Merge<>(windowFn, timestampCombiner), "merge") @@ -85,7 +85,7 @@ public void translate(GroupByKey transform, TranslationContext context) { * A transform used internally to translate Beam's Window to Gearpump's Window. */ protected static class GearpumpWindowFn - implements WindowFunction>, Serializable { + implements WindowFunction, Serializable { private final boolean isNonMerging; @@ -94,9 +94,14 @@ public GearpumpWindowFn(boolean isNonMerging) { } @Override - public Window[] apply(Context> context) { + public Window[] apply(Context context) { try { - return toGearpumpWindows(context.element().getWindows().toArray(new BoundedWindow[0])); + Object element = context.element(); + if (element instanceof TranslatorUtils.RawUnionValue) { + element = ((TranslatorUtils.RawUnionValue) element).getValue(); + } + return toGearpumpWindows(((WindowedValue) element).getWindows() + .toArray(new BoundedWindow[0])); } catch (Exception e) { throw new RuntimeException(e); } diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/functions/DoFnFunction.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/functions/DoFnFunction.java index 6e4fbebd8892a..e2777df13f211 100644 --- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/functions/DoFnFunction.java +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/functions/DoFnFunction.java @@ -145,6 +145,7 @@ public Iterator flatMap(List input Object emptyValue = WindowedValue.of( Lists.newArrayList(), value.getTimestamp(), sideInputWindow, value.getPane()); sideInputReader.addSideInputValue(sideInput, (WindowedValue>) emptyValue); + System.out.println(sideInput + " in " + sideInputWindow.toString() + " not ready"); } } } diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/io/GearpumpSource.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/io/GearpumpSource.java index 60f319dccf8ce..6637a9bdbf08f 100644 --- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/io/GearpumpSource.java +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/io/GearpumpSource.java @@ -31,6 +31,7 @@ import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.util.WindowedValue; +import org.apache.gearpump.DefaultMessage; import org.apache.gearpump.Message; import org.apache.gearpump.streaming.source.DataSource; import org.apache.gearpump.streaming.source.Watermark; @@ -77,9 +78,9 @@ public Message read() { if (available) { T data = reader.getCurrent(); org.joda.time.Instant timestamp = reader.getCurrentTimestamp(); - message = Message.apply( + message = new DefaultMessage( WindowedValue.timestampedValueInGlobalWindow(data, timestamp), - timestamp.getMillis()); + TranslatorUtils.jodaTimeToJava8Time(timestamp)); } available = reader.advance(); } catch (Exception e) { diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/utils/TranslatorUtils.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/utils/TranslatorUtils.java index 282f2613cc942..83fc6e6953120 100644 --- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/utils/TranslatorUtils.java +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/utils/TranslatorUtils.java @@ -76,7 +76,7 @@ public static JavaStream withSideInputStream( JavaStream>> sideInputStream = context.getInputStream( tagToSideInput.getValue()); mainStream = mainStream.merge(sideInputStream.map(new ToRawUnionValue<>( - tagToSideInput.getKey()), "map_to_RawUnionValue"), "merge_to_MainStream"); + tagToSideInput.getKey()), "map_to_RawUnionValue"), 1, "merge_to_MainStream"); } return mainStream; } diff --git a/runners/gearpump/src/test/java/org/apache/beam/runners/gearpump/translators/FlattenPCollectionsTranslatorTest.java b/runners/gearpump/src/test/java/org/apache/beam/runners/gearpump/translators/FlattenPCollectionsTranslatorTest.java index ac12fa4151061..1262177652c64 100644 --- a/runners/gearpump/src/test/java/org/apache/beam/runners/gearpump/translators/FlattenPCollectionsTranslatorTest.java +++ b/runners/gearpump/src/test/java/org/apache/beam/runners/gearpump/translators/FlattenPCollectionsTranslatorTest.java @@ -113,8 +113,8 @@ public void testWithMoreThanOneCollections() { when(translationContext.getInputs()).thenReturn(inputs); when(translationContext.getInputStream(mockCollection1)).thenReturn(javaStream1); when(translationContext.getInputStream(mockCollection2)).thenReturn(javaStream2); - when(javaStream1.merge(javaStream2, transformName)).thenReturn(mergedStream); - when(javaStream2.merge(javaStream1, transformName)).thenReturn(mergedStream); + when(javaStream1.merge(javaStream2, 1, transformName)).thenReturn(mergedStream); + when(javaStream2.merge(javaStream1, 1, transformName)).thenReturn(mergedStream); when(translationContext.getOutput()).thenReturn(output); @@ -144,6 +144,6 @@ public void testWithDuplicatedCollections() { translator.translate(transform, translationContext); verify(javaStream1).map(any(MapFunction.class), eq("dummy")); - verify(javaStream1).merge(any(JavaStream.class), eq(transformName)); + verify(javaStream1).merge(any(JavaStream.class), eq(1), eq(transformName)); } } diff --git a/runners/gearpump/src/test/java/org/apache/beam/runners/gearpump/translators/io/GearpumpSourceTest.java b/runners/gearpump/src/test/java/org/apache/beam/runners/gearpump/translators/io/GearpumpSourceTest.java index b244484cbeb82..44907376c3e7d 100644 --- a/runners/gearpump/src/test/java/org/apache/beam/runners/gearpump/translators/io/GearpumpSourceTest.java +++ b/runners/gearpump/src/test/java/org/apache/beam/runners/gearpump/translators/io/GearpumpSourceTest.java @@ -32,6 +32,7 @@ import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.TimestampedValue; +import org.apache.gearpump.DefaultMessage; import org.apache.gearpump.Message; import org.apache.gearpump.streaming.source.Watermark; import org.junit.Assert; @@ -77,7 +78,7 @@ public void testGearpumpSource() { Assert.assertEquals(expectedWaterMark, sourceForTest.getWatermark()); Message expectedMsg = - Message.apply( + new DefaultMessage( WindowedValue.timestampedValueInGlobalWindow(value, value.getTimestamp()), value.getTimestamp().getMillis()); Message message = sourceForTest.read(); From a7b5d981a57bae38a5ef70feb0c6bd167ce22ab0 Mon Sep 17 00:00:00 2001 From: manuzhang Date: Sat, 17 Jun 2017 11:11:07 +0800 Subject: [PATCH 159/346] Fix PCollectionView translation --- .../gearpump/GearpumpPipelineTranslator.java | 526 ------------------ .../beam/runners/gearpump/GearpumpRunner.java | 1 + ...eateGearpumpPCollectionViewTranslator.java | 9 +- .../CreateStreamingGearpumpView.java | 157 ++++++ .../GearpumpPipelineTranslator.java | 143 +++++ .../translators/TranslationContext.java | 3 +- .../translators/functions/DoFnFunction.java | 1 - .../translators/io/GearpumpSource.java | 2 +- .../translators/utils/TranslatorUtils.java | 1 + ...GearpumpPCollectionViewTranslatorTest.java | 7 +- .../translators/io/GearpumpSourceTest.java | 18 +- .../utils/TranslatorUtilsTest.java | 5 +- 12 files changed, 327 insertions(+), 546 deletions(-) delete mode 100644 runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpPipelineTranslator.java create mode 100644 runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/CreateStreamingGearpumpView.java create mode 100644 runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/GearpumpPipelineTranslator.java diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpPipelineTranslator.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpPipelineTranslator.java deleted file mode 100644 index 58b44a3e9b6c7..0000000000000 --- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpPipelineTranslator.java +++ /dev/null @@ -1,526 +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.gearpump; - -import com.google.common.collect.ImmutableList; - -import java.util.ArrayList; -import java.util.Collections; -import java.util.HashMap; -import java.util.List; -import java.util.Map; - -import org.apache.beam.runners.core.construction.PTransformMatchers; -import org.apache.beam.runners.core.construction.PTransformReplacements; -import org.apache.beam.runners.core.construction.SingleInputOutputOverrideFactory; -import org.apache.beam.runners.gearpump.translators.CreateGearpumpPCollectionViewTranslator; -import org.apache.beam.runners.gearpump.translators.FlattenPCollectionsTranslator; -import org.apache.beam.runners.gearpump.translators.GroupByKeyTranslator; -import org.apache.beam.runners.gearpump.translators.ParDoMultiOutputTranslator; -import org.apache.beam.runners.gearpump.translators.ReadBoundedTranslator; -import org.apache.beam.runners.gearpump.translators.ReadUnboundedTranslator; -import org.apache.beam.runners.gearpump.translators.TransformTranslator; -import org.apache.beam.runners.gearpump.translators.TranslationContext; -import org.apache.beam.runners.gearpump.translators.WindowAssignTranslator; -import org.apache.beam.sdk.Pipeline; -import org.apache.beam.sdk.coders.Coder; -import org.apache.beam.sdk.coders.CoderRegistry; -import org.apache.beam.sdk.coders.KvCoder; -import org.apache.beam.sdk.coders.ListCoder; -import org.apache.beam.sdk.io.Read; -import org.apache.beam.sdk.runners.AppliedPTransform; -import org.apache.beam.sdk.runners.PTransformOverride; -import org.apache.beam.sdk.runners.TransformHierarchy; -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.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.util.InstanceBuilder; -import org.apache.beam.sdk.values.KV; -import org.apache.beam.sdk.values.PCollection; -import org.apache.beam.sdk.values.PCollectionView; -import org.apache.beam.sdk.values.PCollectionViews; -import org.apache.beam.sdk.values.PValue; - -import org.apache.gearpump.util.Graph; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -/** - * {@link GearpumpPipelineTranslator} knows how to translate {@link Pipeline} objects - * into Gearpump {@link Graph}. - */ -@SuppressWarnings({"rawtypes", "unchecked"}) -public class GearpumpPipelineTranslator extends Pipeline.PipelineVisitor.Defaults { - - private static final Logger LOG = LoggerFactory.getLogger( - GearpumpPipelineTranslator.class); - - /** - * A map from {@link PTransform} subclass to the corresponding - * {@link TransformTranslator} to use to translate that transform. - */ - private static final Map, TransformTranslator> - transformTranslators = new HashMap<>(); - - private final TranslationContext translationContext; - - static { - // register TransformTranslators - registerTransformTranslator(Read.Unbounded.class, new ReadUnboundedTranslator()); - registerTransformTranslator(Read.Bounded.class, new ReadBoundedTranslator()); - registerTransformTranslator(GroupByKey.class, new GroupByKeyTranslator()); - registerTransformTranslator(Flatten.PCollections.class, - new FlattenPCollectionsTranslator()); - registerTransformTranslator(ParDo.MultiOutput.class, new ParDoMultiOutputTranslator()); - registerTransformTranslator(Window.Assign.class, new WindowAssignTranslator()); - registerTransformTranslator(CreateGearpumpPCollectionView.class, - new CreateGearpumpPCollectionViewTranslator<>()); - } - - public GearpumpPipelineTranslator(TranslationContext translationContext) { - this.translationContext = translationContext; - } - - public void translate(Pipeline pipeline) { - List overrides = - ImmutableList.builder() - .add(PTransformOverride.of( - PTransformMatchers.classEqualTo(View.AsMap.class), - new ReflectiveOneToOneOverrideFactory(StreamingViewAsMap.class))) - .add(PTransformOverride.of( - PTransformMatchers.classEqualTo(View.AsMultimap.class), - new ReflectiveOneToOneOverrideFactory(StreamingViewAsMultimap.class))) - .add(PTransformOverride.of( - PTransformMatchers.classEqualTo(View.AsSingleton.class), - new ReflectiveOneToOneOverrideFactory(StreamingViewAsSingleton.class))) - .add(PTransformOverride.of( - PTransformMatchers.classEqualTo(View.AsList.class), - new ReflectiveOneToOneOverrideFactory(StreamingViewAsList.class))) - .add(PTransformOverride.of( - PTransformMatchers.classEqualTo(View.AsIterable.class), - new ReflectiveOneToOneOverrideFactory(StreamingViewAsIterable.class))) - .add(PTransformOverride.of( - PTransformMatchers.classEqualTo(Combine.GloballyAsSingletonView.class), - new ReflectiveOneToOneOverrideFactory( - StreamingCombineGloballyAsSingletonView.class))) - .build(); - - pipeline.replaceAll(overrides); - pipeline.traverseTopologically(this); - } - - @Override - public CompositeBehavior enterCompositeTransform(TransformHierarchy.Node node) { - LOG.debug("entering composite transform {}", node.getTransform()); - return CompositeBehavior.ENTER_TRANSFORM; - } - - @Override - public void leaveCompositeTransform(TransformHierarchy.Node node) { - LOG.debug("leaving composite transform {}", node.getTransform()); - } - - @Override - public void visitPrimitiveTransform(TransformHierarchy.Node node) { - LOG.info("visiting transform {}", node.getTransform()); - PTransform transform = node.getTransform(); - TransformTranslator translator = getTransformTranslator(transform.getClass()); - if (null == translator) { - throw new IllegalStateException( - "no translator registered for " + transform); - } - translationContext.setCurrentTransform(node, getPipeline()); - translator.translate(transform, translationContext); - } - - @Override - public void visitValue(PValue value, TransformHierarchy.Node producer) { - LOG.debug("visiting value {}", value); - } - - /** - * Records that instances of the specified PTransform class - * should be translated by default by the corresponding - * {@link TransformTranslator}. - */ - private static void registerTransformTranslator( - Class transformClass, - TransformTranslator transformTranslator) { - if (transformTranslators.put(transformClass, transformTranslator) != null) { - throw new IllegalArgumentException( - "defining multiple translators for " + transformClass); - } - } - - /** - * Returns the {@link TransformTranslator} to use for instances of the - * specified PTransform class, or null if none registered. - */ - private - TransformTranslator getTransformTranslator(Class transformClass) { - return transformTranslators.get(transformClass); - } - - // The following codes are forked from DataflowRunner for View translator - private static class ReflectiveOneToOneOverrideFactory< - InputT, OutputT, TransformT extends PTransform, PCollection>> - extends SingleInputOutputOverrideFactory< - PCollection, PCollection, TransformT> { - private final Class, PCollection>> replacement; - - private ReflectiveOneToOneOverrideFactory( - Class, PCollection>> replacement) { - this.replacement = replacement; - } - - @Override - public PTransformReplacement, PCollection> getReplacementTransform( - AppliedPTransform, PCollection, TransformT> transform) { - return PTransformReplacement.of( - PTransformReplacements.getSingletonMainInput(transform), - InstanceBuilder.ofType(replacement) - .withArg( - (Class, PCollection>>) - transform.getTransform().getClass(), - transform.getTransform()) - .build()); - } - } - - /** - * Specialized implementation for - * {@link org.apache.beam.sdk.transforms.View.AsMap View.AsMap} - * for the Gearpump runner. - */ - private static class StreamingViewAsMap - extends PTransform>, PCollectionView>> { - - private static final long serialVersionUID = 4791080760092950304L; - - public StreamingViewAsMap(View.AsMap transform) {} - - @Override - public PCollectionView> expand(PCollection> input) { - PCollectionView> view = - PCollectionViews.mapView( - input, - input.getWindowingStrategy(), - input.getCoder()); - - @SuppressWarnings({"rawtypes", "unchecked"}) - KvCoder inputCoder = (KvCoder) input.getCoder(); - try { - inputCoder.getKeyCoder().verifyDeterministic(); - } catch (Coder.NonDeterministicException e) { - // throw new RuntimeException(e); - } - - return input - .apply(Combine.globally(new Concatenate>()).withoutDefaults()) - .apply(CreateGearpumpPCollectionView., Map>of(view)); - } - - @Override - protected String getKindString() { - return "StreamingViewAsMap"; - } - } - - /** - * Specialized expansion for {@link - * org.apache.beam.sdk.transforms.View.AsMultimap View.AsMultimap} for the - * Gearpump runner. - */ - private static class StreamingViewAsMultimap - extends PTransform>, PCollectionView>>> { - - private static final long serialVersionUID = 5854899081751333352L; - - public StreamingViewAsMultimap(View.AsMultimap transform) {} - - @Override - public PCollectionView>> expand(PCollection> input) { - PCollectionView>> view = - PCollectionViews.multimapView( - input, - input.getWindowingStrategy(), - input.getCoder()); - - @SuppressWarnings({"rawtypes", "unchecked"}) - KvCoder inputCoder = (KvCoder) input.getCoder(); - try { - inputCoder.getKeyCoder().verifyDeterministic(); - } catch (Coder.NonDeterministicException e) { - // throw new RuntimeException(e); - } - - return input - .apply(Combine.globally(new Concatenate>()).withoutDefaults()) - .apply(CreateGearpumpPCollectionView., Map>>of(view)); - } - - @Override - protected String getKindString() { - return "StreamingViewAsMultimap"; - } - } - - /** - * Specialized implementation for - * {@link org.apache.beam.sdk.transforms.View.AsIterable View.AsIterable} for the - * Gearpump runner. - */ - private static class StreamingViewAsIterable - extends PTransform, PCollectionView>> { - - private static final long serialVersionUID = -3399860618995613421L; - - public StreamingViewAsIterable(View.AsIterable transform) {} - - @Override - public PCollectionView> expand(PCollection input) { - PCollectionView> view = - PCollectionViews.iterableView( - input, - input.getWindowingStrategy(), - input.getCoder()); - - return input.apply(Combine.globally(new Concatenate()).withoutDefaults()) - .apply(CreateGearpumpPCollectionView.>of(view)); - } - - @Override - protected String getKindString() { - return "StreamingViewAsIterable"; - } - } - - /** - * Specialized implementation for - * {@link org.apache.beam.sdk.transforms.View.AsList View.AsList} for the - * Gearpump runner. - */ - private static class StreamingViewAsList - extends PTransform, PCollectionView>> { - - private static final long serialVersionUID = -5018631473886330629L; - - public StreamingViewAsList(View.AsList transform) {} - - @Override - public PCollectionView> expand(PCollection input) { - PCollectionView> view = - PCollectionViews.listView( - input, - input.getWindowingStrategy(), - input.getCoder()); - - return input.apply(Combine.globally(new Concatenate()).withoutDefaults()) - .apply(CreateGearpumpPCollectionView.>of(view)); - } - - @Override - protected String getKindString() { - return "StreamingViewAsList"; - } - } - - - private static class StreamingCombineGloballyAsSingletonView - extends PTransform, PCollectionView> { - - private static final long serialVersionUID = 9064900748869035738L; - private final Combine.GloballyAsSingletonView transform; - - public StreamingCombineGloballyAsSingletonView( - Combine.GloballyAsSingletonView transform) { - this.transform = transform; - } - - @Override - public PCollectionView expand(PCollection input) { - PCollection combined = - input.apply(Combine.globally(transform.getCombineFn()) - .withoutDefaults() - .withFanout(transform.getFanout())); - - PCollectionView view = PCollectionViews.singletonView( - combined, - combined.getWindowingStrategy(), - transform.getInsertDefault(), - transform.getInsertDefault() - ? transform.getCombineFn().defaultValue() : null, - combined.getCoder()); - return combined - .apply(ParDo.of(new WrapAsList())) - .apply(CreateGearpumpPCollectionView.of(view)); - } - - @Override - protected String getKindString() { - return "StreamingCombineGloballyAsSingletonView"; - } - } - - private static class StreamingViewAsSingleton - extends PTransform, PCollectionView> { - - private static final long serialVersionUID = 5870455965625071546L; - private final View.AsSingleton transform; - - public StreamingViewAsSingleton(View.AsSingleton transform) { - this.transform = transform; - } - - @Override - public PCollectionView expand(PCollection input) { - Combine.Globally combine = Combine.globally( - new SingletonCombine<>(transform.hasDefaultValue(), transform.defaultValue())); - if (!transform.hasDefaultValue()) { - combine = combine.withoutDefaults(); - } - return input.apply(combine.asSingletonView()); - } - - @Override - protected String getKindString() { - return "StreamingViewAsSingleton"; - } - - private static class SingletonCombine extends Combine.BinaryCombineFn { - private boolean hasDefaultValue; - private T defaultValue; - - SingletonCombine(boolean hasDefaultValue, T defaultValue) { - this.hasDefaultValue = hasDefaultValue; - this.defaultValue = defaultValue; - } - - @Override - public T apply(T left, T right) { - throw new IllegalArgumentException("PCollection with more than one element " - + "accessed as a singleton view. Consider using Combine.globally().asSingleton() to " - + "combine the PCollection into a single value"); - } - - @Override - public T identity() { - if (hasDefaultValue) { - return defaultValue; - } else { - throw new IllegalArgumentException( - "Empty PCollection accessed as a singleton view. " - + "Consider setting withDefault to provide a default value"); - } - } - } - } - - private static class WrapAsList extends DoFn> { - @ProcessElement - public void processElement(ProcessContext c) { - c.output(Collections.singletonList(c.element())); - } - } - - /** - * Creates a primitive {@link PCollectionView}. - * - *

              For internal use only by runner implementors. - * - * @param The type of the elements of the input PCollection - * @param The type associated with the {@link PCollectionView} used as a side input - */ - public static class CreateGearpumpPCollectionView - extends PTransform>, PCollectionView> { - private static final long serialVersionUID = -2637073020800540542L; - private PCollectionView view; - - private CreateGearpumpPCollectionView(PCollectionView view) { - this.view = view; - } - - public static CreateGearpumpPCollectionView of( - PCollectionView view) { - return new CreateGearpumpPCollectionView<>(view); - } - - public PCollectionView getView() { - return view; - } - - @Override - public PCollectionView expand(PCollection> input) { - return view; - } - } - - /** - * Combiner that combines {@code T}s into a single {@code List} containing all inputs. - * - *

              For internal use by {@link StreamingViewAsMap}, {@link StreamingViewAsMultimap}, - * {@link StreamingViewAsList}, {@link StreamingViewAsIterable}. - * They require the input {@link PCollection} fits in memory. - * For a large {@link PCollection} this is expected to crash! - * - * @param the type of elements to concatenate. - */ - private static class Concatenate extends Combine.CombineFn, List> { - @Override - public List createAccumulator() { - return new ArrayList<>(); - } - - @Override - public List addInput(List accumulator, T input) { - accumulator.add(input); - return accumulator; - } - - @Override - public List mergeAccumulators(Iterable> accumulators) { - List result = createAccumulator(); - for (List accumulator : accumulators) { - result.addAll(accumulator); - } - return result; - } - - @Override - public List extractOutput(List accumulator) { - return accumulator; - } - - @Override - public Coder> getAccumulatorCoder(CoderRegistry registry, Coder inputCoder) { - return ListCoder.of(inputCoder); - } - - @Override - public Coder> getDefaultOutputCoder(CoderRegistry registry, Coder inputCoder) { - return ListCoder.of(inputCoder); - } - } -} diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpRunner.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpRunner.java index 30b19353dc93d..ae59121cc27f6 100644 --- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpRunner.java +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpRunner.java @@ -23,6 +23,7 @@ import java.util.HashMap; import java.util.Map; +import org.apache.beam.runners.gearpump.translators.GearpumpPipelineTranslator; import org.apache.beam.runners.gearpump.translators.TranslationContext; import org.apache.beam.sdk.Pipeline; diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/CreateGearpumpPCollectionViewTranslator.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/CreateGearpumpPCollectionViewTranslator.java index d7588c21c4aa2..559cb28dda4c2 100644 --- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/CreateGearpumpPCollectionViewTranslator.java +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/CreateGearpumpPCollectionViewTranslator.java @@ -20,7 +20,6 @@ import java.util.List; -import org.apache.beam.runners.gearpump.GearpumpPipelineTranslator; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.PCollectionView; import org.apache.gearpump.streaming.dsl.javaapi.JavaStream; @@ -30,17 +29,17 @@ * transforms. */ public class CreateGearpumpPCollectionViewTranslator implements - TransformTranslator> { + TransformTranslator> { private static final long serialVersionUID = -3955521308055056034L; @Override public void translate( - GearpumpPipelineTranslator.CreateGearpumpPCollectionView transform, + CreateStreamingGearpumpView.CreateGearpumpPCollectionView transform, TranslationContext context) { JavaStream>> inputStream = context.getInputStream(context.getInput()); - PCollectionView view = (PCollectionView) context.getOutput(); - context.setOutputStream(view.getPCollection(), inputStream); + PCollectionView view = transform.getView(); + context.setOutputStream(view, inputStream); } } diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/CreateStreamingGearpumpView.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/CreateStreamingGearpumpView.java new file mode 100644 index 0000000000000..60577b338a671 --- /dev/null +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/CreateStreamingGearpumpView.java @@ -0,0 +1,157 @@ +/* + * 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.gearpump.translators; + +import com.google.common.collect.Iterables; + +import java.util.ArrayList; +import java.util.List; +import java.util.Map; + +import org.apache.beam.runners.core.construction.ReplacementOutputs; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.coders.CoderRegistry; +import org.apache.beam.sdk.coders.ListCoder; +import org.apache.beam.sdk.runners.AppliedPTransform; +import org.apache.beam.sdk.runners.PTransformOverrideFactory; +import org.apache.beam.sdk.transforms.Combine; +import org.apache.beam.sdk.transforms.PTransform; +import org.apache.beam.sdk.transforms.View.CreatePCollectionView; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.PCollectionView; +import org.apache.beam.sdk.values.PValue; +import org.apache.beam.sdk.values.TupleTag; + +/** Gearpump streaming overrides for various view (side input) transforms. */ +class CreateStreamingGearpumpView + extends PTransform, PCollection> { + private final PCollectionView view; + + public CreateStreamingGearpumpView(PCollectionView view) { + this.view = view; + } + + @Override + public PCollection expand(PCollection input) { + input + .apply(Combine.globally(new Concatenate()).withoutDefaults()) + .apply(CreateGearpumpPCollectionView.of(view)); + return input; + } + + /** + * Combiner that combines {@code T}s into a single {@code List} containing all inputs. + * + *

              For internal use by {@link CreateStreamingGearpumpView}. This combiner requires that + * the input {@link PCollection} fits in memory. For a large {@link PCollection} this is + * expected to crash! + * + * @param the type of elements to concatenate. + */ + private static class Concatenate extends Combine.CombineFn, List> { + @Override + public List createAccumulator() { + return new ArrayList(); + } + + @Override + public List addInput(List accumulator, T input) { + accumulator.add(input); + return accumulator; + } + + @Override + public List mergeAccumulators(Iterable> accumulators) { + List result = createAccumulator(); + for (List accumulator : accumulators) { + result.addAll(accumulator); + } + return result; + } + + @Override + public List extractOutput(List accumulator) { + return accumulator; + } + + @Override + public Coder> getAccumulatorCoder(CoderRegistry registry, Coder inputCoder) { + return ListCoder.of(inputCoder); + } + + @Override + public Coder> getDefaultOutputCoder(CoderRegistry registry, Coder inputCoder) { + return ListCoder.of(inputCoder); + } + } + + /** + * Creates a primitive {@link PCollectionView}. + * + *

              For internal use only by runner implementors. + * + * @param The type of the elements of the input PCollection + * @param The type associated with the {@link PCollectionView} used as a side input + */ + public static class CreateGearpumpPCollectionView + extends PTransform>, PCollection>> { + private PCollectionView view; + + private CreateGearpumpPCollectionView(PCollectionView view) { + this.view = view; + } + + public static CreateGearpumpPCollectionView of( + PCollectionView view) { + return new CreateGearpumpPCollectionView<>(view); + } + + @Override + public PCollection> expand(PCollection> input) { + return PCollection.>createPrimitiveOutputInternal( + input.getPipeline(), input.getWindowingStrategy(), input.isBounded()) + .setCoder(input.getCoder()); + } + + public PCollectionView getView() { + return view; + } + } + + public static class Factory + implements PTransformOverrideFactory< + PCollection, PCollection, CreatePCollectionView> { + public Factory() {} + + @Override + public PTransformReplacement, PCollection> getReplacementTransform( + AppliedPTransform< + PCollection, PCollection, CreatePCollectionView> + transform) { + return PTransformReplacement.of( + (PCollection) Iterables.getOnlyElement(transform.getInputs().values()), + new CreateStreamingGearpumpView(transform.getTransform().getView())); + } + + @Override + public Map mapOutputs( + Map, PValue> outputs, PCollection newOutput) { + return ReplacementOutputs.singleton(outputs, newOutput); + } + } +} diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/GearpumpPipelineTranslator.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/GearpumpPipelineTranslator.java new file mode 100644 index 0000000000000..ca98aac358937 --- /dev/null +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/GearpumpPipelineTranslator.java @@ -0,0 +1,143 @@ +/* + * 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.gearpump.translators; + +import com.google.common.collect.ImmutableList; + +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import org.apache.beam.runners.core.construction.PTransformMatchers; +import org.apache.beam.sdk.Pipeline; +import org.apache.beam.sdk.io.Read; +import org.apache.beam.sdk.runners.PTransformOverride; +import org.apache.beam.sdk.runners.TransformHierarchy; +import org.apache.beam.sdk.transforms.Flatten; +import org.apache.beam.sdk.transforms.GroupByKey; +import org.apache.beam.sdk.transforms.PTransform; +import org.apache.beam.sdk.transforms.ParDo; +import org.apache.beam.sdk.transforms.View; +import org.apache.beam.sdk.transforms.windowing.Window; +import org.apache.beam.sdk.values.PValue; + +import org.apache.gearpump.util.Graph; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * {@link GearpumpPipelineTranslator} knows how to translate {@link Pipeline} objects + * into Gearpump {@link Graph}. + */ +@SuppressWarnings({"rawtypes", "unchecked"}) +public class GearpumpPipelineTranslator extends Pipeline.PipelineVisitor.Defaults { + + private static final Logger LOG = LoggerFactory.getLogger( + GearpumpPipelineTranslator.class); + + /** + * A map from {@link PTransform} subclass to the corresponding + * {@link TransformTranslator} to use to translate that transform. + */ + private static final Map, TransformTranslator> + transformTranslators = new HashMap<>(); + + private final TranslationContext translationContext; + + static { + // register TransformTranslators + registerTransformTranslator(Read.Unbounded.class, new ReadUnboundedTranslator()); + registerTransformTranslator(Read.Bounded.class, new ReadBoundedTranslator()); + registerTransformTranslator(GroupByKey.class, new GroupByKeyTranslator()); + registerTransformTranslator(Flatten.PCollections.class, + new FlattenPCollectionsTranslator()); + registerTransformTranslator(ParDo.MultiOutput.class, new ParDoMultiOutputTranslator()); + registerTransformTranslator(Window.Assign.class, new WindowAssignTranslator()); + registerTransformTranslator(CreateStreamingGearpumpView.CreateGearpumpPCollectionView.class, + new CreateGearpumpPCollectionViewTranslator()); + } + + public GearpumpPipelineTranslator(TranslationContext translationContext) { + this.translationContext = translationContext; + } + + public void translate(Pipeline pipeline) { + List overrides = + ImmutableList.builder() + .add(PTransformOverride.of( + PTransformMatchers.classEqualTo(View.CreatePCollectionView.class), + new CreateStreamingGearpumpView.Factory())) + .build(); + + pipeline.replaceAll(overrides); + pipeline.traverseTopologically(this); + } + + @Override + public CompositeBehavior enterCompositeTransform(TransformHierarchy.Node node) { + LOG.debug("entering composite transform {}", node.getTransform()); + return CompositeBehavior.ENTER_TRANSFORM; + } + + @Override + public void leaveCompositeTransform(TransformHierarchy.Node node) { + LOG.debug("leaving composite transform {}", node.getTransform()); + } + + @Override + public void visitPrimitiveTransform(TransformHierarchy.Node node) { + LOG.debug("visiting transform {}", node.getTransform()); + PTransform transform = node.getTransform(); + TransformTranslator translator = getTransformTranslator(transform.getClass()); + if (null == translator) { + throw new IllegalStateException( + "no translator registered for " + transform); + } + translationContext.setCurrentTransform(node, getPipeline()); + translator.translate(transform, translationContext); + } + + @Override + public void visitValue(PValue value, TransformHierarchy.Node producer) { + LOG.debug("visiting value {}", value); + } + + /** + * Records that instances of the specified PTransform class + * should be translated by default by the corresponding + * {@link TransformTranslator}. + */ + private static void registerTransformTranslator( + Class transformClass, + TransformTranslator transformTranslator) { + if (transformTranslators.put(transformClass, transformTranslator) != null) { + throw new IllegalArgumentException( + "defining multiple translators for " + transformClass); + } + } + + /** + * Returns the {@link TransformTranslator} to use for instances of the + * specified PTransform class, or null if none registered. + */ + private + TransformTranslator getTransformTranslator(Class transformClass) { + return transformTranslators.get(transformClass); + } +} diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/TranslationContext.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/TranslationContext.java index 64a1e0dd8196d..42b7a536ab8c2 100644 --- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/TranslationContext.java +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/TranslationContext.java @@ -25,6 +25,7 @@ import java.util.HashMap; import java.util.Map; +import org.apache.beam.runners.core.construction.TransformInputs; import org.apache.beam.runners.gearpump.GearpumpPipelineOptions; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.runners.AppliedPTransform; @@ -78,7 +79,7 @@ public Map, PValue> getInputs() { } public PValue getInput() { - return Iterables.getOnlyElement(getInputs().values()); + return Iterables.getOnlyElement(TransformInputs.nonAdditionalInputs(getCurrentTransform())); } public Map, PValue> getOutputs() { diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/functions/DoFnFunction.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/functions/DoFnFunction.java index e2777df13f211..b20896a858ede 100644 --- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/functions/DoFnFunction.java +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/functions/DoFnFunction.java @@ -135,7 +135,6 @@ public Iterator flatMap(List input } } - for (PCollectionView sideInput: sideInputs) { for (WindowedValue value : pushedBackValues) { for (BoundedWindow win: value.getWindows()) { diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/io/GearpumpSource.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/io/GearpumpSource.java index 6637a9bdbf08f..558eb0dadeb41 100644 --- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/io/GearpumpSource.java +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/io/GearpumpSource.java @@ -80,7 +80,7 @@ public Message read() { org.joda.time.Instant timestamp = reader.getCurrentTimestamp(); message = new DefaultMessage( WindowedValue.timestampedValueInGlobalWindow(data, timestamp), - TranslatorUtils.jodaTimeToJava8Time(timestamp)); + timestamp.getMillis()); } available = reader.advance(); } catch (Exception e) { diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/utils/TranslatorUtils.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/utils/TranslatorUtils.java index 83fc6e6953120..b1cd61ca5b2eb 100644 --- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/utils/TranslatorUtils.java +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/utils/TranslatorUtils.java @@ -52,6 +52,7 @@ public static org.joda.time.Instant java8TimeToJodaTime(Instant time) { } public static Window boundedWindowToGearpumpWindow(BoundedWindow window) { + // Gearpump window upper bound is exclusive Instant end = TranslatorUtils.jodaTimeToJava8Time(window.maxTimestamp().plus(1L)); if (window instanceof IntervalWindow) { IntervalWindow intervalWindow = (IntervalWindow) window; diff --git a/runners/gearpump/src/test/java/org/apache/beam/runners/gearpump/translators/CreateGearpumpPCollectionViewTranslatorTest.java b/runners/gearpump/src/test/java/org/apache/beam/runners/gearpump/translators/CreateGearpumpPCollectionViewTranslatorTest.java index b23b0c603ece8..511eed182c81e 100644 --- a/runners/gearpump/src/test/java/org/apache/beam/runners/gearpump/translators/CreateGearpumpPCollectionViewTranslatorTest.java +++ b/runners/gearpump/src/test/java/org/apache/beam/runners/gearpump/translators/CreateGearpumpPCollectionViewTranslatorTest.java @@ -23,7 +23,6 @@ import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; -import org.apache.beam.runners.gearpump.GearpumpPipelineTranslator; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.PValue; import org.apache.gearpump.streaming.dsl.javaapi.JavaStream; @@ -38,8 +37,8 @@ public void testTranslate() { CreateGearpumpPCollectionViewTranslator translator = new CreateGearpumpPCollectionViewTranslator(); - GearpumpPipelineTranslator.CreateGearpumpPCollectionView pCollectionView = - mock(GearpumpPipelineTranslator.CreateGearpumpPCollectionView.class); + CreateStreamingGearpumpView.CreateGearpumpPCollectionView pCollectionView = + mock(CreateStreamingGearpumpView.CreateGearpumpPCollectionView.class); JavaStream javaStream = mock(JavaStream.class); TranslationContext translationContext = mock(TranslationContext.class); @@ -49,7 +48,7 @@ public void testTranslate() { when(translationContext.getInputStream(mockInput)).thenReturn(javaStream); PCollectionView view = mock(PCollectionView.class); - when(translationContext.getOutput()).thenReturn(view); + when(pCollectionView.getView()).thenReturn(view); translator.translate(pCollectionView, translationContext); verify(translationContext, times(1)).setOutputStream(view, javaStream); diff --git a/runners/gearpump/src/test/java/org/apache/beam/runners/gearpump/translators/io/GearpumpSourceTest.java b/runners/gearpump/src/test/java/org/apache/beam/runners/gearpump/translators/io/GearpumpSourceTest.java index 44907376c3e7d..cc4284f54eee6 100644 --- a/runners/gearpump/src/test/java/org/apache/beam/runners/gearpump/translators/io/GearpumpSourceTest.java +++ b/runners/gearpump/src/test/java/org/apache/beam/runners/gearpump/translators/io/GearpumpSourceTest.java @@ -30,6 +30,7 @@ import org.apache.beam.sdk.io.Source; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.options.PipelineOptionsFactory; +import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.TimestampedValue; import org.apache.gearpump.DefaultMessage; @@ -42,10 +43,11 @@ public class GearpumpSourceTest { private static final List> TEST_VALUES = Lists.newArrayList( - TimestampedValue.of("a", new org.joda.time.Instant(Long.MIN_VALUE)), + TimestampedValue.of("a", BoundedWindow.TIMESTAMP_MIN_VALUE), TimestampedValue.of("b", new org.joda.time.Instant(0)), TimestampedValue.of("c", new org.joda.time.Instant(53)), - TimestampedValue.of("d", new org.joda.time.Instant(Long.MAX_VALUE - 1))); + TimestampedValue.of("d", BoundedWindow.TIMESTAMP_MAX_VALUE) + ); private static class SourceForTest extends GearpumpSource { private ValuesSource valuesSource; @@ -72,10 +74,16 @@ public void testGearpumpSource() { new SourceForTest<>(options, valuesSource); sourceForTest.open(null, Instant.EPOCH); - for (TimestampedValue value : TEST_VALUES) { + for (int i = 0; i < TEST_VALUES.size(); i++) { + TimestampedValue value = TEST_VALUES.get(i); + // Check the watermark first since the Source will advance when it's opened - Instant expectedWaterMark = TranslatorUtils.jodaTimeToJava8Time(value.getTimestamp()); - Assert.assertEquals(expectedWaterMark, sourceForTest.getWatermark()); + if (i < TEST_VALUES.size() - 1) { + Instant expectedWaterMark = TranslatorUtils.jodaTimeToJava8Time(value.getTimestamp()); + Assert.assertEquals(expectedWaterMark, sourceForTest.getWatermark()); + } else { + Assert.assertEquals(Watermark.MAX(), sourceForTest.getWatermark()); + } Message expectedMsg = new DefaultMessage( diff --git a/runners/gearpump/src/test/java/org/apache/beam/runners/gearpump/translators/utils/TranslatorUtilsTest.java b/runners/gearpump/src/test/java/org/apache/beam/runners/gearpump/translators/utils/TranslatorUtilsTest.java index 524887dd2057c..6ebe59bfc0828 100644 --- a/runners/gearpump/src/test/java/org/apache/beam/runners/gearpump/translators/utils/TranslatorUtilsTest.java +++ b/runners/gearpump/src/test/java/org/apache/beam/runners/gearpump/translators/utils/TranslatorUtilsTest.java @@ -23,7 +23,6 @@ import com.google.common.collect.Lists; -import java.time.Duration; import java.time.Instant; import java.util.List; @@ -68,7 +67,7 @@ public void testBoundedWindowToGearpumpWindow() { Instant.ofEpochMilli(Long.MAX_VALUE)))); BoundedWindow globalWindow = GlobalWindow.INSTANCE; assertThat(TranslatorUtils.boundedWindowToGearpumpWindow(globalWindow), - equalTo(Window.apply(Instant.ofEpochMilli(Long.MIN_VALUE / 1000), - Instant.ofEpochMilli(Long.MAX_VALUE / 1000).minus(Duration.ofDays(1)).plusMillis(1)))); + equalTo(Window.apply(Instant.ofEpochMilli(BoundedWindow.TIMESTAMP_MIN_VALUE.getMillis()), + Instant.ofEpochMilli(globalWindow.maxTimestamp().getMillis() + 1)))); } } From 11caa9789fa843991231cdba73fbd738cb6d8fe2 Mon Sep 17 00:00:00 2001 From: manuzhang Date: Mon, 19 Jun 2017 19:12:45 +0800 Subject: [PATCH 160/346] Fix kryo exception --- .../java/org/apache/beam/runners/gearpump/GearpumpRunner.java | 2 ++ .../runners/gearpump/translators/functions/DoFnFunction.java | 1 - 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpRunner.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpRunner.java index ae59121cc27f6..5febf3ccf3c51 100644 --- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpRunner.java +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/GearpumpRunner.java @@ -106,6 +106,8 @@ private Config registerSerializers(Config config, Map userSerial serializers.put("org.apache.beam.sdk.values.KV", ""); serializers.put("org.apache.beam.sdk.transforms.windowing.IntervalWindow", ""); serializers.put("org.apache.beam.sdk.values.TimestampedValue", ""); + serializers.put( + "org.apache.beam.runners.gearpump.translators.utils.TranslatorUtils$RawUnionValue", ""); if (userSerializers != null && !userSerializers.isEmpty()) { serializers.putAll(userSerializers); diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/functions/DoFnFunction.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/functions/DoFnFunction.java index b20896a858ede..fde265a83d1cf 100644 --- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/functions/DoFnFunction.java +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/functions/DoFnFunction.java @@ -144,7 +144,6 @@ public Iterator flatMap(List input Object emptyValue = WindowedValue.of( Lists.newArrayList(), value.getTimestamp(), sideInputWindow, value.getPane()); sideInputReader.addSideInputValue(sideInput, (WindowedValue>) emptyValue); - System.out.println(sideInput + " in " + sideInputWindow.toString() + " not ready"); } } } From b21fa04f28f5e7be87b41d5478c4c6decd87e433 Mon Sep 17 00:00:00 2001 From: manuzhang Date: Fri, 23 Jun 2017 10:12:19 +0800 Subject: [PATCH 161/346] Remove unused codes --- .../beam/runners/gearpump/translators/io/GearpumpSource.java | 1 - 1 file changed, 1 deletion(-) diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/io/GearpumpSource.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/io/GearpumpSource.java index 558eb0dadeb41..daa8c81835474 100644 --- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/io/GearpumpSource.java +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/io/GearpumpSource.java @@ -46,7 +46,6 @@ public abstract class GearpumpSource implements DataSource { private Source.Reader reader; private boolean available = false; - private long count = 0L; GearpumpSource(PipelineOptions options) { try { From 22068274428666da93c793a170810ddb42755704 Mon Sep 17 00:00:00 2001 From: manuzhang Date: Fri, 7 Jul 2017 16:41:13 +0800 Subject: [PATCH 162/346] Upgrade to gearpump 0.8.4 --- runners/gearpump/pom.xml | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/runners/gearpump/pom.xml b/runners/gearpump/pom.xml index 3c98d5e5fa235..54c8d5c012d25 100644 --- a/runners/gearpump/pom.xml +++ b/runners/gearpump/pom.xml @@ -43,7 +43,7 @@ UTF-8 UTF-8 - 0.8.4-SNAPSHOT + 0.8.4 @@ -58,7 +58,7 @@ maven-surefire-plugin - runnable-on-service-tests + validates-runner-tests integration-test test From 725f547f5e487dd3e84d5d0f95c0fa3efa853279 Mon Sep 17 00:00:00 2001 From: manuzhang Date: Sat, 8 Jul 2017 00:13:19 +0800 Subject: [PATCH 163/346] Fix ParDoTest#testPipelineOptionsParameter --- .../translators/io/GearpumpSource.java | 12 ++---------- .../translators/utils/DoFnRunnerFactory.java | 5 +++-- .../translators/utils/TranslatorUtils.java | 19 +++++++++++++++++++ 3 files changed, 24 insertions(+), 12 deletions(-) diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/io/GearpumpSource.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/io/GearpumpSource.java index daa8c81835474..2f531399d0aa9 100644 --- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/io/GearpumpSource.java +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/io/GearpumpSource.java @@ -18,9 +18,6 @@ package org.apache.beam.runners.gearpump.translators.io; -import com.fasterxml.jackson.core.JsonProcessingException; -import com.fasterxml.jackson.databind.ObjectMapper; - import java.io.IOException; import java.time.Instant; @@ -48,11 +45,7 @@ public abstract class GearpumpSource implements DataSource { private boolean available = false; GearpumpSource(PipelineOptions options) { - try { - this.serializedOptions = new ObjectMapper().writeValueAsBytes(options); - } catch (JsonProcessingException e) { - throw new RuntimeException(e); - } + this.serializedOptions = TranslatorUtils.serializePipelineOptions(options); } protected abstract Source.Reader createReader(PipelineOptions options) throws IOException; @@ -60,8 +53,7 @@ public abstract class GearpumpSource implements DataSource { @Override public void open(TaskContext context, Instant startTime) { try { - PipelineOptions options = new ObjectMapper() - .readValue(serializedOptions, PipelineOptions.class); + PipelineOptions options = TranslatorUtils.deserializePipelineOptions(serializedOptions); this.reader = createReader(options); this.available = reader.start(); } catch (Exception e) { diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/utils/DoFnRunnerFactory.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/utils/DoFnRunnerFactory.java index 35cf2b5b33ca3..375b696a35417 100644 --- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/utils/DoFnRunnerFactory.java +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/utils/DoFnRunnerFactory.java @@ -43,7 +43,7 @@ public class DoFnRunnerFactory implements Serializable { private static final long serialVersionUID = -4109539010014189725L; private final DoFn fn; - private final transient PipelineOptions options; + private final byte[] serializedOptions; private final Collection> sideInputs; private final DoFnRunners.OutputManager outputManager; private final TupleTag mainOutputTag; @@ -61,7 +61,7 @@ public DoFnRunnerFactory( StepContext stepContext, WindowingStrategy windowingStrategy) { this.fn = doFn; - this.options = pipelineOptions; + this.serializedOptions = TranslatorUtils.serializePipelineOptions(pipelineOptions); this.sideInputs = sideInputs; this.outputManager = outputManager; this.mainOutputTag = mainOutputTag; @@ -72,6 +72,7 @@ public DoFnRunnerFactory( public PushbackSideInputDoFnRunner createRunner( ReadyCheckingSideInputReader sideInputReader) { + PipelineOptions options = TranslatorUtils.deserializePipelineOptions(serializedOptions); DoFnRunner underlying = DoFnRunners.simpleRunner( options, fn, sideInputReader, outputManager, mainOutputTag, sideOutputTags, stepContext, windowingStrategy); diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/utils/TranslatorUtils.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/utils/TranslatorUtils.java index b1cd61ca5b2eb..c14298f290bcc 100644 --- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/utils/TranslatorUtils.java +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/utils/TranslatorUtils.java @@ -18,8 +18,11 @@ package org.apache.beam.runners.gearpump.translators.utils; +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.Lists; +import java.io.IOException; import java.time.Instant; import java.util.Collection; import java.util.HashMap; @@ -27,6 +30,7 @@ import java.util.Map; import org.apache.beam.runners.gearpump.translators.TranslationContext; +import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.GlobalWindow; import org.apache.beam.sdk.transforms.windowing.IntervalWindow; @@ -141,6 +145,21 @@ public RawUnionValue map(WindowedValue windowedValue) { } } + public static byte[] serializePipelineOptions(PipelineOptions options) { + try { + return new ObjectMapper().writeValueAsBytes(options); + } catch (JsonProcessingException e) { + throw new RuntimeException(e); + } + } + + public static PipelineOptions deserializePipelineOptions(byte[] serializedOptions) { + try { + return new ObjectMapper().readValue(serializedOptions, PipelineOptions.class); + } catch (IOException e) { + throw new RuntimeException(e); + } + } /** * This is copied from org.apache.beam.sdk.transforms.join.RawUnionValue. From 4192ac6c72765e8c55480eec66c499ef0798ecf3 Mon Sep 17 00:00:00 2001 From: Stephen Sisk Date: Wed, 14 Jun 2017 09:57:35 -0700 Subject: [PATCH 164/346] Add maven support for invoking perfkit benchmarker to run IO ITs --- .../kubernetes/postgres/pkb-config-local.yml | 34 +++++ .../kubernetes/postgres/pkb-config.yml | 32 ++++ runners/google-cloud-dataflow-java/pom.xml | 23 +++ sdks/java/io/google-cloud-platform/pom.xml | 91 ++++++++++++ sdks/java/io/jdbc/pom.xml | 139 ++++++++++++++++++ sdks/java/io/pom.xml | 36 +++++ 6 files changed, 355 insertions(+) create mode 100644 .test-infra/kubernetes/postgres/pkb-config-local.yml create mode 100644 .test-infra/kubernetes/postgres/pkb-config.yml diff --git a/.test-infra/kubernetes/postgres/pkb-config-local.yml b/.test-infra/kubernetes/postgres/pkb-config-local.yml new file mode 100644 index 0000000000000..1bac0c4a2ae55 --- /dev/null +++ b/.test-infra/kubernetes/postgres/pkb-config-local.yml @@ -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. +# + +# This file is a pkb benchmark configuration file, used when running the IO ITs +# that use this data store. It allows users to run tests when they are on a +# separate network from the kubernetes cluster by reading the postgres IP +# address from the LoadBalancer service. +# +# This file defines pipeline options to pass to beam, as well as how to derive +# the values for those pipeline options from kubernetes (where appropriate.) + +static_pipeline_options: + - postgresUsername: postgres + - postgresPassword: uuinkks + - postgresDatabaseName: postgres + - postgresSsl: false +dynamic_pipeline_options: + - name: postgresServerName + type: LoadBalancerIp + serviceName: postgres-for-dev diff --git a/.test-infra/kubernetes/postgres/pkb-config.yml b/.test-infra/kubernetes/postgres/pkb-config.yml new file mode 100644 index 0000000000000..b943b17264246 --- /dev/null +++ b/.test-infra/kubernetes/postgres/pkb-config.yml @@ -0,0 +1,32 @@ +# +# 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. +# + +# This file is a pkb benchmark configuration file, used when running the IO ITs +# that use this data store. +# +# This file defines pipeline options to pass to beam, as well as how to derive +# the values for those pipeline options from kubernetes (where appropriate.) + +static_pipeline_options: + - postgresUsername: postgres + - postgresPassword: uuinkks + - postgresDatabaseName: postgres + - postgresSsl: false +dynamic_pipeline_options: + - name: postgresServerName + type: NodePortIp + podLabel: name=postgres diff --git a/runners/google-cloud-dataflow-java/pom.xml b/runners/google-cloud-dataflow-java/pom.xml index c8d63ac56a272..1181b79876382 100644 --- a/runners/google-cloud-dataflow-java/pom.xml +++ b/runners/google-cloud-dataflow-java/pom.xml @@ -122,6 +122,29 @@ + + + + io-it + + io-it + + + + + org.apache.maven.plugins + maven-surefire-plugin + ${surefire-plugin.version} + + true + + + + + + false + + diff --git a/sdks/java/io/google-cloud-platform/pom.xml b/sdks/java/io/google-cloud-platform/pom.xml index a1495f2df8ea9..adb7e32fb8d2d 100644 --- a/sdks/java/io/google-cloud-platform/pom.xml +++ b/sdks/java/io/google-cloud-platform/pom.xml @@ -60,6 +60,97 @@ + + + + io-it-suite + + + + ${project.parent.parent.parent.parent.basedir} + + + io-it-suite + + + + org.codehaus.gmaven + groovy-maven-plugin + ${groovy-maven-plugin.version} + + + find-supported-python-for-compile + initialize + + execute + + + ${beamRootProjectDir}/sdks/python/findSupportedPython.groovy + + + + + + org.codehaus.mojo + exec-maven-plugin + ${maven-exec-plugin.version} + + + verify + + exec + + + + + ${python.interpreter.bin} + + ${pkbLocation} + -benchmarks=beam_integration_benchmark + -beam_it_profile=io-it + -beam_location=${beamRootProjectDir} + -beam_prebuilt=true + -beam_sdk=java + -kubeconfig=${kubeconfig} + -kubectl=${kubectl} + + ${pkbBeamRunnerProfile} + ${pkbBeamRunnerOption} + + -beam_it_module=runners/google-cloud-dataflow-java + + -beam_it_class=${gcpIoItClass} + + -beam_it_options=${integrationTestPipelineOptions} + + + + + + org.apache.maven.plugins + maven-surefire-plugin + ${surefire-plugin.version} + + true + + + + + + + org.apache.beam diff --git a/sdks/java/io/jdbc/pom.xml b/sdks/java/io/jdbc/pom.xml index e5f4d7ed0f03f..3e8ba5756f3cc 100644 --- a/sdks/java/io/jdbc/pom.xml +++ b/sdks/java/io/jdbc/pom.xml @@ -65,6 +65,145 @@ + + + io-it-suite + + io-it-suite + + + + ${project.parent.parent.parent.parent.basedir} + + + + + org.codehaus.gmaven + groovy-maven-plugin + ${groovy-maven-plugin.version} + + + find-supported-python-for-compile + initialize + + execute + + + ${beamRootProjectDir}/sdks/python/findSupportedPython.groovy + + + + + + org.codehaus.mojo + exec-maven-plugin + ${maven-exec-plugin.version} + + + verify + + exec + + + + + ${python.interpreter.bin} + + ${pkbLocation} + -benchmarks=beam_integration_benchmark + -beam_it_profile=io-it + -beam_location=${beamRootProjectDir} + -beam_prebuilt=true + -beam_sdk=java + -kubeconfig=${kubeconfig} + -kubectl=${kubectl} + + ${pkbBeamRunnerProfile} + ${pkbBeamRunnerOption} + + -beam_options_config_file=${beamRootProjectDir}/.test-infra/kubernetes/postgres/pkb-config.yml + -beam_kubernetes_scripts=${beamRootProjectDir}/.test-infra/kubernetes/postgres/postgres.yml + -beam_it_module=sdks/java/io/jdbc + -beam_it_class=org.apache.beam.sdk.io.jdbc.JdbcIOIT + + -beam_it_options=${integrationTestPipelineOptions} + + + + + + org.apache.maven.plugins + maven-surefire-plugin + ${surefire-plugin.version} + + true + + + + + + + + + io-it-suite-local + io-it-suite-local + + + + org.codehaus.mojo + exec-maven-plugin + ${maven-exec-plugin.version} + + + verify + + exec + + + + + ${python.interpreter.bin} + + ${pkbLocation} + -benchmarks=beam_integration_benchmark + -beam_it_profile=io-it + -beam_location=${beamRootProjectDir} + -beam_prebuilt=true + -beam_sdk=java + -kubeconfig=${kubeconfig} + -kubectl=${kubectl} + + ${pkbBeamRunnerProfile} + ${pkbBeamRunnerOption} + + -beam_options_config_file=${beamRootProjectDir}/.test-infra/kubernetes/postgres/pkb-config-local.yml + -beam_kubernetes_scripts=${beamRootProjectDir}/.test-infra/kubernetes/postgres/postgres.yml,${beamRootProjectDir}/.test-infra/kubernetes/postgres/postgres-service-for-local-dev.yml + -beam_it_module=sdks/java/io/jdbc + -beam_it_class=org.apache.beam.sdk.io.jdbc.JdbcIOIT + + -beam_it_options=${integrationTestPipelineOptions} + + + + + + + dataflow-runner diff --git a/sdks/java/io/pom.xml b/sdks/java/io/pom.xml index b7909fa7f6c7e..e9aa65f0f6466 100644 --- a/sdks/java/io/pom.xml +++ b/sdks/java/io/pom.xml @@ -32,6 +32,17 @@ Beam SDK Java IO provides different connectivity components (sources and sinks) to consume and produce data from systems. + + + kubectl + + ${user.home}/.kube/config + + + + + + amqp cassandra @@ -81,11 +92,36 @@ + + org.apache.maven.plugins + maven-surefire-plugin + ${surefire-plugin.version} + + true + + false + + + + forceDirectRunner + + + forceDirectRunner + + + + + -beam_runner_profile= + -beam_runner_option= + + From 2f9cbec803791fd4e17fb28ec5590649aabb9622 Mon Sep 17 00:00:00 2001 From: Stephen Sisk Date: Thu, 20 Jul 2017 11:38:40 -0700 Subject: [PATCH 165/346] io-it-suite-local independent of io-it-suite, k8s properties -> root pom --- pom.xml | 5 +++++ sdks/java/io/jdbc/pom.xml | 33 +++++++++++++++++++++++++++++++++ sdks/java/io/pom.xml | 4 ---- 3 files changed, 38 insertions(+), 4 deletions(-) diff --git a/pom.xml b/pom.xml index f2d0ddea730b1..e0ec1362a6782 100644 --- a/pom.xml +++ b/pom.xml @@ -168,6 +168,11 @@ -Xpkginfo:always nothing 0.20.0 + + + kubectl + + ${user.home}/.kube/config pom diff --git a/sdks/java/io/jdbc/pom.xml b/sdks/java/io/jdbc/pom.xml index 3e8ba5756f3cc..357ddc08b7d23 100644 --- a/sdks/java/io/jdbc/pom.xml +++ b/sdks/java/io/jdbc/pom.xml @@ -105,6 +105,7 @@ + org.codehaus.mojo exec-maven-plugin @@ -162,8 +163,31 @@ io-it-suite-local io-it-suite-local + + + ${project.parent.parent.parent.parent.basedir} + + + org.codehaus.gmaven + groovy-maven-plugin + ${groovy-maven-plugin.version} + + + find-supported-python-for-compile + initialize + + execute + + + ${beamRootProjectDir}/sdks/python/findSupportedPython.groovy + + + + + org.codehaus.mojo exec-maven-plugin @@ -200,6 +224,15 @@ + + + org.apache.maven.plugins + maven-surefire-plugin + ${surefire-plugin.version} + + true + + diff --git a/sdks/java/io/pom.xml b/sdks/java/io/pom.xml index e9aa65f0f6466..4e02aa8286f1f 100644 --- a/sdks/java/io/pom.xml +++ b/sdks/java/io/pom.xml @@ -33,10 +33,6 @@ (sources and sinks) to consume and produce data from systems. - - kubectl - - ${user.home}/.kube/config From c52a908cba7765e120a94909ab02c548d1a124ad Mon Sep 17 00:00:00 2001 From: Eugene Kirpichov Date: Tue, 18 Jul 2017 13:40:52 -0700 Subject: [PATCH 166/346] Minor changes to AvroSource in preparation for refactoring --- .../org/apache/beam/sdk/io/AvroSource.java | 171 +++++++----------- .../apache/beam/sdk/io/AvroSourceTest.java | 26 +-- 2 files changed, 70 insertions(+), 127 deletions(-) diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/AvroSource.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/AvroSource.java index 575218bf92ca6..0634774fe935e 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/AvroSource.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/AvroSource.java @@ -37,6 +37,7 @@ import java.util.WeakHashMap; import java.util.zip.Inflater; import java.util.zip.InflaterInputStream; +import javax.annotation.Nullable; import javax.annotation.concurrent.GuardedBy; import org.apache.avro.Schema; import org.apache.avro.file.CodecFactory; @@ -127,15 +128,16 @@ public class AvroSource extends BlockBasedSource { // The default sync interval is 64k. private static final long DEFAULT_MIN_BUNDLE_SIZE = 2 * DataFileConstants.DEFAULT_SYNC_INTERVAL; - // The JSON schema used to encode records. - private final String readSchemaString; + // The type of the records contained in the file. + private final Class type; + + // The JSON schema used to decode records. + @Nullable + private final String readerSchemaString; // The JSON schema that was used to write the source Avro file (may differ from the schema we will // use to read from it). - private final String fileSchemaString; - - // The type of the records contained in the file. - private final Class type; + private final String writerSchemaString; // The following metadata fields are not user-configurable. They are extracted from the object // container file header upon subsource creation. @@ -147,87 +149,75 @@ public class AvroSource extends BlockBasedSource { // The object container file's 16-byte sync marker. private final byte[] syncMarker; - // Default output coder, lazily initialized. - private transient AvroCoder coder = null; - - // Schema of the file, lazily initialized. - private transient Schema fileSchema; - - // Schema used to encode records, lazily initialized. - private transient Schema readSchema; - /** - * Creates an {@link AvroSource} that reads from the given file name or pattern ("glob"). The - * returned source can be further configured by calling {@link #withSchema} to return a type other - * than {@link GenericRecord}. + * Reads from the given file name or pattern ("glob"). The returned source can be further + * configured by calling {@link #withSchema} to return a type other than {@link GenericRecord}. */ public static AvroSource from(String fileNameOrPattern) { return new AvroSource<>( fileNameOrPattern, DEFAULT_MIN_BUNDLE_SIZE, null, GenericRecord.class, null, null); } - /** - * Returns an {@link AvroSource} that's like this one but reads files containing records that - * conform to the given schema. - * - *

              Does not modify this object. - */ + /** Reads files containing records that conform to the given schema. */ public AvroSource withSchema(String schema) { return new AvroSource<>( getFileOrPatternSpec(), getMinBundleSize(), schema, GenericRecord.class, codec, syncMarker); } - /** - * Returns an {@link AvroSource} that's like this one but reads files containing records that - * conform to the given schema. - * - *

              Does not modify this object. - */ + /** Like {@link #withSchema(String)}. */ public AvroSource withSchema(Schema schema) { return new AvroSource<>(getFileOrPatternSpec(), getMinBundleSize(), schema.toString(), GenericRecord.class, codec, syncMarker); } - /** - * Returns an {@link AvroSource} that's like this one but reads files containing records of the - * type of the given class. - * - *

              Does not modify this object. - */ + /** Reads files containing records of the given class. */ public AvroSource withSchema(Class clazz) { - return new AvroSource(getFileOrPatternSpec(), getMinBundleSize(), + return new AvroSource<>(getFileOrPatternSpec(), getMinBundleSize(), ReflectData.get().getSchema(clazz).toString(), clazz, codec, syncMarker); } /** - * Returns an {@link AvroSource} that's like this one but uses the supplied minimum bundle size. - * Refer to {@link OffsetBasedSource} for a description of {@code minBundleSize} and its use. - * - *

              Does not modify this object. + * Sets the minimum bundle size. Refer to {@link OffsetBasedSource} for a description of {@code + * minBundleSize} and its use. */ public AvroSource withMinBundleSize(long minBundleSize) { return new AvroSource<>( - getFileOrPatternSpec(), minBundleSize, readSchemaString, type, codec, syncMarker); + getFileOrPatternSpec(), minBundleSize, readerSchemaString, type, codec, syncMarker); } - private AvroSource(String fileNameOrPattern, long minBundleSize, String schema, Class type, - String codec, byte[] syncMarker) { + /** Constructor for FILEPATTERN mode. */ + private AvroSource( + String fileNameOrPattern, + long minBundleSize, + String readerSchemaString, + Class type, + String codec, + byte[] syncMarker) { super(fileNameOrPattern, minBundleSize); - this.readSchemaString = internSchemaString(schema); + this.readerSchemaString = internSchemaString(readerSchemaString); this.codec = codec; this.syncMarker = syncMarker; this.type = type; - this.fileSchemaString = null; + this.writerSchemaString = null; } - private AvroSource(Metadata metadata, long minBundleSize, long startOffset, long endOffset, - String schema, Class type, String codec, byte[] syncMarker, String fileSchema) { + /** Constructor for SINGLE_FILE_OR_SUBRANGE mode. */ + private AvroSource( + Metadata metadata, + long minBundleSize, + long startOffset, + long endOffset, + String readerSchemaString, + Class type, + String codec, + byte[] syncMarker, + String writerSchemaString) { super(metadata, minBundleSize, startOffset, endOffset); - this.readSchemaString = internSchemaString(schema); + this.readerSchemaString = internSchemaString(readerSchemaString); this.codec = codec; this.syncMarker = syncMarker; this.type = type; - this.fileSchemaString = internSchemaString(fileSchema); + this.writerSchemaString = internSchemaString(writerSchemaString); } @Override @@ -241,13 +231,12 @@ public void validate() { public BlockBasedSource createForSubrangeOfFile(Metadata fileMetadata, long start, long end) { byte[] syncMarker = this.syncMarker; String codec = this.codec; - String readSchemaString = this.readSchemaString; - String fileSchemaString = this.fileSchemaString; + String writerSchemaString = this.writerSchemaString; // codec and syncMarker are initially null when the source is created, as they differ // across input files and must be read from the file. Here, when we are creating a source // for a subrange of a file, we can initialize these values. When the resulting AvroSource // is further split, they do not need to be read again. - if (codec == null || syncMarker == null || fileSchemaString == null) { + if (codec == null || syncMarker == null || writerSchemaString == null) { AvroMetadata metadata; try { metadata = readMetadataFromFile(fileMetadata.resourceId()); @@ -256,20 +245,23 @@ public BlockBasedSource createForSubrangeOfFile(Metadata fileMetadata, long s } codec = metadata.getCodec(); syncMarker = metadata.getSyncMarker(); - fileSchemaString = metadata.getSchemaString(); - // If the source was created with a null schema, use the schema that we read from the file's - // metadata. - if (readSchemaString == null) { - readSchemaString = metadata.getSchemaString(); - } + writerSchemaString = metadata.getSchemaString(); } - // Note that if the fileSchemaString is equivalent to the readSchemaString, "intern"ing + // Note that if the writerSchemaString is equivalent to the readerSchemaString, "intern"ing // the string will occur within the constructor and return the same reference as the - // readSchemaString. This allows for Java to have an efficient serialization since it + // readerSchemaString. This allows for Java to have an efficient serialization since it // will only encode the schema once while just storing pointers to the encoded version // within this source. - return new AvroSource<>(fileMetadata, getMinBundleSize(), start, end, readSchemaString, type, - codec, syncMarker, fileSchemaString); + return new AvroSource<>( + fileMetadata, + getMinBundleSize(), + start, + end, + readerSchemaString, + type, + codec, + syncMarker, + writerSchemaString); } @Override @@ -279,40 +271,13 @@ protected BlockBasedReader createSingleFileReader(PipelineOptions options) { @Override public AvroCoder getDefaultOutputCoder() { - if (coder == null) { - coder = AvroCoder.of(type, internOrParseSchemaString(readSchemaString)); - } - return coder; - } - - public String getSchema() { - return readSchemaString; - } - - @VisibleForTesting - Schema getReadSchema() { - if (readSchemaString == null) { - return null; - } - - // If the schema has not been parsed, parse it. - if (readSchema == null) { - readSchema = internOrParseSchemaString(readSchemaString); - } - return readSchema; + return AvroCoder.of(type, internOrParseSchemaString(readerSchemaString)); } @VisibleForTesting - Schema getFileSchema() { - if (fileSchemaString == null) { - return null; - } - - // If the schema has not been parsed, parse it. - if (fileSchema == null) { - fileSchema = internOrParseSchemaString(fileSchemaString); - } - return fileSchema; + @Nullable + String getReaderSchemaString() { + return readerSchemaString; } private byte[] getSyncMarker() { @@ -427,14 +392,14 @@ static AvroMetadata readMetadataFromFile(ResourceId fileResource) throws IOExcep } private DatumReader createDatumReader() { - Schema readSchema = getReadSchema(); - Schema fileSchema = getFileSchema(); - checkNotNull(readSchema, "No read schema has been initialized for source %s", this); - checkNotNull(fileSchema, "No file schema has been initialized for source %s", this); + checkNotNull(writerSchemaString, "No writer schema has been initialized for source %s", this); + Schema writerSchema = internOrParseSchemaString(writerSchemaString); + Schema readerSchema = + (readerSchemaString == null) ? writerSchema : internOrParseSchemaString(readerSchemaString); if (type == GenericRecord.class) { - return new GenericDatumReader<>(fileSchema, readSchema); + return new GenericDatumReader<>(writerSchema, readerSchema); } else { - return new ReflectDatumReader<>(fileSchema, readSchema); + return new ReflectDatumReader<>(writerSchema, readerSchema); } } @@ -477,16 +442,16 @@ private Object readResolve() throws ObjectStreamException { getMinBundleSize(), getStartOffset(), getEndOffset(), - readSchemaString, + readerSchemaString, type, codec, syncMarker, - fileSchemaString); + writerSchemaString); case FILEPATTERN: return new AvroSource<>( getFileOrPatternSpec(), getMinBundleSize(), - readSchemaString, + readerSchemaString, type, codec, syncMarker); diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/AvroSourceTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/AvroSourceTest.java index d6facba4f8619..0fc2b3e6b121c 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/AvroSourceTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/AvroSourceTest.java @@ -445,33 +445,11 @@ public void testSchemaStringIsInterned() throws Exception { AvroSource sourceA = AvroSource.from(filename).withSchema(schemaA); AvroSource sourceB = AvroSource.from(filename).withSchema(schemaB); - assertSame(sourceA.getSchema(), sourceB.getSchema()); + assertSame(sourceA.getReaderSchemaString(), sourceB.getReaderSchemaString()); // Ensure that deserialization still goes through interning AvroSource sourceC = SerializableUtils.clone(sourceB); - assertSame(sourceA.getSchema(), sourceC.getSchema()); - } - - @Test - public void testSchemaIsInterned() throws Exception { - List birds = createRandomRecords(100); - String filename = generateTestFile("tmp.avro", birds, SyncBehavior.SYNC_DEFAULT, 0, - AvroCoder.of(Bird.class), DataFileConstants.NULL_CODEC); - Metadata fileMetadata = FileSystems.matchSingleFileSpec(filename); - String schemaA = AvroSource.readMetadataFromFile(fileMetadata.resourceId()).getSchemaString(); - String schemaB = AvroSource.readMetadataFromFile(fileMetadata.resourceId()).getSchemaString(); - assertNotSame(schemaA, schemaB); - - AvroSource sourceA = (AvroSource) AvroSource.from(filename) - .withSchema(schemaA).createForSubrangeOfFile(fileMetadata, 0L, 0L); - AvroSource sourceB = (AvroSource) AvroSource.from(filename) - .withSchema(schemaB).createForSubrangeOfFile(fileMetadata, 0L, 0L); - assertSame(sourceA.getReadSchema(), sourceA.getFileSchema()); - assertSame(sourceA.getReadSchema(), sourceB.getReadSchema()); - assertSame(sourceA.getReadSchema(), sourceB.getFileSchema()); - - // Schemas are transient and not serialized thus we don't need to worry about interning - // after deserialization. + assertSame(sourceA.getReaderSchemaString(), sourceC.getReaderSchemaString()); } private void assertEqualsWithGeneric(List expected, List actual) { From d4026da1ad1fa0864052b85a66c4af5975327e9f Mon Sep 17 00:00:00 2001 From: Eugene Kirpichov Date: Tue, 18 Jul 2017 14:09:03 -0700 Subject: [PATCH 167/346] Gets rid of opening Avro files in createForSubrangeOfFile codepath --- .../org/apache/beam/sdk/io/AvroSource.java | 176 ++++++------------ .../apache/beam/sdk/io/AvroSourceTest.java | 11 +- 2 files changed, 63 insertions(+), 124 deletions(-) diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/AvroSource.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/AvroSource.java index 0634774fe935e..30af344c21073 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/AvroSource.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/AvroSource.java @@ -21,6 +21,7 @@ import static com.google.common.base.Preconditions.checkState; import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.MoreObjects; import java.io.ByteArrayInputStream; import java.io.EOFException; import java.io.IOException; @@ -135,45 +136,33 @@ public class AvroSource extends BlockBasedSource { @Nullable private final String readerSchemaString; - // The JSON schema that was used to write the source Avro file (may differ from the schema we will - // use to read from it). - private final String writerSchemaString; - - // The following metadata fields are not user-configurable. They are extracted from the object - // container file header upon subsource creation. - - // The codec used to encode the blocks in the Avro file. String value drawn from those in - // https://avro.apache.org/docs/1.7.7/api/java/org/apache/avro/file/CodecFactory.html - private final String codec; - - // The object container file's 16-byte sync marker. - private final byte[] syncMarker; - /** * Reads from the given file name or pattern ("glob"). The returned source can be further * configured by calling {@link #withSchema} to return a type other than {@link GenericRecord}. */ public static AvroSource from(String fileNameOrPattern) { - return new AvroSource<>( - fileNameOrPattern, DEFAULT_MIN_BUNDLE_SIZE, null, GenericRecord.class, null, null); + return new AvroSource<>(fileNameOrPattern, DEFAULT_MIN_BUNDLE_SIZE, null, GenericRecord.class); } /** Reads files containing records that conform to the given schema. */ public AvroSource withSchema(String schema) { return new AvroSource<>( - getFileOrPatternSpec(), getMinBundleSize(), schema, GenericRecord.class, codec, syncMarker); + getFileOrPatternSpec(), getMinBundleSize(), schema, GenericRecord.class); } /** Like {@link #withSchema(String)}. */ public AvroSource withSchema(Schema schema) { - return new AvroSource<>(getFileOrPatternSpec(), getMinBundleSize(), schema.toString(), - GenericRecord.class, codec, syncMarker); + return new AvroSource<>( + getFileOrPatternSpec(), getMinBundleSize(), schema.toString(), GenericRecord.class); } /** Reads files containing records of the given class. */ public AvroSource withSchema(Class clazz) { - return new AvroSource<>(getFileOrPatternSpec(), getMinBundleSize(), - ReflectData.get().getSchema(clazz).toString(), clazz, codec, syncMarker); + return new AvroSource<>( + getFileOrPatternSpec(), + getMinBundleSize(), + ReflectData.get().getSchema(clazz).toString(), + clazz); } /** @@ -181,24 +170,15 @@ public AvroSource withSchema(Class clazz) { * minBundleSize} and its use. */ public AvroSource withMinBundleSize(long minBundleSize) { - return new AvroSource<>( - getFileOrPatternSpec(), minBundleSize, readerSchemaString, type, codec, syncMarker); + return new AvroSource<>(getFileOrPatternSpec(), minBundleSize, readerSchemaString, type); } /** Constructor for FILEPATTERN mode. */ private AvroSource( - String fileNameOrPattern, - long minBundleSize, - String readerSchemaString, - Class type, - String codec, - byte[] syncMarker) { + String fileNameOrPattern, long minBundleSize, String readerSchemaString, Class type) { super(fileNameOrPattern, minBundleSize); this.readerSchemaString = internSchemaString(readerSchemaString); - this.codec = codec; - this.syncMarker = syncMarker; this.type = type; - this.writerSchemaString = null; } /** Constructor for SINGLE_FILE_OR_SUBRANGE mode. */ @@ -208,16 +188,10 @@ private AvroSource( long startOffset, long endOffset, String readerSchemaString, - Class type, - String codec, - byte[] syncMarker, - String writerSchemaString) { + Class type) { super(metadata, minBundleSize, startOffset, endOffset); this.readerSchemaString = internSchemaString(readerSchemaString); - this.codec = codec; - this.syncMarker = syncMarker; this.type = type; - this.writerSchemaString = internSchemaString(writerSchemaString); } @Override @@ -229,39 +203,7 @@ public void validate() { @Override public BlockBasedSource createForSubrangeOfFile(Metadata fileMetadata, long start, long end) { - byte[] syncMarker = this.syncMarker; - String codec = this.codec; - String writerSchemaString = this.writerSchemaString; - // codec and syncMarker are initially null when the source is created, as they differ - // across input files and must be read from the file. Here, when we are creating a source - // for a subrange of a file, we can initialize these values. When the resulting AvroSource - // is further split, they do not need to be read again. - if (codec == null || syncMarker == null || writerSchemaString == null) { - AvroMetadata metadata; - try { - metadata = readMetadataFromFile(fileMetadata.resourceId()); - } catch (IOException e) { - throw new RuntimeException("Error reading metadata from file " + fileMetadata, e); - } - codec = metadata.getCodec(); - syncMarker = metadata.getSyncMarker(); - writerSchemaString = metadata.getSchemaString(); - } - // Note that if the writerSchemaString is equivalent to the readerSchemaString, "intern"ing - // the string will occur within the constructor and return the same reference as the - // readerSchemaString. This allows for Java to have an efficient serialization since it - // will only encode the schema once while just storing pointers to the encoded version - // within this source. - return new AvroSource<>( - fileMetadata, - getMinBundleSize(), - start, - end, - readerSchemaString, - type, - codec, - syncMarker, - writerSchemaString); + return new AvroSource<>(fileMetadata, getMinBundleSize(), start, end, readerSchemaString, type); } @Override @@ -280,27 +222,17 @@ String getReaderSchemaString() { return readerSchemaString; } - private byte[] getSyncMarker() { - return syncMarker; - } - - private String getCodec() { - return codec; - } - - /** - * Avro file metadata. - */ + /** Avro file metadata. */ @VisibleForTesting static class AvroMetadata { - private byte[] syncMarker; - private String codec; - private String schemaString; + private final byte[] syncMarker; + private final String codec; + private final String schemaString; AvroMetadata(byte[] syncMarker, String codec, String schemaString) { this.syncMarker = checkNotNull(syncMarker, "syncMarker"); this.codec = checkNotNull(codec, "codec"); - this.schemaString = checkNotNull(schemaString, "schemaString"); + this.schemaString = internSchemaString(checkNotNull(schemaString, "schemaString")); } /** @@ -391,18 +323,6 @@ static AvroMetadata readMetadataFromFile(ResourceId fileResource) throws IOExcep return new AvroMetadata(syncMarker, codec, schemaString); } - private DatumReader createDatumReader() { - checkNotNull(writerSchemaString, "No writer schema has been initialized for source %s", this); - Schema writerSchema = internOrParseSchemaString(writerSchemaString); - Schema readerSchema = - (readerSchemaString == null) ? writerSchema : internOrParseSchemaString(readerSchemaString); - if (type == GenericRecord.class) { - return new GenericDatumReader<>(writerSchema, readerSchema); - } else { - return new ReflectDatumReader<>(writerSchema, readerSchema); - } - } - // A logical reference cache used to store schemas and schema strings to allow us to // "intern" values and reduce the number of copies of equivalent objects. private static final Map schemaLogicalReferenceCache = new WeakHashMap<>(); @@ -443,18 +363,10 @@ private Object readResolve() throws ObjectStreamException { getStartOffset(), getEndOffset(), readerSchemaString, - type, - codec, - syncMarker, - writerSchemaString); + type); case FILEPATTERN: return new AvroSource<>( - getFileOrPatternSpec(), - getMinBundleSize(), - readerSchemaString, - type, - codec, - syncMarker); + getFileOrPatternSpec(), getMinBundleSize(), readerSchemaString, type); default: throw new InvalidObjectException( String.format("Unknown mode %s for AvroSource %s", getMode(), this)); @@ -518,11 +430,25 @@ private static InputStream decodeAsInputStream(byte[] data, String codec) throws } } - AvroBlock(byte[] data, long numRecords, AvroSource source) throws IOException { + AvroBlock( + byte[] data, + long numRecords, + Class type, + String readerSchemaString, + String writerSchemaString, + String codec) + throws IOException { this.numRecords = numRecords; - this.reader = source.createDatumReader(); - this.decoder = - DecoderFactory.get().binaryDecoder(decodeAsInputStream(data, source.getCodec()), null); + checkNotNull(writerSchemaString, "writerSchemaString"); + Schema writerSchema = internOrParseSchemaString(writerSchemaString); + Schema readerSchema = + internOrParseSchemaString( + MoreObjects.firstNonNull(readerSchemaString, writerSchemaString)); + this.reader = + (type == GenericRecord.class) + ? new GenericDatumReader(writerSchema, readerSchema) + : new ReflectDatumReader(writerSchema, readerSchema); + this.decoder = DecoderFactory.get().binaryDecoder(decodeAsInputStream(data, codec), null); } @Override @@ -558,6 +484,8 @@ public double getFractionOfBlockConsumed() { */ @Experimental(Experimental.Kind.SOURCE_SINK) public static class AvroReader extends BlockBasedReader { + private AvroMetadata metadata; + // The current block. private AvroBlock currentBlock; @@ -631,10 +559,17 @@ public boolean readNextBlock() throws IOException { "Only able to read %s/%s bytes in the block before EOF reached.", bytesRead, blockSize); - currentBlock = new AvroBlock<>(data, numRecords, getCurrentSource()); + currentBlock = + new AvroBlock<>( + data, + numRecords, + getCurrentSource().type, + getCurrentSource().readerSchemaString, + metadata.getSchemaString(), + metadata.getCodec()); // Read the end of this block, which MUST be a sync marker for correctness. - byte[] syncMarker = getCurrentSource().getSyncMarker(); + byte[] syncMarker = metadata.getSyncMarker(); byte[] readSyncMarker = new byte[syncMarker.length]; long syncMarkerOffset = startOfNextBlock + headerSize + blockSize; bytesRead = IOUtils.readFully(stream, readSyncMarker); @@ -705,7 +640,7 @@ public long getSplitPointsRemaining() { private PushbackInputStream createStream(ReadableByteChannel channel) { return new PushbackInputStream( Channels.newInputStream(channel), - getCurrentSource().getSyncMarker().length); + metadata.getSyncMarker().length); } // Postcondition: the stream is positioned at the beginning of the first block after the start @@ -713,8 +648,15 @@ private PushbackInputStream createStream(ReadableByteChannel channel) { // currentBlockSizeBytes will be set to 0 indicating that the previous block was empty. @Override protected void startReading(ReadableByteChannel channel) throws IOException { + try { + metadata = readMetadataFromFile(getCurrentSource().getSingleFileMetadata().resourceId()); + } catch (IOException e) { + throw new RuntimeException( + "Error reading metadata from file " + getCurrentSource().getSingleFileMetadata(), e); + } + long startOffset = getCurrentSource().getStartOffset(); - byte[] syncMarker = getCurrentSource().getSyncMarker(); + byte[] syncMarker = metadata.getSyncMarker(); long syncMarkerLength = syncMarker.length; if (startOffset != 0) { diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/AvroSourceTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/AvroSourceTest.java index 0fc2b3e6b121c..bf2ac95272ff2 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/AvroSourceTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/AvroSourceTest.java @@ -21,7 +21,6 @@ import static org.hamcrest.Matchers.containsInAnyOrder; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertNotSame; import static org.junit.Assert.assertSame; import static org.junit.Assert.assertThat; import static org.junit.Assert.assertTrue; @@ -439,12 +438,10 @@ public void testSchemaStringIsInterned() throws Exception { String filename = generateTestFile("tmp.avro", birds, SyncBehavior.SYNC_DEFAULT, 0, AvroCoder.of(Bird.class), DataFileConstants.NULL_CODEC); Metadata fileMetadata = FileSystems.matchSingleFileSpec(filename); - String schemaA = AvroSource.readMetadataFromFile(fileMetadata.resourceId()).getSchemaString(); - String schemaB = AvroSource.readMetadataFromFile(fileMetadata.resourceId()).getSchemaString(); - assertNotSame(schemaA, schemaB); - - AvroSource sourceA = AvroSource.from(filename).withSchema(schemaA); - AvroSource sourceB = AvroSource.from(filename).withSchema(schemaB); + String schema = AvroSource.readMetadataFromFile(fileMetadata.resourceId()).getSchemaString(); + // Add "" to the schema to make sure it is not interned. + AvroSource sourceA = AvroSource.from(filename).withSchema("" + schema); + AvroSource sourceB = AvroSource.from(filename).withSchema("" + schema); assertSame(sourceA.getReaderSchemaString(), sourceB.getReaderSchemaString()); // Ensure that deserialization still goes through interning From cfadecb56c64aa155c8b5fd0d8a6654ceb918eba Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Mon, 17 Jul 2017 22:03:09 -0700 Subject: [PATCH 168/346] Uniquify application nodes in TextIOReadTest and only run pipeline once - These nodes have never had stable unique names, but we have always mutated the pipeline before running it so they got lucky. - A pipeline can also be mutated when run() so it should be considered dead after run() is called. This fixes both issues by uniquifying the names and running a given pipeline only once. --- .../apache/beam/sdk/io/TextIOReadTest.java | 22 +++++++++++++++---- 1 file changed, 18 insertions(+), 4 deletions(-) diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/TextIOReadTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/TextIOReadTest.java index 8b531114a3f6a..e733010acf3d4 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/TextIOReadTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/TextIOReadTest.java @@ -94,6 +94,7 @@ public class TextIOReadTest { private static final List TINY = Arrays.asList("Irritable eagle", "Optimistic jay", "Fanciful hawk"); private static final List LARGE = makeLines(1000); + private static int uniquifier = 0; private static Path tempFolder; private static File emptyTxt; @@ -296,17 +297,19 @@ private static void writeToStreamAndClose(List lines, OutputStream outpu private void assertReadingCompressedFileMatchesExpected( File file, CompressionType compressionType, List expected) { + int thisUniquifier = ++uniquifier; + TextIO.Read read = TextIO.read().from(file.getPath()).withCompressionType(compressionType); - PAssert.that(p.apply("Read_" + file + "_" + compressionType.toString(), read)) + PAssert.that( + p.apply("Read_" + file + "_" + compressionType.toString() + "_" + thisUniquifier, read)) .containsInAnyOrder(expected); TextIO.ReadAll readAll = TextIO.readAll().withCompressionType(compressionType).withDesiredBundleSizeBytes(10); PAssert.that( - p.apply("Create_" + file, Create.of(file.getPath())) - .apply("Read_" + compressionType.toString(), readAll)) + p.apply("Create_" + file + "_" + thisUniquifier, Create.of(file.getPath())) + .apply("Read_" + compressionType.toString() + "_" + thisUniquifier, readAll)) .containsInAnyOrder(expected); - p.run(); } /** Helper to make an array of compressible strings. Returns ["word"i] for i in range(0,n). */ @@ -324,6 +327,7 @@ private static List makeLines(int n) { public void testSmallCompressedGzipReadNoExtension() throws Exception { File smallGzNoExtension = writeToFile(TINY, "tiny_gz_no_extension", GZIP); assertReadingCompressedFileMatchesExpected(smallGzNoExtension, GZIP, TINY); + p.run(); } /** @@ -340,6 +344,7 @@ public void testSmallCompressedGzipReadActuallyUncompressed() throws Exception { assertReadingCompressedFileMatchesExpected(smallGzNotCompressed, GZIP, TINY); // Should also work with AUTO mode set. assertReadingCompressedFileMatchesExpected(smallGzNotCompressed, AUTO, TINY); + p.run(); } /** Tests reading from a small, bzip2ed file with no .bz2 extension but BZIP2 compression set. */ @@ -348,6 +353,7 @@ public void testSmallCompressedGzipReadActuallyUncompressed() throws Exception { public void testSmallCompressedBzip2ReadNoExtension() throws Exception { File smallBz2NoExtension = writeToFile(TINY, "tiny_bz2_no_extension", BZIP2); assertReadingCompressedFileMatchesExpected(smallBz2NoExtension, BZIP2, TINY); + p.run(); } /** @@ -393,6 +399,7 @@ public void testTxtRead() throws Exception { assertReadingCompressedFileMatchesExpected(tinyTxt, type, TINY); assertReadingCompressedFileMatchesExpected(largeTxt, type, LARGE); } + p.run(); } @Test @@ -411,6 +418,7 @@ public void testGzipCompressedRead() throws Exception { // GZIP files with non-gz extension should work in GZIP mode. File gzFile = writeToFile(TINY, "tiny_gz_no_extension", GZIP); assertReadingCompressedFileMatchesExpected(gzFile, GZIP, TINY); + p.run(); } @Test @@ -429,6 +437,7 @@ public void testBzip2CompressedRead() throws Exception { // BZ2 files with non-bz2 extension should work in BZIP2 mode. File bz2File = writeToFile(TINY, "tiny_bz2_no_extension", BZIP2); assertReadingCompressedFileMatchesExpected(bz2File, BZIP2, TINY); + p.run(); } @Test @@ -447,6 +456,7 @@ public void testZipCompressedRead() throws Exception { // Zip files with non-zip extension should work in ZIP mode. File zipFile = writeToFile(TINY, "tiny_zip_no_extension", ZIP); assertReadingCompressedFileMatchesExpected(zipFile, ZIP, TINY); + p.run(); } @Test @@ -465,6 +475,7 @@ public void testDeflateCompressedRead() throws Exception { // Deflate files with non-deflate extension should work in DEFLATE mode. File deflateFile = writeToFile(TINY, "tiny_deflate_no_extension", DEFLATE); assertReadingCompressedFileMatchesExpected(deflateFile, DEFLATE, TINY); + p.run(); } /** @@ -476,6 +487,7 @@ public void testDeflateCompressedRead() throws Exception { public void testZipCompressedReadWithNoEntries() throws Exception { String filename = createZipFile(new ArrayList(), "empty zip file"); assertReadingCompressedFileMatchesExpected(new File(filename), CompressionType.ZIP, EMPTY); + p.run(); } /** @@ -493,6 +505,7 @@ public void testZipCompressedReadWithMultiEntriesFile() throws Exception { String filename = createZipFile(expected, "multiple entries", entry0, entry1, entry2); assertReadingCompressedFileMatchesExpected(new File(filename), CompressionType.ZIP, expected); + p.run(); } /** @@ -513,6 +526,7 @@ public void testZipCompressedReadWithComplexEmptyAndPresentEntries() throws Exce assertReadingCompressedFileMatchesExpected( new File(filename), CompressionType.ZIP, Arrays.asList("cat", "dog")); + p.run(); } @Test From e655f53c2e732bcd082449ba8d0b551f417e8aaa Mon Sep 17 00:00:00 2001 From: manuzhang Date: Fri, 21 Jul 2017 14:27:37 +0800 Subject: [PATCH 169/346] Revert accidental changes to sdks/java/pom.xml --- sdks/java/pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sdks/java/pom.xml b/sdks/java/pom.xml index 3c1b6ec407061..3144193b9839e 100644 --- a/sdks/java/pom.xml +++ b/sdks/java/pom.xml @@ -38,7 +38,7 @@ build-tools --> core io - + maven-archetypes extensions - - gearpump-runner - - - org.apache.beam - beam-runners-gearpump - runtime - - - - flink-runner diff --git a/examples/java8/pom.xml b/examples/java8/pom.xml index 6fd29a496bb82..7beb955f4a232 100644 --- a/examples/java8/pom.xml +++ b/examples/java8/pom.xml @@ -151,6 +151,18 @@ + + + + gearpump-runner + + + org.apache.beam + beam-runners-gearpump + runtime + + + diff --git a/runners/gearpump/pom.xml b/runners/gearpump/pom.xml index ee90d484ccaad..3a4722fd7cef5 100644 --- a/runners/gearpump/pom.xml +++ b/runners/gearpump/pom.xml @@ -49,8 +49,7 @@ local-validates-runner-tests - - true + false From 483abc0941f0fb42c506565f6912153296fd94b5 Mon Sep 17 00:00:00 2001 From: Sourabh Bajaj Date: Mon, 24 Jul 2017 15:54:02 -0700 Subject: [PATCH 193/346] We shouldn't write to re-created tables for 2 mins --- sdks/python/apache_beam/io/gcp/bigquery.py | 19 +++++++++++++++---- .../apache_beam/io/gcp/bigquery_test.py | 3 ++- 2 files changed, 17 insertions(+), 5 deletions(-) diff --git a/sdks/python/apache_beam/io/gcp/bigquery.py b/sdks/python/apache_beam/io/gcp/bigquery.py index 23fd31036435e..db6715ad714d8 100644 --- a/sdks/python/apache_beam/io/gcp/bigquery.py +++ b/sdks/python/apache_beam/io/gcp/bigquery.py @@ -1002,12 +1002,23 @@ def get_or_create_table( if found_table and write_disposition != BigQueryDisposition.WRITE_TRUNCATE: return found_table else: + created_table = self._create_table(project_id=project_id, + dataset_id=dataset_id, + table_id=table_id, + schema=schema or found_table.schema) # if write_disposition == BigQueryDisposition.WRITE_TRUNCATE we delete # the table before this point. - return self._create_table(project_id=project_id, - dataset_id=dataset_id, - table_id=table_id, - schema=schema or found_table.schema) + if write_disposition == BigQueryDisposition.WRITE_TRUNCATE: + # BigQuery can route data to the old table for 2 mins max so wait + # that much time before creating the table and writing it + logging.warning('Sleeping for 150 seconds before the write as ' + + 'BigQuery inserts can be routed to deleted table ' + + 'for 2 mins after the delete and create.') + # TODO(BEAM-2673): Remove this sleep by migrating to load api + time.sleep(150) + return created_table + else: + return created_table def run_query(self, project_id, query, use_legacy_sql, flatten_results, dry_run=False): diff --git a/sdks/python/apache_beam/io/gcp/bigquery_test.py b/sdks/python/apache_beam/io/gcp/bigquery_test.py index 14247bad8cbe5..bfd06ace6400b 100644 --- a/sdks/python/apache_beam/io/gcp/bigquery_test.py +++ b/sdks/python/apache_beam/io/gcp/bigquery_test.py @@ -650,7 +650,8 @@ def test_table_empty_and_write_disposition_empty(self): self.assertFalse(client.tables.Delete.called) self.assertFalse(client.tables.Insert.called) - def test_table_with_write_disposition_truncate(self): + @mock.patch('time.sleep', return_value=None) + def test_table_with_write_disposition_truncate(self, _patched_sleep): client = mock.Mock() table = bigquery.Table( tableReference=bigquery.TableReference( From 56a1bceaad8aeec4705c562db4dab08a4d4f36b3 Mon Sep 17 00:00:00 2001 From: Eugene Kirpichov Date: Tue, 25 Jul 2017 20:00:41 -0700 Subject: [PATCH 194/346] Sets desired bundle size on AvroIO.readAll --- sdks/java/core/src/main/java/org/apache/beam/sdk/io/AvroIO.java | 1 + 1 file changed, 1 insertion(+) 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 f2011149a61e4..bc7fecbd2fb1d 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 @@ -178,6 +178,7 @@ public static ReadAll readAllGenericRecords(Schema schema) { return new AutoValue_AvroIO_ReadAll.Builder() .setRecordClass(GenericRecord.class) .setSchema(schema) + .setDesiredBundleSizeBytes(64 * 1024 * 1024L) .build(); } From a282165c64ef7b5812fa9e44afd94777211a8707 Mon Sep 17 00:00:00 2001 From: manuzhang Date: Wed, 26 Jul 2017 20:26:05 +0800 Subject: [PATCH 195/346] Require java 8 for precommit tests and add gearpump-runner --- .../jenkins/job_beam_PreCommit_Java_MavenInstall.groovy | 2 +- examples/java/pom.xml | 8 ++++++++ 2 files changed, 9 insertions(+), 1 deletion(-) diff --git a/.test-infra/jenkins/job_beam_PreCommit_Java_MavenInstall.groovy b/.test-infra/jenkins/job_beam_PreCommit_Java_MavenInstall.groovy index bc130ece20c05..7117f12c46e21 100644 --- a/.test-infra/jenkins/job_beam_PreCommit_Java_MavenInstall.groovy +++ b/.test-infra/jenkins/job_beam_PreCommit_Java_MavenInstall.groovy @@ -41,5 +41,5 @@ mavenJob('beam_PreCommit_Java_MavenInstall') { common_job_properties.setPreCommit(delegate, '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 -DrepoToken=$COVERALLS_REPO_TOKEN -DpullRequest=$ghprbPullId help:effective-settings clean install coveralls:report') + goals('-B -e -Prelease,include-runners,jenkins-precommit,direct-runner,dataflow-runner,spark-runner,flink-runner,apex-runner,gearpump-runner -DrepoToken=$COVERALLS_REPO_TOKEN -DpullRequest=$ghprbPullId help:effective-settings clean install coveralls:report') } diff --git a/examples/java/pom.xml b/examples/java/pom.xml index ae64a79340d0b..bd18889024f14 100644 --- a/examples/java/pom.xml +++ b/examples/java/pom.xml @@ -155,6 +155,9 @@ jenkins-precommit + + [1.8,) + false true @@ -175,6 +178,11 @@ ${apex.kryo.version} runtime + + org.apache.beam + beam-runners-gearpump + runtime + From 809a7f861bcdfb2ccba3690b439940c702c49701 Mon Sep 17 00:00:00 2001 From: Neville Li Date: Sat, 22 Jul 2017 00:14:03 -0700 Subject: [PATCH 196/346] [BEAM-2658] register DefaultCoder first and SerializableCoder last --- .../apache/beam/sdk/coders/CoderRegistry.java | 4 ++ .../apache/beam/sdk/coders/DefaultCoder.java | 2 - .../beam/sdk/coders/SerializableCoder.java | 2 - .../beam/sdk/coders/CoderRegistryTest.java | 69 +++++++++++++++++++ 4 files changed, 73 insertions(+), 4 deletions(-) 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 53cb6d3984928..48389b1abbfbf 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 @@ -147,9 +147,13 @@ public Coder coderFor(TypeDescriptor typeDescriptor, Set registrars = Sets.newTreeSet(ObjectsClassComparator.INSTANCE); registrars.addAll(Lists.newArrayList( ServiceLoader.load(CoderProviderRegistrar.class, ReflectHelpers.findClassLoader()))); + + // DefaultCoder should have the highest precedence and SerializableCoder the lowest + codersToRegister.addAll(new DefaultCoder.DefaultCoderProviderRegistrar().getCoderProviders()); for (CoderProviderRegistrar registrar : registrars) { codersToRegister.addAll(registrar.getCoderProviders()); } + codersToRegister.add(SerializableCoder.getCoderProvider()); REGISTERED_CODER_FACTORIES = ImmutableList.copyOf(codersToRegister); } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/DefaultCoder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/DefaultCoder.java index 6eff9e9654b03..7eb2ecbff3d6b 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/DefaultCoder.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/DefaultCoder.java @@ -17,7 +17,6 @@ */ package org.apache.beam.sdk.coders; -import com.google.auto.service.AutoService; import com.google.common.collect.ImmutableList; import java.lang.annotation.Documented; import java.lang.annotation.ElementType; @@ -57,7 +56,6 @@ * the {@code @DefaultCoder} annotation to provide {@link CoderProvider coder providers} that * creates {@link Coder}s. */ - @AutoService(CoderProviderRegistrar.class) class DefaultCoderProviderRegistrar implements CoderProviderRegistrar { @Override diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/SerializableCoder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/SerializableCoder.java index 66918768818f0..9204942d0f55b 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/SerializableCoder.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/SerializableCoder.java @@ -17,7 +17,6 @@ */ package org.apache.beam.sdk.coders; -import com.google.auto.service.AutoService; import com.google.common.collect.ImmutableList; import java.io.IOException; import java.io.InputStream; @@ -80,7 +79,6 @@ public static CoderProvider getCoderProvider() { * A {@link CoderProviderRegistrar} which registers a {@link CoderProvider} which can handle * serializable types. */ - @AutoService(CoderProviderRegistrar.class) public static class SerializableCoderProviderRegistrar implements CoderProviderRegistrar { @Override 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 d1113f7aa9383..b6430e52bbe86 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 @@ -467,4 +467,73 @@ public List getCoderProviders() { AutoRegistrationClassCoder.INSTANCE)); } } + + @Test + public void testCoderPrecedence() throws Exception { + CoderRegistry registry = CoderRegistry.createDefault(); + + // DefaultCoder precedes CoderProviderRegistrar + assertEquals(AvroCoder.of(MyValueA.class), registry.getCoder(MyValueA.class)); + + // CoderProviderRegistrar precedes SerializableCoder + assertEquals(MyValueBCoder.INSTANCE, registry.getCoder(MyValueB.class)); + + // fallbacks to SerializableCoder at last + assertEquals(SerializableCoder.of(MyValueC.class), registry.getCoder(MyValueC.class)); + } + + @DefaultCoder(AvroCoder.class) + private static class MyValueA implements Serializable {} + + private static class MyValueB implements Serializable {} + + private static class MyValueC implements Serializable {} + + private static class MyValueACoder extends CustomCoder { + private static final MyValueACoder INSTANCE = new MyValueACoder(); + + @Override + public void encode(MyValueA value, OutputStream outStream) throws CoderException, IOException {} + + @Override + public MyValueA decode(InputStream inStream) throws CoderException, IOException { + return null; + } + } + + /** + * A {@link CoderProviderRegistrar} to demonstrate default {@link Coder} registration. + */ + @AutoService(CoderProviderRegistrar.class) + public static class MyValueACoderProviderRegistrar implements CoderProviderRegistrar { + @Override + public List getCoderProviders() { + return ImmutableList.of( + CoderProviders.forCoder(TypeDescriptor.of(MyValueA.class), MyValueACoder.INSTANCE)); + } + } + + private static class MyValueBCoder extends CustomCoder { + private static final MyValueBCoder INSTANCE = new MyValueBCoder(); + + @Override + public void encode(MyValueB value, OutputStream outStream) throws CoderException, IOException {} + + @Override + public MyValueB decode(InputStream inStream) throws CoderException, IOException { + return null; + } + } + + /** + * A {@link CoderProviderRegistrar} to demonstrate default {@link Coder} registration. + */ + @AutoService(CoderProviderRegistrar.class) + public static class MyValueBCoderProviderRegistrar implements CoderProviderRegistrar { + @Override + public List getCoderProviders() { + return ImmutableList.of( + CoderProviders.forCoder(TypeDescriptor.of(MyValueB.class), MyValueBCoder.INSTANCE)); + } + } } From 22e49e6035c54f95c604be8cf15daecba458240a Mon Sep 17 00:00:00 2001 From: Robert Bradshaw Date: Mon, 17 Jul 2017 16:51:52 -0700 Subject: [PATCH 197/346] Use runner API for read operation. --- sdks/python/apache_beam/io/gcp/pubsub.py | 3 +++ sdks/python/apache_beam/io/iobase.py | 27 ++++++++++++++++++- .../runners/dataflow/native_io/iobase.py | 8 +++++- sdks/python/apache_beam/utils/urns.py | 3 +++ 4 files changed, 39 insertions(+), 2 deletions(-) diff --git a/sdks/python/apache_beam/io/gcp/pubsub.py b/sdks/python/apache_beam/io/gcp/pubsub.py index 32d388a9e50f6..7d1f3553f4fac 100644 --- a/sdks/python/apache_beam/io/gcp/pubsub.py +++ b/sdks/python/apache_beam/io/gcp/pubsub.py @@ -183,6 +183,9 @@ def reader(self): raise NotImplementedError( 'PubSubPayloadSource is not supported in local execution.') + def is_bounded(self): + return False + class _PubSubPayloadSink(dataflow_io.NativeSink): """Sink for the payload of a message as bytes to a Cloud Pub/Sub topic.""" diff --git a/sdks/python/apache_beam/io/iobase.py b/sdks/python/apache_beam/io/iobase.py index 7e40d833e4729..bdd06e185c78d 100644 --- a/sdks/python/apache_beam/io/iobase.py +++ b/sdks/python/apache_beam/io/iobase.py @@ -37,6 +37,7 @@ from apache_beam import pvalue from apache_beam import coders +from apache_beam.portability.api import beam_runner_api_pb2 from apache_beam.pvalue import AsIter from apache_beam.pvalue import AsSingleton from apache_beam.transforms import core @@ -44,6 +45,7 @@ from apache_beam.transforms import window from apache_beam.transforms.display import HasDisplayData from apache_beam.transforms.display import DisplayDataItem +from apache_beam.utils import urns from apache_beam.utils.windowed_value import WindowedValue __all__ = ['BoundedSource', 'RangeTracker', 'Read', 'Sink', 'Write', 'Writer'] @@ -70,7 +72,7 @@ 'weight source start_position stop_position') -class BoundedSource(HasDisplayData): +class BoundedSource(HasDisplayData, urns.RunnerApiFn): """A source that reads a finite amount of input records. This class defines following operations which can be used to read the source @@ -189,6 +191,11 @@ def default_output_coder(self): """ return coders.registry.get_coder(object) + def is_bounded(self): + return True + + urns.RunnerApiFn.register_pickle_urn(urns.PICKLED_SOURCE) + class RangeTracker(object): """A thread safe object used by Dataflow source framework. @@ -820,6 +827,24 @@ def display_data(self): label='Read Source'), 'source_dd': self.source} + def to_runner_api_parameter(self, context): + return (urns.READ_TRANSFORM, + beam_runner_api_pb2.ReadPayload( + source=self.source.to_runner_api(context), + is_bounded=beam_runner_api_pb2.BOUNDED + if self.source.is_bounded() + else beam_runner_api_pb2.UNBOUNDED)) + + @staticmethod + def from_runner_api_parameter(parameter, context): + return Read(BoundedSource.from_runner_api(parameter.source, context)) + + +ptransform.PTransform.register_urn( + urns.READ_TRANSFORM, + beam_runner_api_pb2.ReadPayload, + Read.from_runner_api_parameter) + class Write(ptransform.PTransform): """A ``PTransform`` that writes to a sink. diff --git a/sdks/python/apache_beam/runners/dataflow/native_io/iobase.py b/sdks/python/apache_beam/runners/dataflow/native_io/iobase.py index c1f4238178ace..f29b714e0a8d1 100644 --- a/sdks/python/apache_beam/runners/dataflow/native_io/iobase.py +++ b/sdks/python/apache_beam/runners/dataflow/native_io/iobase.py @@ -25,6 +25,7 @@ from apache_beam import pvalue from apache_beam.transforms import ptransform from apache_beam.transforms.display import HasDisplayData +from apache_beam.utils import urns def _dict_printable_fields(dict_object, skip_fields): @@ -42,7 +43,7 @@ def _dict_printable_fields(dict_object, skip_fields): 'compression_type'] -class NativeSource(HasDisplayData): +class NativeSource(HasDisplayData, urns.RunnerApiFn): """A source implemented by Dataflow service. This class is to be only inherited by sources natively implemented by Cloud @@ -55,12 +56,17 @@ def reader(self): """Returns a NativeSourceReader instance associated with this source.""" raise NotImplementedError + def is_bounded(self): + return True + def __repr__(self): return '<{name} {vals}>'.format( name=self.__class__.__name__, vals=', '.join(_dict_printable_fields(self.__dict__, _minor_fields))) + urns.RunnerApiFn.register_pickle_urn(urns.PICKLED_SOURCE) + class NativeSourceReader(object): """A reader for a source implemented by Dataflow service.""" diff --git a/sdks/python/apache_beam/utils/urns.py b/sdks/python/apache_beam/utils/urns.py index e553eea95f6a0..c155cfd4001c9 100644 --- a/sdks/python/apache_beam/utils/urns.py +++ b/sdks/python/apache_beam/utils/urns.py @@ -36,8 +36,11 @@ PICKLED_TRANSFORM = "beam:ptransform:pickled_python:v0.1" FLATTEN_TRANSFORM = "beam:ptransform:flatten:v0.1" +READ_TRANSFORM = "beam:ptransform:read:v0.1" WINDOW_INTO_TRANSFORM = "beam:ptransform:window_into:v0.1" +PICKLED_SOURCE = "beam:source:pickled_python:v0.1" + class RunnerApiFn(object): """Abstract base class that provides urn registration utilities. From 24b39e457519f45df55a90a6c6c92fa8df5128b7 Mon Sep 17 00:00:00 2001 From: Robert Bradshaw Date: Mon, 24 Jul 2017 17:19:47 -0700 Subject: [PATCH 198/346] Disable abc metaclass due to issues with pickling. --- sdks/python/apache_beam/utils/urns.py | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/sdks/python/apache_beam/utils/urns.py b/sdks/python/apache_beam/utils/urns.py index c155cfd4001c9..9e4635d2b2302 100644 --- a/sdks/python/apache_beam/utils/urns.py +++ b/sdks/python/apache_beam/utils/urns.py @@ -53,7 +53,8 @@ class RunnerApiFn(object): to register serialization via pickling. """ - __metaclass__ = abc.ABCMeta + # TODO(robertwb): Figure out issue with dill + local classes + abc metaclass + # __metaclass__ = abc.ABCMeta _known_urns = {} From 2d307f29b77e6aeb28f624f50246e99422a1173d Mon Sep 17 00:00:00 2001 From: Robert Bradshaw Date: Tue, 25 Jul 2017 13:33:03 -0700 Subject: [PATCH 199/346] Create common superclass for all Sources. --- sdks/python/apache_beam/io/iobase.py | 12 ++++++++---- .../apache_beam/runners/dataflow/native_io/iobase.py | 6 ++---- 2 files changed, 10 insertions(+), 8 deletions(-) diff --git a/sdks/python/apache_beam/io/iobase.py b/sdks/python/apache_beam/io/iobase.py index bdd06e185c78d..db75fe38baa99 100644 --- a/sdks/python/apache_beam/io/iobase.py +++ b/sdks/python/apache_beam/io/iobase.py @@ -72,7 +72,13 @@ 'weight source start_position stop_position') -class BoundedSource(HasDisplayData, urns.RunnerApiFn): +class SourceBase(HasDisplayData, urns.RunnerApiFn): + """Base class for all sources that can be passed to beam.io.Read(...). + """ + urns.RunnerApiFn.register_pickle_urn(urns.PICKLED_SOURCE) + + +class BoundedSource(SourceBase): """A source that reads a finite amount of input records. This class defines following operations which can be used to read the source @@ -194,8 +200,6 @@ def default_output_coder(self): def is_bounded(self): return True - urns.RunnerApiFn.register_pickle_urn(urns.PICKLED_SOURCE) - class RangeTracker(object): """A thread safe object used by Dataflow source framework. @@ -837,7 +841,7 @@ def to_runner_api_parameter(self, context): @staticmethod def from_runner_api_parameter(parameter, context): - return Read(BoundedSource.from_runner_api(parameter.source, context)) + return Read(SourceBase.from_runner_api(parameter.source, context)) ptransform.PTransform.register_urn( diff --git a/sdks/python/apache_beam/runners/dataflow/native_io/iobase.py b/sdks/python/apache_beam/runners/dataflow/native_io/iobase.py index f29b714e0a8d1..2f2316f6f1d08 100644 --- a/sdks/python/apache_beam/runners/dataflow/native_io/iobase.py +++ b/sdks/python/apache_beam/runners/dataflow/native_io/iobase.py @@ -23,9 +23,9 @@ import logging from apache_beam import pvalue +from apache_beam.io import iobase from apache_beam.transforms import ptransform from apache_beam.transforms.display import HasDisplayData -from apache_beam.utils import urns def _dict_printable_fields(dict_object, skip_fields): @@ -43,7 +43,7 @@ def _dict_printable_fields(dict_object, skip_fields): 'compression_type'] -class NativeSource(HasDisplayData, urns.RunnerApiFn): +class NativeSource(iobase.SourceBase): """A source implemented by Dataflow service. This class is to be only inherited by sources natively implemented by Cloud @@ -65,8 +65,6 @@ def __repr__(self): vals=', '.join(_dict_printable_fields(self.__dict__, _minor_fields))) - urns.RunnerApiFn.register_pickle_urn(urns.PICKLED_SOURCE) - class NativeSourceReader(object): """A reader for a source implemented by Dataflow service.""" From 82cc1e1ad54d55d87ae912a96c148fd16503d017 Mon Sep 17 00:00:00 2001 From: Robert Bradshaw Date: Tue, 25 Jul 2017 16:12:16 -0700 Subject: [PATCH 200/346] Translate combining operations through the Runner API. --- .../runners/dataflow/dataflow_runner.py | 4 +- sdks/python/apache_beam/transforms/core.py | 37 ++++++++++++++++++- .../apache_beam/transforms/ptransform.py | 12 +++++- sdks/python/apache_beam/utils/urns.py | 5 ++- 4 files changed, 52 insertions(+), 6 deletions(-) diff --git a/sdks/python/apache_beam/runners/dataflow/dataflow_runner.py b/sdks/python/apache_beam/runners/dataflow/dataflow_runner.py index aec7d0042e254..d653e91d0923a 100644 --- a/sdks/python/apache_beam/runners/dataflow/dataflow_runner.py +++ b/sdks/python/apache_beam/runners/dataflow/dataflow_runner.py @@ -591,8 +591,8 @@ def run_CombineValues(self, transform_node): PropertyNames.OUTPUT_NAME: input_step.get_output(input_tag)}) # Note that the accumulator must not have a WindowedValue encoding, while # the output of this step does in fact have a WindowedValue encoding. - accumulator_encoding = self._get_encoded_output_coder(transform_node, - window_value=False) + accumulator_encoding = self._get_cloud_encoding( + transform_node.fn.get_accumulator_coder()) output_encoding = self._get_encoded_output_coder(transform_node) step.encoding = output_encoding diff --git a/sdks/python/apache_beam/transforms/core.py b/sdks/python/apache_beam/transforms/core.py index 92b8737f3e9fa..25fe39ff37450 100644 --- a/sdks/python/apache_beam/transforms/core.py +++ b/sdks/python/apache_beam/transforms/core.py @@ -25,6 +25,7 @@ from apache_beam import pvalue from apache_beam import typehints +from apache_beam import coders from apache_beam.coders import typecoders from apache_beam.internal import util from apache_beam.portability.api import beam_runner_api_pb2 @@ -311,7 +312,7 @@ def _process_argspec_fn(self): return getattr(self._fn, '_argspec_fn', self._fn) -class CombineFn(WithTypeHints, HasDisplayData): +class CombineFn(WithTypeHints, HasDisplayData, urns.RunnerApiFn): """A function object used by a Combine transform with custom processing. A CombineFn specifies how multiple values in all or part of a PCollection can @@ -430,6 +431,11 @@ def from_callable(fn): def maybe_from_callable(fn): return fn if isinstance(fn, CombineFn) else CallableWrapperCombineFn(fn) + def get_accumulator_coder(self): + return coders.registry.get_coder(object) + + urns.RunnerApiFn.register_pickle_urn(urns.PICKLED_COMBINE_FN) + class CallableWrapperCombineFn(CombineFn): """For internal use only; no backwards-compatibility guarantees. @@ -816,6 +822,13 @@ def Filter(fn, *args, **kwargs): # pylint: disable=invalid-name return pardo +def _combine_payload(combine_fn, context): + return beam_runner_api_pb2.CombinePayload( + combine_fn=combine_fn.to_runner_api(context), + accumulator_coder_id=context.coders.get_id( + combine_fn.get_accumulator_coder())) + + class CombineGlobally(PTransform): """A CombineGlobally transform. @@ -973,6 +986,17 @@ def expand(self, pcoll): return pcoll | GroupByKey() | 'Combine' >> CombineValues( self.fn, *args, **kwargs) + def to_runner_api_parameter(self, context): + return ( + urns.COMBINE_PER_KEY_TRANSFORM, + _combine_payload(self.fn, context)) + + @PTransform.register_urn( + urns.COMBINE_PER_KEY_TRANSFORM, beam_runner_api_pb2.CombinePayload) + def from_runner_api_parameter(combine_payload, context): + return CombinePerKey( + CombineFn.from_runner_api(combine_payload.combine_fn, context)) + # TODO(robertwb): Rename to CombineGroupedValues? class CombineValues(PTransformWithSideInputs): @@ -995,6 +1019,17 @@ def expand(self, pcoll): CombineValuesDoFn(key_type, self.fn, runtime_type_check), *args, **kwargs) + def to_runner_api_parameter(self, context): + return ( + urns.COMBINE_GROUPED_VALUES_TRANSFORM, + _combine_payload(self.fn, context)) + + @PTransform.register_urn( + urns.COMBINE_GROUPED_VALUES_TRANSFORM, beam_runner_api_pb2.CombinePayload) + def from_runner_api_parameter(combine_payload, context): + return CombineValues( + CombineFn.from_runner_api(combine_payload.combine_fn, context)) + class CombineValuesDoFn(DoFn): """DoFn for performing per-key Combine transforms.""" diff --git a/sdks/python/apache_beam/transforms/ptransform.py b/sdks/python/apache_beam/transforms/ptransform.py index cd84122d5e1ec..da113e055c946 100644 --- a/sdks/python/apache_beam/transforms/ptransform.py +++ b/sdks/python/apache_beam/transforms/ptransform.py @@ -426,8 +426,16 @@ def _dict_tuple_leaves(pvalueish): _known_urns = {} @classmethod - def register_urn(cls, urn, parameter_type, constructor): - cls._known_urns[urn] = parameter_type, constructor + def register_urn(cls, urn, parameter_type, constructor=None): + def register(constructor): + cls._known_urns[urn] = parameter_type, constructor + return staticmethod(constructor) + if constructor: + # Used as a statement. + register(constructor) + else: + # Used as a decorator. + return register def to_runner_api(self, context): from apache_beam.portability.api import beam_runner_api_pb2 diff --git a/sdks/python/apache_beam/utils/urns.py b/sdks/python/apache_beam/utils/urns.py index 9e4635d2b2302..7110802f3d476 100644 --- a/sdks/python/apache_beam/utils/urns.py +++ b/sdks/python/apache_beam/utils/urns.py @@ -32,9 +32,12 @@ SLIDING_WINDOWS_FN = "beam:windowfn:sliding_windows:v0.1" SESSION_WINDOWS_FN = "beam:windowfn:session_windows:v0.1" +PICKLED_COMBINE_FN = "beam:combinefn:pickled_python:v0.1" PICKLED_CODER = "beam:coder:pickled_python:v0.1" PICKLED_TRANSFORM = "beam:ptransform:pickled_python:v0.1" +COMBINE_PER_KEY_TRANSFORM = "beam:ptransform:combine_per_key:v0.1" +COMBINE_GROUPED_VALUES_TRANSFORM = "beam:ptransform:combine_grouped_values:v0.1" FLATTEN_TRANSFORM = "beam:ptransform:flatten:v0.1" READ_TRANSFORM = "beam:ptransform:read:v0.1" WINDOW_INTO_TRANSFORM = "beam:ptransform:window_into:v0.1" @@ -53,7 +56,7 @@ class RunnerApiFn(object): to register serialization via pickling. """ - # TODO(robertwb): Figure out issue with dill + local classes + abc metaclass + # TODO(BEAM-2685): Issue with dill + local classes + abc metaclass # __metaclass__ = abc.ABCMeta _known_urns = {} From 51867adfea0d78a7dcb0fb09b5c257cf8e0e2ff4 Mon Sep 17 00:00:00 2001 From: Thomas Groh Date: Thu, 20 Jul 2017 13:36:04 -0700 Subject: [PATCH 201/346] Migrate checkCombineFn in TestUtils to CombineFnTester This makes CombineFnTester significantly more discoverable, and usable without having dependencies on the test JAR. Update existing tests. --- .../beam/sdk/testing/CombineFnTester.java | 147 ++++++++++ .../java/org/apache/beam/sdk/TestUtils.java | 88 ------ .../beam/sdk/testing/CombineFnTesterTest.java | 276 ++++++++++++++++++ .../transforms/ApproximateQuantilesTest.java | 24 +- .../beam/sdk/transforms/CombineTest.java | 12 +- .../apache/beam/sdk/transforms/MaxTest.java | 8 +- .../apache/beam/sdk/transforms/MeanTest.java | 4 +- .../apache/beam/sdk/transforms/MinTest.java | 8 +- .../apache/beam/sdk/transforms/SumTest.java | 8 +- 9 files changed, 455 insertions(+), 120 deletions(-) create mode 100644 sdks/java/core/src/main/java/org/apache/beam/sdk/testing/CombineFnTester.java create mode 100644 sdks/java/core/src/test/java/org/apache/beam/sdk/testing/CombineFnTesterTest.java diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/CombineFnTester.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/CombineFnTester.java new file mode 100644 index 0000000000000..efd2af3544f16 --- /dev/null +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/CombineFnTester.java @@ -0,0 +1,147 @@ +/* + * 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 org.hamcrest.Matchers.is; +import static org.junit.Assert.assertThat; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import org.apache.beam.sdk.transforms.Combine.CombineFn; +import org.hamcrest.Matcher; + +/** + * Utilities for testing {@link CombineFn CombineFns}. Ensures that the {@link CombineFn} gives + * correct results across various permutations and shardings of the input. + */ +public class CombineFnTester { + /** + * Tests that the the {@link CombineFn}, when applied to the provided input, produces the provided + * output. Tests a variety of permutations of the input. + */ + public static void testCombineFn( + CombineFn fn, List input, final OutputT expected) { + testCombineFn(fn, input, is(expected)); + Collections.shuffle(input); + testCombineFn(fn, input, is(expected)); + } + + public static void testCombineFn( + CombineFn fn, List input, Matcher matcher) { + int size = input.size(); + checkCombineFnShardsMultipleOrders(fn, Collections.singletonList(input), matcher); + checkCombineFnShardsMultipleOrders(fn, shardEvenly(input, 2), matcher); + if (size > 4) { + checkCombineFnShardsMultipleOrders(fn, shardEvenly(input, size / 2), matcher); + checkCombineFnShardsMultipleOrders( + fn, shardEvenly(input, (int) (size / Math.sqrt(size))), matcher); + } + checkCombineFnShardsMultipleOrders(fn, shardExponentially(input, 1.4), matcher); + checkCombineFnShardsMultipleOrders(fn, shardExponentially(input, 2), matcher); + checkCombineFnShardsMultipleOrders(fn, shardExponentially(input, Math.E), matcher); + } + + private static void checkCombineFnShardsMultipleOrders( + CombineFn fn, + List> shards, + Matcher matcher) { + checkCombineFnShardsSingleMerge(fn, shards, matcher); + checkCombineFnShardsWithEmptyAccumulators(fn, shards, matcher); + checkCombineFnShardsIncrementalMerging(fn, shards, matcher); + Collections.shuffle(shards); + checkCombineFnShardsSingleMerge(fn, shards, matcher); + checkCombineFnShardsWithEmptyAccumulators(fn, shards, matcher); + checkCombineFnShardsIncrementalMerging(fn, shards, matcher); + } + + private static void checkCombineFnShardsSingleMerge( + CombineFn fn, + Iterable> shards, + Matcher matcher) { + List accumulators = combineInputs(fn, shards); + AccumT merged = fn.mergeAccumulators(accumulators); + assertThat(fn.extractOutput(merged), matcher); + } + + private static void checkCombineFnShardsWithEmptyAccumulators( + CombineFn fn, + Iterable> shards, + Matcher matcher) { + List accumulators = combineInputs(fn, shards); + accumulators.add(0, fn.createAccumulator()); + accumulators.add(fn.createAccumulator()); + AccumT merged = fn.mergeAccumulators(accumulators); + assertThat(fn.extractOutput(merged), matcher); + } + + private static void checkCombineFnShardsIncrementalMerging( + CombineFn fn, + List> shards, + Matcher matcher) { + AccumT accumulator = null; + for (AccumT inputAccum : combineInputs(fn, shards)) { + if (accumulator == null) { + accumulator = inputAccum; + } else { + accumulator = fn.mergeAccumulators(Arrays.asList(accumulator, inputAccum)); + } + } + assertThat(fn.extractOutput(accumulator), matcher); + } + + private static List combineInputs( + CombineFn fn, Iterable> shards) { + List accumulators = new ArrayList<>(); + int maybeCompact = 0; + for (Iterable shard : shards) { + AccumT accumulator = fn.createAccumulator(); + for (InputT elem : shard) { + accumulator = fn.addInput(accumulator, elem); + } + if (maybeCompact++ % 2 == 0) { + accumulator = fn.compact(accumulator); + } + accumulators.add(accumulator); + } + return accumulators; + } + + private static List> shardEvenly(List input, int numShards) { + List> shards = new ArrayList<>(numShards); + for (int i = 0; i < numShards; i++) { + shards.add(input.subList(i * input.size() / numShards, + (i + 1) * input.size() / numShards)); + } + return shards; + } + + private static List> shardExponentially( + List input, double base) { + assert base > 1.0; + List> shards = new ArrayList<>(); + int end = input.size(); + while (end > 0) { + int start = (int) (end / base); + shards.add(input.subList(start, end)); + end = start; + } + return shards; + } +} diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/TestUtils.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/TestUtils.java index 1224f10e0e8b3..5ccc1ac73ff6d 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/TestUtils.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/TestUtils.java @@ -17,15 +17,9 @@ */ package org.apache.beam.sdk; -import static org.junit.Assert.assertThat; - -import java.util.ArrayList; import java.util.Arrays; -import java.util.Collections; import java.util.List; -import org.apache.beam.sdk.transforms.Combine.CombineFn; import org.apache.beam.sdk.values.KV; -import org.hamcrest.CoreMatchers; import org.hamcrest.Description; import org.hamcrest.Matcher; import org.hamcrest.TypeSafeMatcher; @@ -127,86 +121,4 @@ public void describeTo(Description description) { .appendText(")"); } } - - //////////////////////////////////////////////////////////////////////////// - // Utilities for testing CombineFns, ensuring they give correct results - // across various permutations and shardings of the input. - - public static void checkCombineFn( - CombineFn fn, List input, final OutputT expected) { - checkCombineFn(fn, input, CoreMatchers.is(expected)); - } - - public static void checkCombineFn( - CombineFn fn, List input, Matcher matcher) { - checkCombineFnInternal(fn, input, matcher); - Collections.shuffle(input); - checkCombineFnInternal(fn, input, matcher); - } - - private static void checkCombineFnInternal( - CombineFn fn, List input, Matcher matcher) { - int size = input.size(); - checkCombineFnShards(fn, Collections.singletonList(input), matcher); - checkCombineFnShards(fn, shardEvenly(input, 2), matcher); - if (size > 4) { - checkCombineFnShards(fn, shardEvenly(input, size / 2), matcher); - checkCombineFnShards( - fn, shardEvenly(input, (int) (size / Math.sqrt(size))), matcher); - } - checkCombineFnShards(fn, shardExponentially(input, 1.4), matcher); - checkCombineFnShards(fn, shardExponentially(input, 2), matcher); - checkCombineFnShards(fn, shardExponentially(input, Math.E), matcher); - } - - public static void checkCombineFnShards( - CombineFn fn, - List> shards, - Matcher matcher) { - checkCombineFnShardsInternal(fn, shards, matcher); - Collections.shuffle(shards); - checkCombineFnShardsInternal(fn, shards, matcher); - } - - private static void checkCombineFnShardsInternal( - CombineFn fn, - Iterable> shards, - Matcher matcher) { - List accumulators = new ArrayList<>(); - int maybeCompact = 0; - for (Iterable shard : shards) { - AccumT accumulator = fn.createAccumulator(); - for (InputT elem : shard) { - accumulator = fn.addInput(accumulator, elem); - } - if (maybeCompact++ % 2 == 0) { - accumulator = fn.compact(accumulator); - } - accumulators.add(accumulator); - } - AccumT merged = fn.mergeAccumulators(accumulators); - assertThat(fn.extractOutput(merged), matcher); - } - - private static List> shardEvenly(List input, int numShards) { - List> shards = new ArrayList<>(numShards); - for (int i = 0; i < numShards; i++) { - shards.add(input.subList(i * input.size() / numShards, - (i + 1) * input.size() / numShards)); - } - return shards; - } - - private static List> shardExponentially( - List input, double base) { - assert base > 1.0; - List> shards = new ArrayList<>(); - int end = input.size(); - while (end > 0) { - int start = (int) (end / base); - shards.add(input.subList(start, end)); - end = start; - } - return shards; - } } diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/CombineFnTesterTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/CombineFnTesterTest.java new file mode 100644 index 0000000000000..15198b21e7c50 --- /dev/null +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/testing/CombineFnTesterTest.java @@ -0,0 +1,276 @@ +/* + * 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 org.hamcrest.Matchers.is; +import static org.junit.Assert.assertThat; +import static org.junit.Assert.fail; + +import com.google.common.collect.Iterables; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.concurrent.atomic.AtomicBoolean; +import org.apache.beam.sdk.transforms.Combine.CombineFn; +import org.apache.beam.sdk.transforms.Sum; +import org.hamcrest.Description; +import org.hamcrest.Matcher; +import org.hamcrest.Matchers; +import org.hamcrest.TypeSafeMatcher; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +/** + * Tests for {@link CombineFnTester}. + */ +@RunWith(JUnit4.class) +public class CombineFnTesterTest { + @Test + public void checksMergeWithEmptyAccumulators() { + final AtomicBoolean sawEmpty = new AtomicBoolean(false); + CombineFn combineFn = + new CombineFn() { + @Override + public Integer createAccumulator() { + return 0; + } + + @Override + public Integer addInput(Integer accumulator, Integer input) { + return accumulator + input; + } + + @Override + public Integer mergeAccumulators(Iterable accumulators) { + int result = 0; + for (int accum : accumulators) { + if (accum == 0) { + sawEmpty.set(true); + } + result += accum; + } + return result; + } + + @Override + public Integer extractOutput(Integer accumulator) { + return accumulator; + } + }; + + CombineFnTester.testCombineFn(combineFn, Arrays.asList(1, 2, 3, 4, 5), 15); + assertThat(sawEmpty.get(), is(true)); + } + + @Test + public void checksWithSingleShard() { + final AtomicBoolean sawSingleShard = new AtomicBoolean(); + CombineFn combineFn = + new CombineFn() { + int accumCount = 0; + + @Override + public Integer createAccumulator() { + accumCount++; + return 0; + } + + @Override + public Integer addInput(Integer accumulator, Integer input) { + return accumulator + input; + } + + @Override + public Integer mergeAccumulators(Iterable accumulators) { + int result = 0; + for (int accum : accumulators) { + result += accum; + } + return result; + } + + @Override + public Integer extractOutput(Integer accumulator) { + if (accumCount == 1) { + sawSingleShard.set(true); + } + accumCount = 0; + return accumulator; + } + }; + + CombineFnTester.testCombineFn(combineFn, Arrays.asList(1, 2, 3, 4, 5), 15); + assertThat(sawSingleShard.get(), is(true)); + } + + @Test + public void checksWithShards() { + final AtomicBoolean sawManyShards = new AtomicBoolean(); + CombineFn combineFn = + new CombineFn() { + + @Override + public Integer createAccumulator() { + return 0; + } + + @Override + public Integer addInput(Integer accumulator, Integer input) { + return accumulator + input; + } + + @Override + public Integer mergeAccumulators(Iterable accumulators) { + if (Iterables.size(accumulators) > 2) { + sawManyShards.set(true); + } + int result = 0; + for (int accum : accumulators) { + result += accum; + } + return result; + } + + @Override + public Integer extractOutput(Integer accumulator) { + return accumulator; + } + }; + + CombineFnTester.testCombineFn( + combineFn, Arrays.asList(1, 1, 1, 1, 1, 2, 2, 2, 2, 2, 3, 3, 3, 3, 3), 30); + assertThat(sawManyShards.get(), is(true)); + } + + @Test + public void checksWithMultipleMerges() { + final AtomicBoolean sawMultipleMerges = new AtomicBoolean(); + CombineFn combineFn = + new CombineFn() { + int mergeCalls = 0; + + @Override + public Integer createAccumulator() { + return 0; + } + + @Override + public Integer addInput(Integer accumulator, Integer input) { + return accumulator + input; + } + + @Override + public Integer mergeAccumulators(Iterable accumulators) { + mergeCalls++; + int result = 0; + for (int accum : accumulators) { + result += accum; + } + return result; + } + + @Override + public Integer extractOutput(Integer accumulator) { + if (mergeCalls > 1) { + sawMultipleMerges.set(true); + } + mergeCalls = 0; + return accumulator; + } + }; + + CombineFnTester.testCombineFn(combineFn, Arrays.asList(1, 1, 2, 2, 3, 3, 4, 4, 5, 5), 30); + assertThat(sawMultipleMerges.get(), is(true)); + } + + @Test + public void checksAlternateOrder() { + final AtomicBoolean sawOutOfOrder = new AtomicBoolean(); + CombineFn, Integer> combineFn = + new CombineFn, Integer>() { + @Override + public List createAccumulator() { + return new ArrayList<>(); + } + + @Override + public List addInput(List accumulator, Integer input) { + // If the input is being added to an empty accumulator, it's not known to be + // out of order, and it cannot be compared to the previous element. If the elements + // are out of order (relative to the input) a greater element will be added before + // a smaller one. + if (!accumulator.isEmpty() && accumulator.get(accumulator.size() - 1) > input) { + sawOutOfOrder.set(true); + } + accumulator.add(input); + return accumulator; + } + + @Override + public List mergeAccumulators(Iterable> accumulators) { + List result = new ArrayList<>(); + for (List accum : accumulators) { + result.addAll(accum); + } + return result; + } + + @Override + public Integer extractOutput(List accumulator) { + int value = 0; + for (int i : accumulator) { + value += i; + } + return value; + } + }; + + CombineFnTester.testCombineFn( + combineFn, Arrays.asList(1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14), 105); + assertThat(sawOutOfOrder.get(), is(true)); + } + + @Test + public void usesMatcher() { + final AtomicBoolean matcherUsed = new AtomicBoolean(); + Matcher matcher = + new TypeSafeMatcher() { + @Override + public void describeTo(Description description) {} + + @Override + protected boolean matchesSafely(Integer item) { + matcherUsed.set(true); + return item == 30; + } + }; + CombineFnTester.testCombineFn( + Sum.ofIntegers(), Arrays.asList(1, 1, 2, 2, 3, 3, 4, 4, 5, 5), matcher); + assertThat(matcherUsed.get(), is(true)); + try { + CombineFnTester.testCombineFn( + Sum.ofIntegers(), Arrays.asList(1, 2, 3, 4, 5), Matchers.not(Matchers.equalTo(15))); + } catch (AssertionError ignored) { + // Success! Return to avoid the call to fail(); + return; + } + fail("The matcher should have failed, throwing an error"); + } +} 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 9e0b3cc9069ec..e18083364cefb 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 @@ -17,7 +17,7 @@ */ package org.apache.beam.sdk.transforms; -import static org.apache.beam.sdk.TestUtils.checkCombineFn; +import static org.apache.beam.sdk.testing.CombineFnTester.testCombineFn; import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasDisplayItem; import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.collection.IsIterableContainingInOrder.contains; @@ -129,7 +129,7 @@ public void testQuantilesPerKey_reversed() { @Test public void testSingleton() { - checkCombineFn( + testCombineFn( ApproximateQuantilesCombineFn.create(5), Arrays.asList(389), Arrays.asList(389, 389, 389, 389, 389)); @@ -137,7 +137,7 @@ public void testSingleton() { @Test public void testSimpleQuantiles() { - checkCombineFn( + testCombineFn( ApproximateQuantilesCombineFn.create(5), intRange(101), Arrays.asList(0, 25, 50, 75, 100)); @@ -145,7 +145,7 @@ public void testSimpleQuantiles() { @Test public void testUnevenQuantiles() { - checkCombineFn( + testCombineFn( ApproximateQuantilesCombineFn.create(37), intRange(5000), quantileMatcher(5000, 37, 20 /* tolerance */)); @@ -153,7 +153,7 @@ public void testUnevenQuantiles() { @Test public void testLargerQuantiles() { - checkCombineFn( + testCombineFn( ApproximateQuantilesCombineFn.create(50), intRange(10001), quantileMatcher(10001, 50, 20 /* tolerance */)); @@ -161,7 +161,7 @@ public void testLargerQuantiles() { @Test public void testTightEpsilon() { - checkCombineFn( + testCombineFn( ApproximateQuantilesCombineFn.create(10).withEpsilon(0.01), intRange(10001), quantileMatcher(10001, 10, 5 /* tolerance */)); @@ -174,7 +174,7 @@ public void testDuplicates() { for (int i = 0; i < 10; i++) { all.addAll(intRange(size)); } - checkCombineFn( + testCombineFn( ApproximateQuantilesCombineFn.create(5), all, Arrays.asList(0, 25, 50, 75, 100)); @@ -190,7 +190,7 @@ public void testLotsOfDuplicates() { for (int i = 300; i < 1000; i++) { all.add(3); } - checkCombineFn( + testCombineFn( ApproximateQuantilesCombineFn.create(5), all, Arrays.asList(1, 2, 3, 3, 3)); @@ -202,7 +202,7 @@ public void testLogDistribution() { for (int i = 1; i < 1000; i++) { all.add((int) Math.log(i)); } - checkCombineFn( + testCombineFn( ApproximateQuantilesCombineFn.create(5), all, Arrays.asList(0, 5, 6, 6, 6)); @@ -214,7 +214,7 @@ public void testZipfianDistribution() { for (int i = 1; i < 1000; i++) { all.add(1000 / i); } - checkCombineFn( + testCombineFn( ApproximateQuantilesCombineFn.create(5), all, Arrays.asList(1, 1, 2, 4, 1000)); @@ -224,11 +224,11 @@ public void testZipfianDistribution() { public void testAlternateComparator() { List inputs = Arrays.asList( "aa", "aaa", "aaaa", "b", "ccccc", "dddd", "zz"); - checkCombineFn( + testCombineFn( ApproximateQuantilesCombineFn.create(3), inputs, Arrays.asList("aa", "b", "zz")); - checkCombineFn( + testCombineFn( ApproximateQuantilesCombineFn.create(3, new OrderByLength()), inputs, Arrays.asList("b", "aaa", "ccccc")); 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 b24d82dee4621..52fedc62ce6f9 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 @@ -19,7 +19,7 @@ import static com.google.common.base.Preconditions.checkNotNull; import static com.google.common.base.Preconditions.checkState; -import static org.apache.beam.sdk.TestUtils.checkCombineFn; +import static org.apache.beam.sdk.testing.CombineFnTester.testCombineFn; import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasDisplayItem; import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasNamespace; import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.includesDisplayDataFor; @@ -695,11 +695,11 @@ public void testBinaryCombineFn() { @Test public void testBinaryCombineFnWithNulls() { - checkCombineFn(new NullCombiner(), Arrays.asList(3, 3, 5), 45); - checkCombineFn(new NullCombiner(), Arrays.asList(null, 3, 5), 30); - checkCombineFn(new NullCombiner(), Arrays.asList(3, 3, null), 18); - checkCombineFn(new NullCombiner(), Arrays.asList(null, 3, null), 12); - checkCombineFn(new NullCombiner(), Arrays.asList(null, null, null), 8); + testCombineFn(new NullCombiner(), Arrays.asList(3, 3, 5), 45); + testCombineFn(new NullCombiner(), Arrays.asList(null, 3, 5), 30); + testCombineFn(new NullCombiner(), Arrays.asList(3, 3, null), 18); + testCombineFn(new NullCombiner(), Arrays.asList(null, 3, null), 12); + testCombineFn(new NullCombiner(), Arrays.asList(null, null, null), 8); } private static final class TestProdInt extends Combine.BinaryCombineIntegerFn { diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/MaxTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/MaxTest.java index 52043e18597c5..a298a5e79df1c 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/MaxTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/MaxTest.java @@ -17,7 +17,7 @@ */ package org.apache.beam.sdk.transforms; -import static org.apache.beam.sdk.TestUtils.checkCombineFn; +import static org.apache.beam.sdk.testing.CombineFnTester.testCombineFn; import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasDisplayItem; import static org.hamcrest.MatcherAssert.assertThat; import static org.junit.Assert.assertEquals; @@ -45,7 +45,7 @@ public void testMaxGetNames() { @Test public void testMaxIntegerFn() { - checkCombineFn( + testCombineFn( Max.ofIntegers(), Lists.newArrayList(1, 2, 3, 4), 4); @@ -53,7 +53,7 @@ public void testMaxIntegerFn() { @Test public void testMaxLongFn() { - checkCombineFn( + testCombineFn( Max.ofLongs(), Lists.newArrayList(1L, 2L, 3L, 4L), 4L); @@ -61,7 +61,7 @@ public void testMaxLongFn() { @Test public void testMaxDoubleFn() { - checkCombineFn( + testCombineFn( Max.ofDoubles(), Lists.newArrayList(1.0, 2.0, 3.0, 4.0), 4.0); diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/MeanTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/MeanTest.java index 79ebc25864023..e1381353b5c9f 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/MeanTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/MeanTest.java @@ -17,7 +17,7 @@ */ package org.apache.beam.sdk.transforms; -import static org.apache.beam.sdk.TestUtils.checkCombineFn; +import static org.apache.beam.sdk.testing.CombineFnTester.testCombineFn; import static org.junit.Assert.assertEquals; import com.google.common.collect.Lists; @@ -64,7 +64,7 @@ public void testCountSumCoderSerializable() throws Exception { @Test public void testMeanFn() throws Exception { - checkCombineFn( + testCombineFn( Mean.of(), Lists.newArrayList(1, 2, 3, 4), 2.5); diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/MinTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/MinTest.java index 1ece09b532f7f..a515b630af624 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/MinTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/MinTest.java @@ -18,7 +18,7 @@ package org.apache.beam.sdk.transforms; -import static org.apache.beam.sdk.TestUtils.checkCombineFn; +import static org.apache.beam.sdk.testing.CombineFnTester.testCombineFn; import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasDisplayItem; import static org.hamcrest.MatcherAssert.assertThat; import static org.junit.Assert.assertEquals; @@ -45,7 +45,7 @@ public void testMinGetNames() { } @Test public void testMinIntegerFn() { - checkCombineFn( + testCombineFn( Min.ofIntegers(), Lists.newArrayList(1, 2, 3, 4), 1); @@ -53,7 +53,7 @@ public void testMinIntegerFn() { @Test public void testMinLongFn() { - checkCombineFn( + testCombineFn( Min.ofLongs(), Lists.newArrayList(1L, 2L, 3L, 4L), 1L); @@ -61,7 +61,7 @@ public void testMinLongFn() { @Test public void testMinDoubleFn() { - checkCombineFn( + testCombineFn( Min.ofDoubles(), Lists.newArrayList(1.0, 2.0, 3.0, 4.0), 1.0); diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/SumTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/SumTest.java index 9d2c6f6722e3d..e5bf904ba0638 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/SumTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/SumTest.java @@ -17,7 +17,7 @@ */ package org.apache.beam.sdk.transforms; -import static org.apache.beam.sdk.TestUtils.checkCombineFn; +import static org.apache.beam.sdk.testing.CombineFnTester.testCombineFn; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNotEquals; @@ -51,7 +51,7 @@ public void testSumGetNames() { @Test public void testSumIntegerFn() { - checkCombineFn( + testCombineFn( Sum.ofIntegers(), Lists.newArrayList(1, 2, 3, 4), 10); @@ -59,7 +59,7 @@ public void testSumIntegerFn() { @Test public void testSumLongFn() { - checkCombineFn( + testCombineFn( Sum.ofLongs(), Lists.newArrayList(1L, 2L, 3L, 4L), 10L); @@ -67,7 +67,7 @@ public void testSumLongFn() { @Test public void testSumDoubleFn() { - checkCombineFn( + testCombineFn( Sum.ofDoubles(), Lists.newArrayList(1.0, 2.0, 3.0, 4.0), 10.0); From 18daed908d47aa2605507e7c91f42695dd8fae9f Mon Sep 17 00:00:00 2001 From: manuzhang Date: Thu, 27 Jul 2017 08:54:18 +0800 Subject: [PATCH 202/346] Override maven-enforcer-plugin for precommit tests --- examples/java/pom.xml | 22 ++++++++++++++++++++++ 1 file changed, 22 insertions(+) diff --git a/examples/java/pom.xml b/examples/java/pom.xml index bd18889024f14..6a1ce991ad65d 100644 --- a/examples/java/pom.xml +++ b/examples/java/pom.xml @@ -187,6 +187,28 @@ + + org.apache.maven.plugins + maven-enforcer-plugin + + + enforce + + enforce + + + + + 1.8 + + + [1.8,) + + + + + + org.apache.maven.plugins maven-failsafe-plugin From bf15483566ea759d2bc4af14173927402277738b Mon Sep 17 00:00:00 2001 From: manuzhang Date: Thu, 27 Jul 2017 12:57:28 +0800 Subject: [PATCH 203/346] Remove gearpump-runner in precommit tests --- ...ob_beam_PreCommit_Java_MavenInstall.groovy | 2 +- examples/java/pom.xml | 30 ------------------- 2 files changed, 1 insertion(+), 31 deletions(-) diff --git a/.test-infra/jenkins/job_beam_PreCommit_Java_MavenInstall.groovy b/.test-infra/jenkins/job_beam_PreCommit_Java_MavenInstall.groovy index 7117f12c46e21..bc130ece20c05 100644 --- a/.test-infra/jenkins/job_beam_PreCommit_Java_MavenInstall.groovy +++ b/.test-infra/jenkins/job_beam_PreCommit_Java_MavenInstall.groovy @@ -41,5 +41,5 @@ mavenJob('beam_PreCommit_Java_MavenInstall') { common_job_properties.setPreCommit(delegate, '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,gearpump-runner -DrepoToken=$COVERALLS_REPO_TOKEN -DpullRequest=$ghprbPullId help:effective-settings clean install coveralls:report') + goals('-B -e -Prelease,include-runners,jenkins-precommit,direct-runner,dataflow-runner,spark-runner,flink-runner,apex-runner -DrepoToken=$COVERALLS_REPO_TOKEN -DpullRequest=$ghprbPullId help:effective-settings clean install coveralls:report') } diff --git a/examples/java/pom.xml b/examples/java/pom.xml index 6a1ce991ad65d..ae64a79340d0b 100644 --- a/examples/java/pom.xml +++ b/examples/java/pom.xml @@ -155,9 +155,6 @@ jenkins-precommit - - [1.8,) - false true @@ -178,37 +175,10 @@ ${apex.kryo.version} runtime - - org.apache.beam - beam-runners-gearpump - runtime - - - org.apache.maven.plugins - maven-enforcer-plugin - - - enforce - - enforce - - - - - 1.8 - - - [1.8,) - - - - - - org.apache.maven.plugins maven-failsafe-plugin From 86e335595f1274248835e5715229bc67d4233fb3 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Isma=C3=ABl=20Mej=C3=ADa?= Date: Thu, 27 Jul 2017 10:57:15 +0200 Subject: [PATCH 204/346] [BEAM-1984] Fix scope for dependencies needed only for test purposes --- runners/spark/pom.xml | 1 + sdks/java/io/amqp/pom.xml | 1 + sdks/java/io/elasticsearch/pom.xml | 16 ++++++---------- .../elasticsearch/ElasticsearchTestDataSet.java | 2 +- sdks/java/io/hcatalog/pom.xml | 13 +++++++------ sdks/java/io/jdbc/pom.xml | 10 +++++----- 6 files changed, 21 insertions(+), 22 deletions(-) diff --git a/runners/spark/pom.xml b/runners/spark/pom.xml index 35e933b470f40..e823060119f08 100644 --- a/runners/spark/pom.xml +++ b/runners/spark/pom.xml @@ -200,6 +200,7 @@ org.apache.avro avro + test org.slf4j diff --git a/sdks/java/io/amqp/pom.xml b/sdks/java/io/amqp/pom.xml index 8da94483bff52..4369bb8250f97 100644 --- a/sdks/java/io/amqp/pom.xml +++ b/sdks/java/io/amqp/pom.xml @@ -39,6 +39,7 @@ org.slf4j slf4j-api + test diff --git a/sdks/java/io/elasticsearch/pom.xml b/sdks/java/io/elasticsearch/pom.xml index e0a7f21e0f23a..a021420fc56ad 100644 --- a/sdks/java/io/elasticsearch/pom.xml +++ b/sdks/java/io/elasticsearch/pom.xml @@ -36,11 +36,6 @@ beam-sdks-java-core - - org.slf4j - slf4j-api - - com.google.guava guava @@ -86,11 +81,6 @@ 4.5.2 - - joda-time - joda-time - - com.google.auto.value @@ -131,6 +121,12 @@ test + + org.slf4j + slf4j-api + test + + org.slf4j slf4j-jdk14 diff --git a/sdks/java/io/elasticsearch/src/test/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchTestDataSet.java b/sdks/java/io/elasticsearch/src/test/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchTestDataSet.java index 2a2dbe902defe..e2c291bb68525 100644 --- a/sdks/java/io/elasticsearch/src/test/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchTestDataSet.java +++ b/sdks/java/io/elasticsearch/src/test/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchTestDataSet.java @@ -37,7 +37,7 @@ public class ElasticsearchTestDataSet { public static final long NUM_DOCS = 60000; public static final int AVERAGE_DOC_SIZE = 25; public static final int MAX_DOC_SIZE = 35; - private static String writeIndex = ES_INDEX + org.joda.time.Instant.now().getMillis(); + private static String writeIndex = ES_INDEX + System.currentTimeMillis(); /** * Use this to create the index for reading before IT read tests. diff --git a/sdks/java/io/hcatalog/pom.xml b/sdks/java/io/hcatalog/pom.xml index 2aa661ef7edfb..52eae345dbbd4 100644 --- a/sdks/java/io/hcatalog/pom.xml +++ b/sdks/java/io/hcatalog/pom.xml @@ -70,12 +70,6 @@ - - commons-io - commons-io - ${apache.commons.version} - - org.slf4j slf4j-api @@ -132,6 +126,13 @@ ${hive.version} + + commons-io + commons-io + ${apache.commons.version} + test + + junit junit diff --git a/sdks/java/io/jdbc/pom.xml b/sdks/java/io/jdbc/pom.xml index 357ddc08b7d23..c559ad4aaf554 100644 --- a/sdks/java/io/jdbc/pom.xml +++ b/sdks/java/io/jdbc/pom.xml @@ -256,11 +256,6 @@ beam-sdks-java-core - - org.slf4j - slf4j-api - - com.google.guava guava @@ -318,6 +313,11 @@ hamcrest-all test + + org.slf4j + slf4j-api + test + org.slf4j slf4j-jdk14 From f61e797fbb5cf16942199c47bfff3ad84739700c Mon Sep 17 00:00:00 2001 From: Etienne Chauchot Date: Wed, 5 Jul 2017 11:35:59 +0200 Subject: [PATCH 205/346] [BEAM-2541] Check Elasticsearch backend version when the pipeline is run not when it is constructed --- .../sdk/io/elasticsearch/ElasticsearchIO.java | 40 +++++++++---------- .../ElasticsearchTestDataSet.java | 11 ++--- 2 files changed, 23 insertions(+), 28 deletions(-) diff --git a/sdks/java/io/elasticsearch/src/main/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIO.java b/sdks/java/io/elasticsearch/src/main/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIO.java index 4d7688772a00c..8e6e25371c639 100644 --- a/sdks/java/io/elasticsearch/src/main/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIO.java +++ b/sdks/java/io/elasticsearch/src/main/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIO.java @@ -183,10 +183,8 @@ abstract static class Builder { * @param index the index toward which the requests will be issued * @param type the document type toward which the requests will be issued * @return the connection configuration object - * @throws IOException when it fails to connect to Elasticsearch */ - public static ConnectionConfiguration create(String[] addresses, String index, String type) - throws IOException { + public static ConnectionConfiguration create(String[] addresses, String index, String type){ checkArgument( addresses != null, "ConnectionConfiguration.create(addresses, index, type) called with null address"); @@ -206,25 +204,9 @@ public static ConnectionConfiguration create(String[] addresses, String index, S .setIndex(index) .setType(type) .build(); - checkVersion(connectionConfiguration); return connectionConfiguration; } - private static void checkVersion(ConnectionConfiguration connectionConfiguration) - throws IOException { - RestClient restClient = connectionConfiguration.createClient(); - Response response = restClient.performRequest("GET", "", new BasicHeader("", "")); - JsonNode jsonNode = parseResponse(response); - String version = jsonNode.path("version").path("number").asText(); - boolean version2x = version.startsWith("2."); - restClient.close(); - checkArgument( - version2x, - "ConnectionConfiguration.create(addresses, index, type): " - + "the Elasticsearch version to connect to is different of 2.x. " - + "This version of the ElasticsearchIO is only compatible with Elasticsearch v2.x"); - } - /** * If Elasticsearch authentication is enabled, provide the username. * @@ -398,10 +380,12 @@ public PCollection expand(PBegin input) { @Override public void validate(PipelineOptions options) { + ConnectionConfiguration connectionConfiguration = getConnectionConfiguration(); checkState( - getConnectionConfiguration() != null, + connectionConfiguration != null, "ElasticsearchIO.read() requires a connection configuration" + " to be set via withConnectionConfiguration(configuration)"); + checkVersion(connectionConfiguration); } @Override @@ -715,10 +699,12 @@ public Write withMaxBatchSizeBytes(long batchSizeBytes) { @Override public void validate(PipelineOptions options) { + ConnectionConfiguration connectionConfiguration = getConnectionConfiguration(); checkState( - getConnectionConfiguration() != null, + connectionConfiguration != null, "ElasticsearchIO.write() requires a connection configuration" + " to be set via withConnectionConfiguration(configuration)"); + checkVersion(connectionConfiguration); } @Override @@ -828,4 +814,16 @@ public void closeClient() throws Exception { } } } + private static void checkVersion(ConnectionConfiguration connectionConfiguration){ + try (RestClient restClient = connectionConfiguration.createClient()) { + Response response = restClient.performRequest("GET", "", new BasicHeader("", "")); + JsonNode jsonNode = parseResponse(response); + String version = jsonNode.path("version").path("number").asText(); + boolean version2x = version.startsWith("2."); + checkArgument(version2x, "The Elasticsearch version to connect to is different of 2.x. " + + "This version of the ElasticsearchIO is only compatible with Elasticsearch v2.x"); + } catch (IOException e) { + throw new IllegalArgumentException("Cannot check Elasticsearch version"); + } + } } diff --git a/sdks/java/io/elasticsearch/src/test/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchTestDataSet.java b/sdks/java/io/elasticsearch/src/test/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchTestDataSet.java index e2c291bb68525..a6e1cc09a855e 100644 --- a/sdks/java/io/elasticsearch/src/test/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchTestDataSet.java +++ b/sdks/java/io/elasticsearch/src/test/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchTestDataSet.java @@ -18,7 +18,6 @@ package org.apache.beam.sdk.io.elasticsearch; -import java.io.IOException; import org.apache.beam.sdk.io.common.IOTestPipelineOptions; import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.elasticsearch.client.RestClient; @@ -37,7 +36,7 @@ public class ElasticsearchTestDataSet { public static final long NUM_DOCS = 60000; public static final int AVERAGE_DOC_SIZE = 25; public static final int MAX_DOC_SIZE = 35; - private static String writeIndex = ES_INDEX + System.currentTimeMillis(); + private static final String writeIndex = ES_INDEX + System.currentTimeMillis(); /** * Use this to create the index for reading before IT read tests. @@ -63,17 +62,15 @@ public static void main(String[] args) throws Exception { } private static void createAndPopulateReadIndex(IOTestPipelineOptions options) throws Exception { - RestClient restClient = getConnectionConfiguration(options, ReadOrWrite.READ).createClient(); // automatically creates the index and insert docs - try { + try (RestClient restClient = getConnectionConfiguration(options, ReadOrWrite.READ) + .createClient()) { ElasticSearchIOTestUtils.insertTestDocuments(ES_INDEX, ES_TYPE, NUM_DOCS, restClient); - } finally { - restClient.close(); } } static ElasticsearchIO.ConnectionConfiguration getConnectionConfiguration( - IOTestPipelineOptions options, ReadOrWrite rOw) throws IOException { + IOTestPipelineOptions options, ReadOrWrite rOw){ ElasticsearchIO.ConnectionConfiguration connectionConfiguration = ElasticsearchIO.ConnectionConfiguration.create( new String[] { From 0249da517c84e6ae609a95f20b37c50c0cc46d11 Mon Sep 17 00:00:00 2001 From: Eugene Kirpichov Date: Wed, 26 Jul 2017 10:45:09 -0700 Subject: [PATCH 206/346] Fixes BigQueryIO.Write javadoc for to(ValueInSingleWindow) --- .../org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java | 9 ++++++--- 1 file changed, 6 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 cc288e14f2b88..7ca4ce2d2cc7d 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 @@ -185,13 +185,16 @@ * quotes.apply(Window.into(CalendarWindows.days(1))) * .apply(BigQueryIO.writeTableRows() * .withSchema(schema) - * .to(new SerializableFunction() { - * public String apply(ValueInSingleWindow value) { + * .to(new SerializableFunction, TableDestination>() { + * public TableDestination apply(ValueInSingleWindow value) { * // The cast below is safe because CalendarWindows.days(1) produces IntervalWindows. * String dayString = DateTimeFormat.forPattern("yyyy_MM_dd") * .withZone(DateTimeZone.UTC) * .print(((IntervalWindow) value.getWindow()).start()); - * return "my-project:output.output_table_" + dayString; + * return new TableDestination( + * "my-project:output.output_table_" + dayString, // Table spec + * "Output for day " + dayString // Table description + * ); * } * })); * } From f515c22d6bd583cb97fb33c6537c1ecc6513141a Mon Sep 17 00:00:00 2001 From: Eugene Kirpichov Date: Wed, 19 Jul 2017 11:50:58 -0700 Subject: [PATCH 207/346] [BEAM-2640] Introduces Create.ofProvider(ValueProvider) I also converted DatastoreV1 to use this overload, and, as an exercise, added a withQuery(ValueProvider) overload to JdbcIO. --- .../beam/sdk/options/ValueProvider.java | 6 +++ .../beam/sdk/options/ValueProviders.java | 2 +- .../apache/beam/sdk/testing/TestPipeline.java | 7 ++- .../apache/beam/sdk/transforms/Create.java | 42 +++++++++++++++ .../beam/sdk/transforms/CreateTest.java | 52 +++++++++++++++++++ .../sdk/io/gcp/datastore/DatastoreV1.java | 19 ++++--- .../org/apache/beam/sdk/io/jdbc/JdbcIO.java | 13 +++-- 7 files changed, 126 insertions(+), 15 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 c7f1e094559a2..94187a9349a52 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 @@ -41,13 +41,19 @@ import java.util.concurrent.ConcurrentHashMap; import javax.annotation.Nullable; import org.apache.beam.sdk.annotations.Internal; +import org.apache.beam.sdk.transforms.Create; import org.apache.beam.sdk.transforms.SerializableFunction; +import org.apache.beam.sdk.values.PCollection; /** * A {@link ValueProvider} abstracts the notion of fetching a value that may or may not be currently * available. * *

              This can be used to parameterize transforms that only read values in at runtime, for example. + * + *

              A common task is to create a {@link PCollection} containing the value of this + * {@link ValueProvider} regardless of whether it's accessible at construction time or not. + * For that, use {@link Create#ofProvider}. */ @JsonSerialize(using = ValueProvider.Serializer.class) @JsonDeserialize(using = ValueProvider.Deserializer.class) diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/options/ValueProviders.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/options/ValueProviders.java index 1cc46fe4a83fe..bc479a2a533c3 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/options/ValueProviders.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/options/ValueProviders.java @@ -28,7 +28,7 @@ /** * Utilities for working with the {@link ValueProvider} interface. */ -class ValueProviders { +public class ValueProviders { private ValueProviders() {} /** 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 34f1c83443ba6..b67b14f28f5bc 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 @@ -328,6 +328,11 @@ public void evaluate() throws Throwable { * testing. */ public PipelineResult run() { + return run(getOptions()); + } + + /** Like {@link #run} but with the given potentially modified options. */ + public PipelineResult run(PipelineOptions options) { checkState( enforcement.isPresent(), "Is your TestPipeline declaration missing a @Rule annotation? Usage: " @@ -336,7 +341,7 @@ public PipelineResult run() { final PipelineResult pipelineResult; try { enforcement.get().beforePipelineExecution(); - pipelineResult = super.run(); + pipelineResult = super.run(options); verifyPAssertsSucceeded(this, pipelineResult); } catch (RuntimeException exc) { Throwable cause = exc.getCause(); 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 7af8fb8fbaa40..09e12efe10b49 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 @@ -18,6 +18,7 @@ package org.apache.beam.sdk.transforms; 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; @@ -52,6 +53,7 @@ import org.apache.beam.sdk.io.OffsetBasedSource.OffsetBasedReader; import org.apache.beam.sdk.io.Read; import org.apache.beam.sdk.options.PipelineOptions; +import org.apache.beam.sdk.options.ValueProvider; import org.apache.beam.sdk.util.CoderUtils; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PBegin; @@ -199,6 +201,14 @@ public static Values> of(Map elems) { return of(kvs); } + /** + * Returns an {@link OfValueProvider} transform that produces a {@link PCollection} + * of a single element provided by the given {@link ValueProvider}. + */ + public static OfValueProvider ofProvider(ValueProvider provider, Coder coder) { + return new OfValueProvider<>(provider, coder); + } + /** * Returns a new {@link Create.TimestampedValues} transform that produces a * {@link PCollection} containing the elements of the provided {@code Iterable} @@ -485,6 +495,38 @@ protected boolean advanceImpl() throws IOException { ///////////////////////////////////////////////////////////////////////////// + /** Implementation of {@link #ofProvider}. */ + public static class OfValueProvider extends PTransform> { + private final ValueProvider provider; + private final Coder coder; + + private OfValueProvider(ValueProvider provider, Coder coder) { + this.provider = checkNotNull(provider, "provider"); + this.coder = checkNotNull(coder, "coder"); + } + + @Override + public PCollection expand(PBegin input) { + if (provider.isAccessible()) { + Values values = Create.of(provider.get()); + return input.apply(values.withCoder(coder)); + } + return input + .apply(Create.of((Void) null)) + .apply( + MapElements.via( + new SimpleFunction() { + @Override + public T apply(Void input) { + return provider.get(); + } + })) + .setCoder(coder); + } + } + + ///////////////////////////////////////////////////////////////////////////// + /** * A {@code PTransform} that creates a {@code PCollection} whose elements have * associated timestamps. 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 a05d31c29d4db..1e7ce2d6454ab 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 @@ -25,7 +25,9 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertThat; +import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; import com.google.common.collect.Lists; import java.io.IOException; import java.io.InputStream; @@ -47,6 +49,10 @@ import org.apache.beam.sdk.io.BoundedSource; 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.options.ValueProvider.NestedValueProvider; +import org.apache.beam.sdk.options.ValueProvider.StaticValueProvider; +import org.apache.beam.sdk.options.ValueProviders; import org.apache.beam.sdk.testing.NeedsRunner; import org.apache.beam.sdk.testing.PAssert; import org.apache.beam.sdk.testing.SourceTestUtils; @@ -54,6 +60,7 @@ import org.apache.beam.sdk.testing.ValidatesRunner; import org.apache.beam.sdk.transforms.Create.Values.CreateSource; import org.apache.beam.sdk.util.SerializableUtils; +import org.apache.beam.sdk.util.common.ReflectHelpers; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PBegin; import org.apache.beam.sdk.values.PCollection; @@ -353,6 +360,51 @@ public void testCreateWithKVVoidType() throws Exception { p.run(); } + private static final ObjectMapper MAPPER = new ObjectMapper().registerModules( + ObjectMapper.findModules(ReflectHelpers.findClassLoader())); + + public interface CreateOfProviderOptions extends PipelineOptions { + ValueProvider getFoo(); + void setFoo(ValueProvider value); + } + + @Test + @Category(ValidatesRunner.class) + public void testCreateOfProvider() throws Exception { + PAssert.that( + p.apply( + "Static", Create.ofProvider(StaticValueProvider.of("foo"), StringUtf8Coder.of()))) + .containsInAnyOrder("foo"); + PAssert.that( + p.apply( + "Static nested", + Create.ofProvider( + NestedValueProvider.of( + StaticValueProvider.of("foo"), + new SerializableFunction() { + @Override + public String apply(String input) { + return input + "bar"; + } + }), + StringUtf8Coder.of()))) + .containsInAnyOrder("foobar"); + CreateOfProviderOptions submitOptions = + p.getOptions().as(CreateOfProviderOptions.class); + PAssert.that( + p.apply("Runtime", Create.ofProvider(submitOptions.getFoo(), StringUtf8Coder.of()))) + .containsInAnyOrder("runtime foo"); + + String serializedOptions = MAPPER.writeValueAsString(p.getOptions()); + String runnerString = ValueProviders.updateSerializedOptions( + serializedOptions, ImmutableMap.of("foo", "runtime foo")); + CreateOfProviderOptions runtimeOptions = + MAPPER.readValue(runnerString, PipelineOptions.class).as(CreateOfProviderOptions.class); + + p.run(runtimeOptions); + } + + @Test public void testCreateGetName() { assertEquals("Create.Values", Create.of(1, 2, 3).getName()); 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 1ed643014a739..7e40db42e8081 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 @@ -71,7 +71,7 @@ import org.apache.beam.sdk.PipelineRunner; import org.apache.beam.sdk.annotations.Experimental; import org.apache.beam.sdk.annotations.Experimental.Kind; -import org.apache.beam.sdk.coders.SerializableCoder; +import org.apache.beam.sdk.coders.StringUtf8Coder; import org.apache.beam.sdk.extensions.gcp.options.GcpOptions; import org.apache.beam.sdk.metrics.Counter; import org.apache.beam.sdk.metrics.Metrics; @@ -99,7 +99,6 @@ import org.apache.beam.sdk.values.PBegin; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PDone; -import org.apache.beam.sdk.values.TypeDescriptor; import org.joda.time.Duration; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -611,10 +610,10 @@ public PCollection expand(PBegin input) { if (getQuery() != null) { inputQuery = input.apply(Create.of(getQuery())); } else { - inputQuery = input - .apply(Create.of(getLiteralGqlQuery()) - .withCoder(SerializableCoder.of(new TypeDescriptor>() {}))) - .apply(ParDo.of(new GqlQueryTranslateFn(v1Options))); + inputQuery = + input + .apply(Create.ofProvider(getLiteralGqlQuery(), StringUtf8Coder.of())) + .apply(ParDo.of(new GqlQueryTranslateFn(v1Options))); } PCollection> splitQueries = inputQuery @@ -730,7 +729,7 @@ public void populateDisplayData(DisplayData.Builder builder) { /** * A DoFn that translates a Cloud Datastore gql query string to {@code Query}. */ - static class GqlQueryTranslateFn extends DoFn, Query> { + static class GqlQueryTranslateFn extends DoFn { private final V1Options v1Options; private transient Datastore datastore; private final V1DatastoreFactory datastoreFactory; @@ -751,9 +750,9 @@ public void startBundle(StartBundleContext c) throws Exception { @ProcessElement public void processElement(ProcessContext c) throws Exception { - ValueProvider gqlQuery = c.element(); - LOG.info("User query: '{}'", gqlQuery.get()); - Query query = translateGqlQueryWithLimitCheck(gqlQuery.get(), datastore, + String gqlQuery = c.element(); + LOG.info("User query: '{}'", gqlQuery); + Query query = translateGqlQueryWithLimitCheck(gqlQuery, datastore, v1Options.getNamespace()); LOG.info("User gql query translated to Query({})", query); c.output(query); 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 bf73dbef63f47..51f34aef6e3c8 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 @@ -31,7 +31,9 @@ import javax.sql.DataSource; import org.apache.beam.sdk.annotations.Experimental; 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.options.ValueProvider; import org.apache.beam.sdk.transforms.Create; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.Flatten; @@ -272,7 +274,7 @@ public interface StatementPreparator extends Serializable { @AutoValue public abstract static class Read extends PTransform> { @Nullable abstract DataSourceConfiguration getDataSourceConfiguration(); - @Nullable abstract String getQuery(); + @Nullable abstract ValueProvider getQuery(); @Nullable abstract StatementPreparator getStatementPreparator(); @Nullable abstract RowMapper getRowMapper(); @Nullable abstract Coder getCoder(); @@ -282,7 +284,7 @@ public abstract static class Read extends PTransform> @AutoValue.Builder abstract static class Builder { abstract Builder setDataSourceConfiguration(DataSourceConfiguration config); - abstract Builder setQuery(String query); + abstract Builder setQuery(ValueProvider query); abstract Builder setStatementPreparator(StatementPreparator statementPreparator); abstract Builder setRowMapper(RowMapper rowMapper); abstract Builder setCoder(Coder coder); @@ -296,6 +298,11 @@ public Read withDataSourceConfiguration(DataSourceConfiguration configuration } public Read withQuery(String query) { + checkArgument(query != null, "JdbcIO.read().withQuery(query) called with null query"); + return withQuery(ValueProvider.StaticValueProvider.of(query)); + } + + public Read withQuery(ValueProvider query) { checkArgument(query != null, "JdbcIO.read().withQuery(query) called with null query"); return toBuilder().setQuery(query).build(); } @@ -321,7 +328,7 @@ public Read withCoder(Coder coder) { @Override public PCollection expand(PBegin input) { return input - .apply(Create.of(getQuery())) + .apply(Create.ofProvider(getQuery(), StringUtf8Coder.of())) .apply(ParDo.of(new ReadFn<>(this))).setCoder(getCoder()) .apply(ParDo.of(new DoFn>() { private Random random; From 476efa65adc32da56c0ee4a3a85cddfb9e239b09 Mon Sep 17 00:00:00 2001 From: Eugene Kirpichov Date: Wed, 19 Jul 2017 11:51:33 -0700 Subject: [PATCH 208/346] [BEAM-2641] Introduces TextIO.read().withHintMatchesManyFiles() In that case it expands to TextIO.readAll(). --- .../java/org/apache/beam/sdk/io/TextIO.java | 46 ++++++++++++++----- .../apache/beam/sdk/io/TextIOReadTest.java | 20 ++++++-- .../beam/sdk/transforms/CreateTest.java | 1 + 3 files changed, 53 insertions(+), 14 deletions(-) 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 73040daafedb0..765a84243d635 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 @@ -34,11 +34,11 @@ import org.apache.beam.sdk.io.FileBasedSink.DynamicDestinations; import org.apache.beam.sdk.io.FileBasedSink.FilenamePolicy; import org.apache.beam.sdk.io.FileBasedSink.WritableByteChannelFactory; -import org.apache.beam.sdk.io.Read.Bounded; import org.apache.beam.sdk.io.fs.ResourceId; 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.Create; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.SerializableFunction; import org.apache.beam.sdk.transforms.SerializableFunctions; @@ -68,6 +68,11 @@ * PCollection lines = p.apply(TextIO.read().from("/local/path/to/file.txt")); * } * + *

              If it is known that the filepattern will match a very large number of files (e.g. tens of + * thousands or more), use {@link Read#withHintMatchesManyFiles} for better performance and + * scalability. Note that it may decrease performance if the filepattern matches only a small + * number of files. + * *

              Example 2: reading a PCollection of filenames. * *

              {@code
              @@ -143,7 +148,10 @@ public class TextIO {
                  * {@link PCollection} containing one element for each line of the input files.
                  */
                 public static Read read() {
              -    return new AutoValue_TextIO_Read.Builder().setCompressionType(CompressionType.AUTO).build();
              +    return new AutoValue_TextIO_Read.Builder()
              +        .setCompressionType(CompressionType.AUTO)
              +        .setHintMatchesManyFiles(false)
              +        .build();
                 }
               
                 /**
              @@ -206,6 +214,7 @@ public static  TypedWrite writeCustomType(SerializableFunction
                 public abstract static class Read extends PTransform> {
                   @Nullable abstract ValueProvider getFilepattern();
                   abstract CompressionType getCompressionType();
              +    abstract boolean getHintMatchesManyFiles();
               
                   abstract Builder toBuilder();
               
              @@ -213,6 +222,7 @@ public abstract static class Read extends PTransform
                   abstract static class Builder {
                     abstract Builder setFilepattern(ValueProvider filepattern);
                     abstract Builder setCompressionType(CompressionType compressionType);
              +      abstract Builder setHintMatchesManyFiles(boolean hintManyFiles);
               
                     abstract Read build();
                   }
              @@ -226,6 +236,9 @@ abstract static class Builder {
                    *
                    * 

              Standard Java * Filesystem glob patterns ("*", "?", "[..]") are supported. + * + *

              If it is known that the filepattern will match a very large number of files (at least tens + * of thousands), use {@link #withHintMatchesManyFiles} for better performance and scalability. */ public Read from(String filepattern) { checkNotNull(filepattern, "Filepattern cannot be empty."); @@ -248,17 +261,28 @@ public Read withCompressionType(TextIO.CompressionType compressionType) { return toBuilder().setCompressionType(compressionType).build(); } + /** + * Hints that the filepattern specified in {@link #from(String)} matches a very large number of + * files. + * + *

              This hint may cause a runner to execute the transform differently, in a way that improves + * performance for this case, but it may worsen performance if the filepattern matches only + * a small number of files (e.g., in a runner that supports dynamic work rebalancing, it will + * happen less efficiently within individual files). + */ + public Read withHintMatchesManyFiles() { + return toBuilder().setHintMatchesManyFiles(true).build(); + } + @Override public PCollection expand(PBegin input) { - if (getFilepattern() == null) { - throw new IllegalStateException("need to set the filepattern of a TextIO.Read transform"); - } - - final Bounded read = org.apache.beam.sdk.io.Read.from(getSource()); - PCollection pcol = input.getPipeline().apply("Read", read); - // Honor the default output coder that would have been used by this PTransform. - pcol.setCoder(getDefaultOutputCoder()); - return pcol; + checkNotNull(getFilepattern(), "need to set the filepattern of a TextIO.Read transform"); + return getHintMatchesManyFiles() + ? input + .apply( + "Create filepattern", Create.ofProvider(getFilepattern(), StringUtf8Coder.of())) + .apply(readAll().withCompressionType(getCompressionType())) + : input.apply("Read", org.apache.beam.sdk.io.Read.from(getSource())); } // Helper to create a source specific to the requested compression type. diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/TextIOReadTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/TextIOReadTest.java index e733010acf3d4..8ad60304389ee 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/TextIOReadTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/TextIOReadTest.java @@ -290,9 +290,16 @@ private static void writeToStreamAndClose(List lines, OutputStream outpu } /** - * Helper method that runs TextIO.read().from(filename).withCompressionType(compressionType) and - * TextIO.readAll().withCompressionType(compressionType) applied to the single filename, - * and asserts that the results match the given expected output. + * Helper method that runs a variety of ways to read a single file using TextIO + * and checks that they all match the given expected output. + * + *

              The transforms being verified are: + *

                + *
              • TextIO.read().from(filename).withCompressionType(compressionType) + *
              • TextIO.read().from(filename).withCompressionType(compressionType) + * .withHintMatchesManyFiles() + *
              • TextIO.readAll().withCompressionType(compressionType) + *
              and */ private void assertReadingCompressedFileMatchesExpected( File file, CompressionType compressionType, List expected) { @@ -300,10 +307,17 @@ private void assertReadingCompressedFileMatchesExpected( int thisUniquifier = ++uniquifier; TextIO.Read read = TextIO.read().from(file.getPath()).withCompressionType(compressionType); + PAssert.that( p.apply("Read_" + file + "_" + compressionType.toString() + "_" + thisUniquifier, read)) .containsInAnyOrder(expected); + PAssert.that( + p.apply( + "Read_" + file + "_" + compressionType.toString() + "_many" + "_" + thisUniquifier, + read.withHintMatchesManyFiles())) + .containsInAnyOrder(expected); + TextIO.ReadAll readAll = TextIO.readAll().withCompressionType(compressionType).withDesiredBundleSizeBytes(10); PAssert.that( 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 1e7ce2d6454ab..6a682ef658f75 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 @@ -363,6 +363,7 @@ public void testCreateWithKVVoidType() throws Exception { private static final ObjectMapper MAPPER = new ObjectMapper().registerModules( ObjectMapper.findModules(ReflectHelpers.findClassLoader())); + /** Testing options for {@link #testCreateOfProvider()}. */ public interface CreateOfProviderOptions extends PipelineOptions { ValueProvider getFoo(); void setFoo(ValueProvider value); From e3335773018a5644eb07b0f9b2b0619fbab19ec5 Mon Sep 17 00:00:00 2001 From: Eugene Kirpichov Date: Tue, 25 Jul 2017 18:13:06 -0700 Subject: [PATCH 209/346] Adds AvroIO.read().withHintMatchesManyFiles() --- .../java/org/apache/beam/sdk/io/AvroIO.java | 53 ++++++++++++++++--- .../org/apache/beam/sdk/io/AvroIOTest.java | 13 ++++- 2 files changed, 57 insertions(+), 9 deletions(-) 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 bc7fecbd2fb1d..018b84f68ad17 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 @@ -36,6 +36,7 @@ import org.apache.beam.sdk.annotations.Experimental.Kind; import org.apache.beam.sdk.coders.AvroCoder; import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.coders.StringUtf8Coder; import org.apache.beam.sdk.coders.VoidCoder; import org.apache.beam.sdk.io.FileBasedSink.DynamicDestinations; import org.apache.beam.sdk.io.FileBasedSink.FilenamePolicy; @@ -43,6 +44,7 @@ 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.Create; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.SerializableFunction; import org.apache.beam.sdk.transforms.SerializableFunctions; @@ -84,6 +86,11 @@ * .from("gs://my_bucket/path/to/records-*.avro")); * }
              * + *

              If it is known that the filepattern will match a very large number of files (e.g. tens of + * thousands or more), use {@link Read#withHintMatchesManyFiles} for better performance and + * scalability. Note that it may decrease performance if the filepattern matches only a small + * number of files. + * *

              Reading from a {@link PCollection} of filepatterns: * *

              {@code
              @@ -147,6 +154,7 @@ public static  Read read(Class recordClass) {
                   return new AutoValue_AvroIO_Read.Builder()
                       .setRecordClass(recordClass)
                       .setSchema(ReflectData.get().getSchema(recordClass))
              +        .setHintMatchesManyFiles(false)
                       .build();
                 }
               
              @@ -167,6 +175,7 @@ public static Read readGenericRecords(Schema schema) {
                   return new AutoValue_AvroIO_Read.Builder()
                       .setRecordClass(GenericRecord.class)
                       .setSchema(schema)
              +        .setHintMatchesManyFiles(false)
                       .build();
                 }
               
              @@ -240,6 +249,7 @@ public abstract static class Read extends PTransform>
                   @Nullable abstract ValueProvider getFilepattern();
                   @Nullable abstract Class getRecordClass();
                   @Nullable abstract Schema getSchema();
              +    abstract boolean getHintMatchesManyFiles();
               
                   abstract Builder toBuilder();
               
              @@ -248,11 +258,17 @@ abstract static class Builder {
                     abstract Builder setFilepattern(ValueProvider filepattern);
                     abstract Builder setRecordClass(Class recordClass);
                     abstract Builder setSchema(Schema schema);
              +      abstract Builder setHintMatchesManyFiles(boolean hintManyFiles);
               
                     abstract Read build();
                   }
               
              -    /** Reads from the given filename or filepattern. */
              +    /**
              +     * Reads from the given filename or filepattern.
              +     *
              +     * 

              If it is known that the filepattern will match a very large number of files (at least tens + * of thousands), use {@link #withHintMatchesManyFiles} for better performance and scalability. + */ public Read from(ValueProvider filepattern) { return toBuilder().setFilepattern(filepattern).build(); } @@ -262,16 +278,39 @@ public Read from(String filepattern) { return from(StaticValueProvider.of(filepattern)); } + /** + * Hints that the filepattern specified in {@link #from(String)} matches a very large number of + * files. + * + *

              This hint may cause a runner to execute the transform differently, in a way that improves + * performance for this case, but it may worsen performance if the filepattern matches only a + * small number of files (e.g., in a runner that supports dynamic work rebalancing, it will + * happen less efficiently within individual files). + */ + public Read withHintMatchesManyFiles() { + return toBuilder().setHintMatchesManyFiles(true).build(); + } + @Override public PCollection expand(PBegin input) { checkNotNull(getFilepattern(), "filepattern"); checkNotNull(getSchema(), "schema"); - return input - .getPipeline() - .apply( - "Read", - org.apache.beam.sdk.io.Read.from( - createSource(getFilepattern(), getRecordClass(), getSchema()))); + if (getHintMatchesManyFiles()) { + ReadAll readAll = + (getRecordClass() == GenericRecord.class) + ? (ReadAll) readAllGenericRecords(getSchema()) + : readAll(getRecordClass()); + return input + .apply(Create.ofProvider(getFilepattern(), StringUtf8Coder.of())) + .apply(readAll); + } else { + return input + .getPipeline() + .apply( + "Read", + org.apache.beam.sdk.io.Read.from( + createSource(getFilepattern(), getRecordClass(), getSchema()))); + } } @Override 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 df5d26c49f063..90cd824d5de82 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 @@ -153,11 +153,20 @@ public void testAvroIOWriteAndReadASingleFile() throws Throwable { .apply(AvroIO.write(GenericClass.class).to(outputFile.getAbsolutePath()).withoutSharding()); writePipeline.run().waitUntilFinish(); - // Test both read() and readAll() + // Test the same data via read(), read().withHintMatchesManyFiles(), and readAll() + PAssert.that( + readPipeline.apply( + "Read", AvroIO.read(GenericClass.class).from(outputFile.getAbsolutePath()))) + .containsInAnyOrder(values); PAssert.that( - readPipeline.apply(AvroIO.read(GenericClass.class).from(outputFile.getAbsolutePath()))) + readPipeline.apply( + "Read withHintMatchesManyFiles", + AvroIO.read(GenericClass.class) + .from(outputFile.getAbsolutePath()) + .withHintMatchesManyFiles())) .containsInAnyOrder(values); PAssert.that( + "ReadAll", readPipeline .apply(Create.of(outputFile.getAbsolutePath())) .apply(AvroIO.readAll(GenericClass.class).withDesiredBundleSizeBytes(10))) From 0884be5269bdd8a943cbb7a3c3df8013c282129e Mon Sep 17 00:00:00 2001 From: Alex Filatov Date: Fri, 28 Jul 2017 12:07:07 +0300 Subject: [PATCH 210/346] Fix View.asMultimap javadoc. --- .../core/src/main/java/org/apache/beam/sdk/transforms/View.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 c94fad6a6aa57..57dccbcde062d 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 @@ -229,7 +229,7 @@ public static AsMap asMap() { *

                  * {@code
                  * PCollection> input = ... // maybe more than one occurrence of a some keys
              -   * PCollectionView> output = input.apply(View.asMultimap());
              +   * PCollectionView>> output = input.apply(View.asMultimap());
                  * }
              * *

              Currently, the resulting map is required to fit into memory. From d9fc36ae7ffd127b93a0b9a03b4ce8c86e5cc1fb Mon Sep 17 00:00:00 2001 From: Etienne Chauchot Date: Mon, 24 Jul 2017 12:03:24 +0200 Subject: [PATCH 211/346] Add missing displayData to ElasticsearchIO.Read --- .../org/apache/beam/sdk/io/elasticsearch/ElasticsearchIO.java | 2 ++ 1 file changed, 2 insertions(+) diff --git a/sdks/java/io/elasticsearch/src/main/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIO.java b/sdks/java/io/elasticsearch/src/main/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIO.java index 8e6e25371c639..acc7f2f0e5043 100644 --- a/sdks/java/io/elasticsearch/src/main/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIO.java +++ b/sdks/java/io/elasticsearch/src/main/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIO.java @@ -392,6 +392,8 @@ public void validate(PipelineOptions options) { public void populateDisplayData(DisplayData.Builder builder) { super.populateDisplayData(builder); builder.addIfNotNull(DisplayData.item("query", getQuery())); + builder.addIfNotNull(DisplayData.item("batchSize", getBatchSize())); + builder.addIfNotNull(DisplayData.item("scrollKeepalive", getScrollKeepalive())); getConnectionConfiguration().populateDisplayData(builder); } } From 62c922b3c2dc3163b180f972a7449cf5e6ac501a Mon Sep 17 00:00:00 2001 From: Eugene Kirpichov Date: Tue, 25 Jul 2017 17:18:16 -0700 Subject: [PATCH 212/346] Introduces TypeDescriptors.extractFromTypeParameters --- .../org/apache/beam/sdk/io/FileBasedSink.java | 24 ++-- .../beam/sdk/values/TypeDescriptor.java | 64 +++++++--- .../beam/sdk/values/TypeDescriptors.java | 118 +++++++++++++++++- .../beam/sdk/values/TypeDescriptorsTest.java | 49 ++++++++ .../io/gcp/bigquery/DynamicDestinations.java | 22 ++-- 5 files changed, 235 insertions(+), 42 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 995397500d53d..3bf5d5b303ac3 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 @@ -23,6 +23,7 @@ import static com.google.common.base.Preconditions.checkState; import static com.google.common.base.Verify.verifyNotNull; import static org.apache.beam.sdk.io.WriteFiles.UNKNOWN_SHARDNUM; +import static org.apache.beam.sdk.values.TypeDescriptors.extractFromTypeParameters; import com.google.common.annotations.VisibleForTesting; import com.google.common.base.MoreObjects; @@ -33,7 +34,6 @@ import java.io.InputStream; import java.io.OutputStream; import java.io.Serializable; -import java.lang.reflect.TypeVariable; import java.nio.channels.Channels; import java.nio.channels.WritableByteChannel; import java.util.ArrayList; @@ -75,6 +75,7 @@ import org.apache.beam.sdk.transforms.windowing.PaneInfo.PaneInfoCoder; import org.apache.beam.sdk.util.MimeTypes; import org.apache.beam.sdk.values.TypeDescriptor; +import org.apache.beam.sdk.values.TypeDescriptors.TypeVariableExtractor; import org.apache.commons.compress.compressors.bzip2.BZip2CompressorOutputStream; import org.apache.commons.compress.compressors.deflate.DeflateCompressorOutputStream; import org.joda.time.Instant; @@ -255,17 +256,16 @@ final Coder getDestinationCoderWithDefault(CoderRegistry registry) return destinationCoder; } // If dynamicDestinations doesn't provide a coder, try to find it in the coder registry. - // We must first use reflection to figure out what the type parameter is. - TypeDescriptor superDescriptor = - TypeDescriptor.of(getClass()).getSupertype(DynamicDestinations.class); - if (!superDescriptor.getRawType().equals(DynamicDestinations.class)) { - throw new AssertionError( - "Couldn't find the DynamicDestinations superclass of " + this.getClass()); - } - TypeVariable typeVariable = superDescriptor.getTypeParameter("DestinationT"); - @SuppressWarnings("unchecked") - TypeDescriptor descriptor = - (TypeDescriptor) superDescriptor.resolveType(typeVariable); + @Nullable TypeDescriptor descriptor = + extractFromTypeParameters( + this, + DynamicDestinations.class, + new TypeVariableExtractor< + DynamicDestinations, DestinationT>() {}); + checkArgument( + descriptor != null, + "Unable to infer a coder for DestinationT, " + + "please specify it explicitly by overriding getDestinationCoder()"); return registry.getCoder(descriptor); } } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/TypeDescriptor.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/TypeDescriptor.java index 14f2cb8eeb5b4..dd6a0fdbdf87c 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/TypeDescriptor.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/TypeDescriptor.java @@ -328,30 +328,64 @@ public Iterable getClasses() { } /** - * Returns a new {@code TypeDescriptor} where type variables represented by - * {@code typeParameter} are substituted by {@code typeDescriptor}. For example, it can be used to - * construct {@code Map} for any {@code K} and {@code V} type:

               {@code
              -   *   static  TypeDescriptor> mapOf(
              -   *       TypeDescriptor keyType, TypeDescriptor valueType) {
              -   *     return new TypeDescriptor>() {}
              -   *         .where(new TypeParameter() {}, keyType)
              -   *         .where(new TypeParameter() {}, valueType);
              -   *   }}
              + * Returns a new {@code TypeDescriptor} where the type variable represented by {@code + * typeParameter} are substituted by {@code typeDescriptor}. For example, it can be used to + * construct {@code Map} for any {@code K} and {@code V} type: + * + *
              {@code
              +   * static  TypeDescriptor> mapOf(
              +   *     TypeDescriptor keyType, TypeDescriptor valueType) {
              +   *   return new TypeDescriptor>() {}
              +   *       .where(new TypeParameter() {}, keyType)
              +   *       .where(new TypeParameter() {}, valueType);
              +   * }
              +   * }
              * * @param The parameter type * @param typeParameter the parameter type variable * @param typeDescriptor the actual type to substitute */ @SuppressWarnings("unchecked") - public TypeDescriptor where(TypeParameter typeParameter, - TypeDescriptor typeDescriptor) { - TypeResolver resolver = - new TypeResolver() - .where( - typeParameter.typeVariable, typeDescriptor.getType()); + public TypeDescriptor where( + TypeParameter typeParameter, TypeDescriptor typeDescriptor) { + return where(typeParameter.typeVariable, typeDescriptor.getType()); + } + + /** + * A more general form of {@link #where(TypeParameter, TypeDescriptor)} that returns a new {@code + * TypeDescriptor} by matching {@code formal} against {@code actual} to resolve type variables in + * the current {@link TypeDescriptor}. + */ + @SuppressWarnings("unchecked") + public TypeDescriptor where(Type formal, Type actual) { + TypeResolver resolver = new TypeResolver().where(formal, actual); return (TypeDescriptor) TypeDescriptor.of(resolver.resolveType(token.getType())); } + /** + * Returns whether this {@link TypeDescriptor} has any unresolved type parameters, as opposed to + * being a concrete type. + * + *

              For example: + *

              {@code
              +   *   TypeDescriptor.of(new ArrayList() {}.getClass()).hasUnresolvedTypeParameters()
              +   *     => false, because the anonymous class is instantiated with a concrete type
              +   *
              +   *   class TestUtils {
              +   *      ArrayList createTypeErasedList() {
              +   *       return new ArrayList() {};
              +   *     }
              +   *   }
              +   *
              +   *   TypeDescriptor.of(TestUtils.createTypeErasedList().getClass())
              +   *     => true, because the type variable T got type-erased and the anonymous ArrayList class
              +   *     is instantiated with an unresolved type variable T.
              +   * }
              + */ + public boolean hasUnresolvedParameters() { + return hasUnresolvedParameters(getType()); + } + @Override public String toString() { return token.toString(); diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/TypeDescriptors.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/TypeDescriptors.java index a4626c95e99f8..8207f06531ac6 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/TypeDescriptors.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/TypeDescriptors.java @@ -17,16 +17,20 @@ */ package org.apache.beam.sdk.values; +import java.lang.reflect.ParameterizedType; +import java.lang.reflect.Type; import java.math.BigDecimal; import java.math.BigInteger; import java.util.List; import java.util.Set; +import javax.annotation.Nullable; +import org.apache.beam.sdk.transforms.SerializableFunction; /** - * A utility class containing the Java primitives for - * {@link TypeDescriptor} equivalents. Also, has methods - * for classes that wrap Java primitives like {@link KV}, - * {@link Set}, {@link List}, and {@link Iterable}. + * A utility class for creating {@link TypeDescriptor} objects for different types, such as Java + * primitive types, containers and {@link KV KVs} of other {@link TypeDescriptor} objects, and + * extracting type variables of parameterized types (e.g. extracting the {@code OutputT} type + * variable of a {@code DoFn}). */ public class TypeDescriptors { /** @@ -286,4 +290,110 @@ public static TypeDescriptor nulls() { return typeDescriptor; } + + /** + * A helper interface for use with {@link #extractFromTypeParameters(Object, Class, + * TypeVariableExtractor)}. + */ + public interface TypeVariableExtractor {} + + /** + * Extracts a type from the actual type parameters of a parameterized class, subject to Java type + * erasure. The type to extract is specified in a way that is safe w.r.t. changing the type + * signature of the parameterized class, as opposed to specifying the name or index of a type + * variable. + * + *

              Example of use: + *

              {@code
              +   *   class Foo {
              +   *     private SerializableFunction fn;
              +   *
              +   *     TypeDescriptor inferBarTypeDescriptorFromFn() {
              +   *       return TypeDescriptors.extractFromTypeParameters(
              +   *         fn,
              +   *         SerializableFunction.class,
              +   *         // The actual type of "fn" is matched against the input type of the extractor,
              +   *         // and the obtained values of type variables of the superclass are substituted
              +   *         // into the output type of the extractor.
              +   *         new TypeVariableExtractor, BarT>() {});
              +   *     }
              +   *   }
              +   * }
              + * + * @param instance The object being analyzed + * @param supertype Parameterized superclass of interest + * @param extractor A class for specifying the type to extract from the supertype + * + * @return A {@link TypeDescriptor} for the actual value of the result type of the extractor, + * or {@code null} if the type was erased. + */ + @SuppressWarnings("unchecked") + @Nullable + public static TypeDescriptor extractFromTypeParameters( + T instance, Class supertype, TypeVariableExtractor extractor) { + return extractFromTypeParameters( + (TypeDescriptor) TypeDescriptor.of(instance.getClass()), supertype, extractor); + } + + /** + * Like {@link #extractFromTypeParameters(Object, Class, TypeVariableExtractor)}, but takes a + * {@link TypeDescriptor} of the instance being analyzed rather than the instance itself. + */ + @SuppressWarnings("unchecked") + @Nullable + public static TypeDescriptor extractFromTypeParameters( + TypeDescriptor type, Class supertype, TypeVariableExtractor extractor) { + // Get the type signature of the extractor, e.g. + // TypeVariableExtractor, BarT> + TypeDescriptor> extractorSupertype = + (TypeDescriptor>) + TypeDescriptor.of(extractor.getClass()).getSupertype(TypeVariableExtractor.class); + + // Get the actual type argument, e.g. SerializableFunction + Type inputT = ((ParameterizedType) extractorSupertype.getType()).getActualTypeArguments()[0]; + + // Get the actual supertype of the type being analyzed, hopefully with all type parameters + // resolved, e.g. SerializableFunction + TypeDescriptor supertypeDescriptor = type.getSupertype(supertype); + + // Substitute actual supertype into the extractor, e.g. + // TypeVariableExtractor, Integer> + TypeDescriptor> extractorT = + extractorSupertype.where(inputT, supertypeDescriptor.getType()); + + // Get output of the extractor. + Type outputT = ((ParameterizedType) extractorT.getType()).getActualTypeArguments()[1]; + TypeDescriptor res = TypeDescriptor.of(outputT); + if (res.hasUnresolvedParameters()) { + return null; + } else { + return (TypeDescriptor) res; + } + } + + /** + * Returns a type descriptor for the input of the given {@link SerializableFunction}, subject to + * Java type erasure: returns {@code null} if the type was erased. + */ + @Nullable + public static TypeDescriptor inputOf( + SerializableFunction fn) { + return extractFromTypeParameters( + fn, + SerializableFunction.class, + new TypeVariableExtractor, InputT>() {}); + } + + /** + * Returns a type descriptor for the output of the given {@link SerializableFunction}, subject to + * Java type erasure: returns {@code null} if the type was erased. + */ + @Nullable + public static TypeDescriptor outputOf( + SerializableFunction fn) { + return extractFromTypeParameters( + fn, + SerializableFunction.class, + new TypeVariableExtractor, OutputT>() {}); + } } diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/values/TypeDescriptorsTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/values/TypeDescriptorsTest.java index 1bf0fc9cf20da..a4f58da3e9845 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/values/TypeDescriptorsTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/values/TypeDescriptorsTest.java @@ -25,6 +25,7 @@ import static org.apache.beam.sdk.values.TypeDescriptors.strings; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNotEquals; +import static org.junit.Assert.assertNull; import java.util.List; import java.util.Set; @@ -70,4 +71,52 @@ public void testTypeDescriptorsListsOfLists() throws Exception { assertNotEquals(descriptor, new TypeDescriptor>() {}); assertNotEquals(descriptor, new TypeDescriptor>() {}); } + + private interface Generic {} + + private static Generic typeErasedGeneric() { + return new Generic() {}; + } + + private static TypeDescriptor extractFooT( + Generic instance) { + return TypeDescriptors.extractFromTypeParameters( + instance, + Generic.class, + new TypeDescriptors.TypeVariableExtractor< + Generic, ActualFooT>() {}); + } + + private static TypeDescriptor extractBarT( + Generic instance) { + return TypeDescriptors.extractFromTypeParameters( + instance, + Generic.class, + new TypeDescriptors.TypeVariableExtractor< + Generic, ActualBarT>() {}); + } + + private static TypeDescriptor> extractKV( + Generic instance) { + return TypeDescriptors.extractFromTypeParameters( + instance, + Generic.class, + new TypeDescriptors.TypeVariableExtractor< + Generic, KV>() {}); + } + + @Test + public void testTypeDescriptorsTypeParameterOf() throws Exception { + assertEquals(strings(), extractFooT(new Generic() {})); + assertEquals(integers(), extractBarT(new Generic() {})); + assertEquals(kvs(strings(), integers()), extractKV(new Generic() {})); + } + + @Test + public void testTypeDescriptorsTypeParameterOfErased() throws Exception { + Generic instance = TypeDescriptorsTest.typeErasedGeneric(); + assertNull(extractFooT(instance)); + assertEquals(strings(), extractBarT(instance)); + assertNull(extractKV(instance)); + } } diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/DynamicDestinations.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/DynamicDestinations.java index c5c2462be3276..ea4fc4ec8c627 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/DynamicDestinations.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/DynamicDestinations.java @@ -19,11 +19,11 @@ package org.apache.beam.sdk.io.gcp.bigquery; import static com.google.common.base.Preconditions.checkArgument; +import static org.apache.beam.sdk.values.TypeDescriptors.extractFromTypeParameters; import com.google.api.services.bigquery.model.TableSchema; import com.google.common.collect.Lists; import java.io.Serializable; -import java.lang.reflect.TypeVariable; import java.util.List; import javax.annotation.Nullable; import org.apache.beam.sdk.coders.CannotProvideCoderException; @@ -32,6 +32,7 @@ import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TypeDescriptor; +import org.apache.beam.sdk.values.TypeDescriptors; import org.apache.beam.sdk.values.ValueInSingleWindow; /** @@ -157,17 +158,16 @@ Coder getDestinationCoderWithDefault(CoderRegistry registry) return destinationCoder; } // If dynamicDestinations doesn't provide a coder, try to find it in the coder registry. - // We must first use reflection to figure out what the type parameter is. - TypeDescriptor superDescriptor = - TypeDescriptor.of(getClass()).getSupertype(DynamicDestinations.class); - if (!superDescriptor.getRawType().equals(DynamicDestinations.class)) { - throw new AssertionError( - "Couldn't find the DynamicDestinations superclass of " + this.getClass()); - } - TypeVariable typeVariable = superDescriptor.getTypeParameter("DestinationT"); - @SuppressWarnings("unchecked") TypeDescriptor descriptor = - (TypeDescriptor) superDescriptor.resolveType(typeVariable); + extractFromTypeParameters( + this, + DynamicDestinations.class, + new TypeDescriptors.TypeVariableExtractor< + DynamicDestinations, DestinationT>() {}); + checkArgument( + descriptor != null, + "Unable to infer a coder for DestinationT, " + + "please specify it explicitly by overriding getDestinationCoder()"); return registry.getCoder(descriptor); } } From ebd004119c387787d0e0fcd0487e1b2754c7dbc5 Mon Sep 17 00:00:00 2001 From: Eugene Kirpichov Date: Mon, 24 Jul 2017 15:07:15 -0700 Subject: [PATCH 213/346] [BEAM-2677] AvroIO.parseGenericRecords - schemaless AvroIO.read --- .../java/org/apache/beam/sdk/io/AvroIO.java | 199 +++++++++++++++++- .../org/apache/beam/sdk/io/AvroSource.java | 166 +++++++++++---- .../org/apache/beam/sdk/io/AvroIOTest.java | 89 +++++--- .../apache/beam/sdk/io/AvroSourceTest.java | 30 ++- .../io/gcp/bigquery/BigQuerySourceBase.java | 4 +- 5 files changed, 406 insertions(+), 82 deletions(-) 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 018b84f68ad17..27c9073b2937c 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 @@ -35,7 +35,9 @@ import org.apache.beam.sdk.annotations.Experimental; import org.apache.beam.sdk.annotations.Experimental.Kind; import org.apache.beam.sdk.coders.AvroCoder; +import org.apache.beam.sdk.coders.CannotProvideCoderException; import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.coders.CoderRegistry; import org.apache.beam.sdk.coders.StringUtf8Coder; import org.apache.beam.sdk.coders.VoidCoder; import org.apache.beam.sdk.io.FileBasedSink.DynamicDestinations; @@ -53,13 +55,16 @@ import org.apache.beam.sdk.values.PBegin; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PDone; +import org.apache.beam.sdk.values.TypeDescriptor; +import org.apache.beam.sdk.values.TypeDescriptors; /** * {@link PTransform}s for reading and writing Avro files. * - *

              To read a {@link PCollection} from one or more Avro files, use {@code AvroIO.read()}, using - * {@link AvroIO.Read#from} to specify the filename or filepattern to read from. Alternatively, if - * the filepatterns to be read are themselves in a {@link PCollection}, apply {@link #readAll}. + *

              To read a {@link PCollection} from one or more Avro files with the same schema known at + * pipeline construction time, use {@code AvroIO.read()}, using {@link AvroIO.Read#from} to specify + * the filename or filepattern to read from. Alternatively, if the filepatterns to be read are + * themselves in a {@link PCollection}, apply {@link #readAll}. * *

              See {@link FileSystems} for information on supported file systems and filepatterns. * @@ -70,6 +75,12 @@ * schema. Likewise, to read a {@link PCollection} of filepatterns, apply {@link * #readAllGenericRecords}. * + *

              To read records from files whose schema is unknown at pipeline construction time or differs + * between files, use {@link #parseGenericRecords} - in this case, you will need to specify a + * parsing function for converting each {@link GenericRecord} into a value of your custom type. + * Likewise, to read a {@link PCollection} of filepatterns with unknown schema, use {@link + * #parseAllGenericRecords}. + * *

              For example: * *

              {@code
              @@ -84,12 +95,20 @@
                * PCollection records =
                *     p.apply(AvroIO.readGenericRecords(schema)
                *                .from("gs://my_bucket/path/to/records-*.avro"));
              + *
              + * PCollection records =
              + *     p.apply(AvroIO.parseGenericRecords(new SerializableFunction() {
              + *       public Foo apply(GenericRecord record) {
              + *         // If needed, access the schema of the record using record.getSchema()
              + *         return ...;
              + *       }
              + *     }));
                * }
              * *

              If it is known that the filepattern will match a very large number of files (e.g. tens of - * thousands or more), use {@link Read#withHintMatchesManyFiles} for better performance and - * scalability. Note that it may decrease performance if the filepattern matches only a small - * number of files. + * thousands or more), use {@link Read#withHintMatchesManyFiles} or {@link + * Parse#withHintMatchesManyFiles} for better performance and scalability. Note that it may decrease + * performance if the filepattern matches only a small number of files. * *

              Reading from a {@link PCollection} of filepatterns: * @@ -101,6 +120,8 @@ * filepatterns.apply(AvroIO.read(AvroAutoGenClass.class)); * PCollection genericRecords = * filepatterns.apply(AvroIO.readGenericRecords(schema)); + * PCollection records = + * filepatterns.apply(AvroIO.parseAllGenericRecords(new SerializableFunction...); * }

              * *

              To write a {@link PCollection} to one or more Avro files, use {@link AvroIO.Write}, using @@ -207,6 +228,29 @@ public static ReadAll readAllGenericRecords(String schema) { return readAllGenericRecords(new Schema.Parser().parse(schema)); } + /** + * Reads Avro file(s) containing records of an unspecified schema and converting each record to a + * custom type. + */ + public static Parse parseGenericRecords(SerializableFunction parseFn) { + return new AutoValue_AvroIO_Parse.Builder() + .setParseFn(parseFn) + .setHintMatchesManyFiles(false) + .build(); + } + + /** + * Like {@link #parseGenericRecords(SerializableFunction)}, but reads each filepattern in the + * input {@link PCollection}. + */ + public static ParseAll parseAllGenericRecords( + SerializableFunction parseFn) { + return new AutoValue_AvroIO_ParseAll.Builder() + .setParseFn(parseFn) + .setDesiredBundleSizeBytes(64 * 1024 * 1024L) + .build(); + } + /** * Writes a {@link PCollection} to an Avro file (or multiple Avro files matching a sharding * pattern). @@ -387,6 +431,149 @@ public FileBasedSource apply(String input) { ///////////////////////////////////////////////////////////////////////////// + /** Implementation of {@link #parseGenericRecords}. */ + @AutoValue + public abstract static class Parse extends PTransform> { + @Nullable abstract ValueProvider getFilepattern(); + abstract SerializableFunction getParseFn(); + @Nullable abstract Coder getCoder(); + abstract boolean getHintMatchesManyFiles(); + + abstract Builder toBuilder(); + + @AutoValue.Builder + abstract static class Builder { + abstract Builder setFilepattern(ValueProvider filepattern); + abstract Builder setParseFn(SerializableFunction parseFn); + abstract Builder setCoder(Coder coder); + abstract Builder setHintMatchesManyFiles(boolean hintMatchesManyFiles); + + abstract Parse build(); + } + + /** Reads from the given filename or filepattern. */ + public Parse from(String filepattern) { + return from(StaticValueProvider.of(filepattern)); + } + + /** Like {@link #from(String)}. */ + public Parse from(ValueProvider filepattern) { + return toBuilder().setFilepattern(filepattern).build(); + } + + /** Sets a coder for the result of the parse function. */ + public Parse withCoder(Coder coder) { + return toBuilder().setCoder(coder).build(); + } + + /** Like {@link Read#withHintMatchesManyFiles()}. */ + public Parse withHintMatchesManyFiles() { + return toBuilder().setHintMatchesManyFiles(true).build(); + } + + @Override + public PCollection expand(PBegin input) { + checkNotNull(getFilepattern(), "filepattern"); + Coder coder = inferCoder(getCoder(), getParseFn(), input.getPipeline().getCoderRegistry()); + if (getHintMatchesManyFiles()) { + return input + .apply(Create.ofProvider(getFilepattern(), StringUtf8Coder.of())) + .apply(parseAllGenericRecords(getParseFn()).withCoder(getCoder())); + } + return input.apply( + org.apache.beam.sdk.io.Read.from( + AvroSource.from(getFilepattern()).withParseFn(getParseFn(), coder))); + } + + private static Coder inferCoder( + @Nullable Coder explicitCoder, + SerializableFunction parseFn, + CoderRegistry coderRegistry) { + if (explicitCoder != null) { + return explicitCoder; + } + // If a coder was not specified explicitly, infer it from parse fn. + TypeDescriptor descriptor = TypeDescriptors.outputOf(parseFn); + String message = + "Unable to infer coder for output of parseFn. Specify it explicitly using withCoder()."; + checkArgument(descriptor != null, message); + try { + return coderRegistry.getCoder(descriptor); + } catch (CannotProvideCoderException e) { + throw new IllegalArgumentException(message, e); + } + } + + @Override + public void populateDisplayData(DisplayData.Builder builder) { + super.populateDisplayData(builder); + builder + .addIfNotNull( + DisplayData.item("filePattern", getFilepattern()).withLabel("Input File Pattern")) + .add(DisplayData.item("parseFn", getParseFn().getClass()).withLabel("Parse function")); + } + } + + ///////////////////////////////////////////////////////////////////////////// + + /** Implementation of {@link #parseAllGenericRecords}. */ + @AutoValue + public abstract static class ParseAll extends PTransform, PCollection> { + abstract SerializableFunction getParseFn(); + @Nullable abstract Coder getCoder(); + abstract long getDesiredBundleSizeBytes(); + + abstract Builder toBuilder(); + + @AutoValue.Builder + abstract static class Builder { + abstract Builder setParseFn(SerializableFunction parseFn); + abstract Builder setCoder(Coder coder); + abstract Builder setDesiredBundleSizeBytes(long desiredBundleSizeBytes); + + abstract ParseAll build(); + } + + /** Specifies the coder for the result of the {@code parseFn}. */ + public ParseAll withCoder(Coder coder) { + return toBuilder().setCoder(coder).build(); + } + + @VisibleForTesting + ParseAll withDesiredBundleSizeBytes(long desiredBundleSizeBytes) { + return toBuilder().setDesiredBundleSizeBytes(desiredBundleSizeBytes).build(); + } + + @Override + public PCollection expand(PCollection input) { + final Coder coder = + Parse.inferCoder(getCoder(), getParseFn(), input.getPipeline().getCoderRegistry()); + SerializableFunction> createSource = + new SerializableFunction>() { + @Override + public FileBasedSource apply(String input) { + return AvroSource.from(input).withParseFn(getParseFn(), coder); + } + }; + return input + .apply( + "Parse all via FileBasedSource", + new ReadAllViaFileBasedSource<>( + SerializableFunctions.constant(true) /* isSplittable */, + getDesiredBundleSizeBytes(), + createSource)) + .setCoder(coder); + } + + @Override + public void populateDisplayData(DisplayData.Builder builder) { + super.populateDisplayData(builder); + builder.add(DisplayData.item("parseFn", getParseFn().getClass()).withLabel("Parse function")); + } + } + + ///////////////////////////////////////////////////////////////////////////// + /** Implementation of {@link #write}. */ @AutoValue public abstract static class Write extends PTransform, PDone> { diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/AvroSource.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/AvroSource.java index a98d870f02967..d277503d7dc51 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/AvroSource.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/AvroSource.java @@ -17,6 +17,7 @@ */ package org.apache.beam.sdk.io; +import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.base.Preconditions.checkNotNull; import static com.google.common.base.Preconditions.checkState; @@ -27,8 +28,10 @@ import java.io.IOException; import java.io.InputStream; import java.io.InvalidObjectException; +import java.io.ObjectInputStream; import java.io.ObjectStreamException; import java.io.PushbackInputStream; +import java.io.Serializable; import java.nio.ByteBuffer; import java.nio.channels.Channels; import java.nio.channels.ReadableByteChannel; @@ -53,10 +56,12 @@ import org.apache.beam.sdk.PipelineRunner; import org.apache.beam.sdk.annotations.Experimental; import org.apache.beam.sdk.coders.AvroCoder; +import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.io.fs.MatchResult.Metadata; import org.apache.beam.sdk.io.fs.ResourceId; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.options.ValueProvider; +import org.apache.beam.sdk.transforms.SerializableFunction; import org.apache.beam.sdk.values.PCollection; import org.apache.commons.compress.compressors.bzip2.BZip2CompressorInputStream; import org.apache.commons.compress.compressors.snappy.SnappyCompressorInputStream; @@ -130,19 +135,84 @@ public class AvroSource extends BlockBasedSource { // The default sync interval is 64k. private static final long DEFAULT_MIN_BUNDLE_SIZE = 2 * DataFileConstants.DEFAULT_SYNC_INTERVAL; - // The type of the records contained in the file. - private final Class type; + // Use cases of AvroSource are: + // 1) AvroSource Reading GenericRecord records with a specified schema. + // 2) AvroSource Reading records of a generated Avro class Foo. + // 3) AvroSource Reading GenericRecord records with an unspecified schema + // and converting them to type T. + // | Case 1 | Case 2 | Case 3 | + // type | GenericRecord | Foo | GenericRecord | + // readerSchemaString | non-null | non-null | null | + // parseFn | null | null | non-null | + // outputCoder | null | null | non-null | + private static class Mode implements Serializable { + private final Class type; + + // The JSON schema used to decode records. + @Nullable + private String readerSchemaString; + + @Nullable + private final SerializableFunction parseFn; + + @Nullable + private final Coder outputCoder; + + private Mode( + Class type, + @Nullable String readerSchemaString, + @Nullable SerializableFunction parseFn, + @Nullable Coder outputCoder) { + this.type = type; + this.readerSchemaString = internSchemaString(readerSchemaString); + this.parseFn = parseFn; + this.outputCoder = outputCoder; + } - // The JSON schema used to decode records. - @Nullable - private final String readerSchemaString; + private void readObject(ObjectInputStream is) throws IOException, ClassNotFoundException { + is.defaultReadObject(); + readerSchemaString = internSchemaString(readerSchemaString); + } + + private Coder getOutputCoder() { + if (parseFn == null) { + return AvroCoder.of((Class) type, internOrParseSchemaString(readerSchemaString)); + } else { + return outputCoder; + } + } + + private void validate() { + if (parseFn == null) { + checkArgument( + readerSchemaString != null, + "schema must be specified using withSchema() when not using a parse fn"); + } + } + } + + private static Mode readGenericRecordsWithSchema(String schema) { + return new Mode<>(GenericRecord.class, schema, null, null); + } + private static Mode readGeneratedClasses(Class clazz) { + return new Mode<>(clazz, ReflectData.get().getSchema(clazz).toString(), null, null); + } + private static Mode parseGenericRecords( + SerializableFunction parseFn, Coder outputCoder) { + return new Mode<>(GenericRecord.class, null, parseFn, outputCoder); + } + + private final Mode mode; /** - * Reads from the given file name or pattern ("glob"). The returned source can be further + * Reads from the given file name or pattern ("glob"). The returned source needs to be further * configured by calling {@link #withSchema} to return a type other than {@link GenericRecord}. */ public static AvroSource from(ValueProvider fileNameOrPattern) { - return new AvroSource<>(fileNameOrPattern, DEFAULT_MIN_BUNDLE_SIZE, null, GenericRecord.class); + return new AvroSource<>( + fileNameOrPattern, + DEFAULT_MIN_BUNDLE_SIZE, + readGenericRecordsWithSchema(null /* will need to be specified in withSchema */)); } /** Like {@link #from(ValueProvider)}. */ @@ -152,23 +222,40 @@ public static AvroSource from(String fileNameOrPattern) { /** Reads files containing records that conform to the given schema. */ public AvroSource withSchema(String schema) { + checkNotNull(schema, "schema"); return new AvroSource<>( - getFileOrPatternSpecProvider(), getMinBundleSize(), schema, GenericRecord.class); + getFileOrPatternSpecProvider(), + getMinBundleSize(), + readGenericRecordsWithSchema(schema)); } /** Like {@link #withSchema(String)}. */ public AvroSource withSchema(Schema schema) { - return new AvroSource<>( - getFileOrPatternSpecProvider(), getMinBundleSize(), schema.toString(), GenericRecord.class); + checkNotNull(schema, "schema"); + return withSchema(schema.toString()); } /** Reads files containing records of the given class. */ public AvroSource withSchema(Class clazz) { + checkNotNull(clazz, "clazz"); + return new AvroSource<>( + getFileOrPatternSpecProvider(), + getMinBundleSize(), + readGeneratedClasses(clazz)); + } + + /** + * Reads {@link GenericRecord} of unspecified schema and maps them to instances of a custom type + * using the given {@code parseFn} and encoded using the given coder. + */ + public AvroSource withParseFn( + SerializableFunction parseFn, Coder coder) { + checkNotNull(parseFn, "parseFn"); + checkNotNull(parseFn, "coder"); return new AvroSource<>( getFileOrPatternSpecProvider(), getMinBundleSize(), - ReflectData.get().getSchema(clazz).toString(), - clazz); + parseGenericRecords(parseFn, coder)); } /** @@ -176,19 +263,16 @@ public AvroSource withSchema(Class clazz) { * minBundleSize} and its use. */ public AvroSource withMinBundleSize(long minBundleSize) { - return new AvroSource<>( - getFileOrPatternSpecProvider(), minBundleSize, readerSchemaString, type); + return new AvroSource<>(getFileOrPatternSpecProvider(), minBundleSize, mode); } /** Constructor for FILEPATTERN mode. */ private AvroSource( ValueProvider fileNameOrPattern, long minBundleSize, - String readerSchemaString, - Class type) { + Mode mode) { super(fileNameOrPattern, minBundleSize); - this.readerSchemaString = internSchemaString(readerSchemaString); - this.type = type; + this.mode = mode; } /** Constructor for SINGLE_FILE_OR_SUBRANGE mode. */ @@ -197,18 +281,15 @@ private AvroSource( long minBundleSize, long startOffset, long endOffset, - String readerSchemaString, - Class type) { + Mode mode) { super(metadata, minBundleSize, startOffset, endOffset); - this.readerSchemaString = internSchemaString(readerSchemaString); - this.type = type; + this.mode = mode; } @Override public void validate() { - // AvroSource objects do not need to be configured with more than a file pattern. Overridden to - // make this explicit. super.validate(); + mode.validate(); } /** @@ -225,7 +306,7 @@ public BlockBasedSource createForSubrangeOfFile(String fileName, long start, @Override public BlockBasedSource createForSubrangeOfFile(Metadata fileMetadata, long start, long end) { - return new AvroSource<>(fileMetadata, getMinBundleSize(), start, end, readerSchemaString, type); + return new AvroSource<>(fileMetadata, getMinBundleSize(), start, end, mode); } @Override @@ -234,14 +315,14 @@ protected BlockBasedReader createSingleFileReader(PipelineOptions options) { } @Override - public AvroCoder getDefaultOutputCoder() { - return AvroCoder.of(type, internOrParseSchemaString(readerSchemaString)); + public Coder getDefaultOutputCoder() { + return mode.getOutputCoder(); } @VisibleForTesting @Nullable String getReaderSchemaString() { - return readerSchemaString; + return mode.readerSchemaString; } /** Avro file metadata. */ @@ -380,15 +461,9 @@ private Object readResolve() throws ObjectStreamException { switch (getMode()) { case SINGLE_FILE_OR_SUBRANGE: return new AvroSource<>( - getSingleFileMetadata(), - getMinBundleSize(), - getStartOffset(), - getEndOffset(), - readerSchemaString, - type); + getSingleFileMetadata(), getMinBundleSize(), getStartOffset(), getEndOffset(), mode); case FILEPATTERN: - return new AvroSource<>( - getFileOrPatternSpecProvider(), getMinBundleSize(), readerSchemaString, type); + return new AvroSource<>(getFileOrPatternSpecProvider(), getMinBundleSize(), mode); default: throw new InvalidObjectException( String.format("Unknown mode %s for AvroSource %s", getMode(), this)); @@ -402,6 +477,8 @@ private Object readResolve() throws ObjectStreamException { */ @Experimental(Experimental.Kind.SOURCE_SINK) static class AvroBlock extends Block { + private final Mode mode; + // The number of records in the block. private final long numRecords; @@ -412,7 +489,7 @@ static class AvroBlock extends Block { private long currentRecordIndex = 0; // A DatumReader to read records from the block. - private final DatumReader reader; + private final DatumReader reader; // A BinaryDecoder used by the reader to decode records. private final BinaryDecoder decoder; @@ -455,19 +532,19 @@ private static InputStream decodeAsInputStream(byte[] data, String codec) throws AvroBlock( byte[] data, long numRecords, - Class type, - String readerSchemaString, + Mode mode, String writerSchemaString, String codec) throws IOException { + this.mode = mode; this.numRecords = numRecords; checkNotNull(writerSchemaString, "writerSchemaString"); Schema writerSchema = internOrParseSchemaString(writerSchemaString); Schema readerSchema = internOrParseSchemaString( - MoreObjects.firstNonNull(readerSchemaString, writerSchemaString)); + MoreObjects.firstNonNull(mode.readerSchemaString, writerSchemaString)); this.reader = - (type == GenericRecord.class) + (mode.type == GenericRecord.class) ? new GenericDatumReader(writerSchema, readerSchema) : new ReflectDatumReader(writerSchema, readerSchema); this.decoder = DecoderFactory.get().binaryDecoder(decodeAsInputStream(data, codec), null); @@ -483,7 +560,9 @@ public boolean readNextRecord() throws IOException { if (currentRecordIndex >= numRecords) { return false; } - currentRecord = reader.read(null, decoder); + Object record = reader.read(null, decoder); + currentRecord = + (mode.parseFn == null) ? ((T) record) : mode.parseFn.apply((GenericRecord) record); currentRecordIndex++; return true; } @@ -585,8 +664,7 @@ public boolean readNextBlock() throws IOException { new AvroBlock<>( data, numRecords, - getCurrentSource().type, - getCurrentSource().readerSchemaString, + getCurrentSource().mode, metadata.getSchemaString(), metadata.getCodec()); 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 90cd824d5de82..154ff5a59e0ef 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 @@ -67,6 +67,7 @@ import org.apache.beam.sdk.testing.UsesTestStream; import org.apache.beam.sdk.testing.ValidatesRunner; import org.apache.beam.sdk.transforms.Create; +import org.apache.beam.sdk.transforms.SerializableFunction; import org.apache.beam.sdk.transforms.display.DisplayData; import org.apache.beam.sdk.transforms.display.DisplayDataEvaluator; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; @@ -114,9 +115,9 @@ static class GenericClass { public GenericClass() {} - public GenericClass(int intValue, String stringValue) { - this.intField = intValue; - this.stringField = stringValue; + public GenericClass(int intField, String stringField) { + this.intField = intField; + this.stringField = stringField; } @Override @@ -142,9 +143,18 @@ public boolean equals(Object other) { } } + private static class ParseGenericClass + implements SerializableFunction { + @Override + public GenericClass apply(GenericRecord input) { + return new GenericClass( + (int) input.get("intField"), input.get("stringField").toString()); + } + } + @Test @Category(NeedsRunner.class) - public void testAvroIOWriteAndReadASingleFile() throws Throwable { + public void testAvroIOWriteAndReadAndParseASingleFile() throws Throwable { List values = ImmutableList.of(new GenericClass(3, "hi"), new GenericClass(5, "bar")); File outputFile = tmpFolder.newFile("output.avro"); @@ -153,23 +163,45 @@ public void testAvroIOWriteAndReadASingleFile() throws Throwable { .apply(AvroIO.write(GenericClass.class).to(outputFile.getAbsolutePath()).withoutSharding()); writePipeline.run().waitUntilFinish(); - // Test the same data via read(), read().withHintMatchesManyFiles(), and readAll() + // Test the same data using all versions of read(). + PCollection path = + readPipeline.apply("Create path", Create.of(outputFile.getAbsolutePath())); PAssert.that( - readPipeline.apply( - "Read", AvroIO.read(GenericClass.class).from(outputFile.getAbsolutePath()))) + readPipeline.apply( + "Read", AvroIO.read(GenericClass.class).from(outputFile.getAbsolutePath()))) .containsInAnyOrder(values); PAssert.that( - readPipeline.apply( - "Read withHintMatchesManyFiles", - AvroIO.read(GenericClass.class) - .from(outputFile.getAbsolutePath()) - .withHintMatchesManyFiles())) + readPipeline.apply( + "Read withHintMatchesManyFiles", + AvroIO.read(GenericClass.class) + .from(outputFile.getAbsolutePath()) + .withHintMatchesManyFiles())) .containsInAnyOrder(values); PAssert.that( - "ReadAll", - readPipeline - .apply(Create.of(outputFile.getAbsolutePath())) - .apply(AvroIO.readAll(GenericClass.class).withDesiredBundleSizeBytes(10))) + path.apply( + "ReadAll", AvroIO.readAll(GenericClass.class).withDesiredBundleSizeBytes(10))) + .containsInAnyOrder(values); + PAssert.that( + readPipeline.apply( + "Parse", + AvroIO.parseGenericRecords(new ParseGenericClass()) + .from(outputFile.getAbsolutePath()) + .withCoder(AvroCoder.of(GenericClass.class)))) + .containsInAnyOrder(values); + PAssert.that( + readPipeline.apply( + "Parse withHintMatchesManyFiles", + AvroIO.parseGenericRecords(new ParseGenericClass()) + .from(outputFile.getAbsolutePath()) + .withCoder(AvroCoder.of(GenericClass.class)) + .withHintMatchesManyFiles())) + .containsInAnyOrder(values); + PAssert.that( + path.apply( + "ParseAll", + AvroIO.parseAllGenericRecords(new ParseGenericClass()) + .withCoder(AvroCoder.of(GenericClass.class)) + .withDesiredBundleSizeBytes(10))) .containsInAnyOrder(values); readPipeline.run(); @@ -200,7 +232,7 @@ public void testAvroIOWriteAndReadMultipleFilepatterns() throws Throwable { .withNumShards(3)); writePipeline.run().waitUntilFinish(); - // Test both read() and readAll() + // Test read(), readAll(), and parseAllGenericRecords(). PAssert.that( readPipeline.apply( "Read first", @@ -213,15 +245,22 @@ public void testAvroIOWriteAndReadMultipleFilepatterns() throws Throwable { AvroIO.read(GenericClass.class) .from(tmpFolder.getRoot().getAbsolutePath() + "/second*"))) .containsInAnyOrder(secondValues); + PCollection paths = + readPipeline.apply( + "Create paths", + Create.of( + tmpFolder.getRoot().getAbsolutePath() + "/first*", + tmpFolder.getRoot().getAbsolutePath() + "/second*")); + PAssert.that( + paths.apply( + "Read all", AvroIO.readAll(GenericClass.class).withDesiredBundleSizeBytes(10))) + .containsInAnyOrder(Iterables.concat(firstValues, secondValues)); PAssert.that( - readPipeline - .apply( - "Create paths", - Create.of( - tmpFolder.getRoot().getAbsolutePath() + "/first*", - tmpFolder.getRoot().getAbsolutePath() + "/second*")) - .apply( - "Read all", AvroIO.readAll(GenericClass.class).withDesiredBundleSizeBytes(10))) + paths.apply( + "Parse all", + AvroIO.parseAllGenericRecords(new ParseGenericClass()) + .withCoder(AvroCoder.of(GenericClass.class)) + .withDesiredBundleSizeBytes(10))) .containsInAnyOrder(Iterables.concat(firstValues, secondValues)); readPipeline.run(); diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/AvroSourceTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/AvroSourceTest.java index bf2ac95272ff2..714e029764143 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/AvroSourceTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/AvroSourceTest.java @@ -59,6 +59,7 @@ import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.testing.SourceTestUtils; +import org.apache.beam.sdk.transforms.SerializableFunction; import org.apache.beam.sdk.transforms.display.DisplayData; import org.apache.beam.sdk.util.SerializableUtils; import org.hamcrest.Matchers; @@ -407,11 +408,6 @@ public void testCreationWithSchema() throws Exception { source = AvroSource.from(filename).withSchema(schemaString); records = SourceTestUtils.readFromSource(source, null); assertEqualsWithGeneric(expected, records); - - // Create a source with no schema - source = AvroSource.from(filename); - records = SourceTestUtils.readFromSource(source, null); - assertEqualsWithGeneric(expected, records); } @Test @@ -449,6 +445,30 @@ public void testSchemaStringIsInterned() throws Exception { assertSame(sourceA.getReaderSchemaString(), sourceC.getReaderSchemaString()); } + @Test + public void testParseFn() throws Exception { + List expected = createRandomRecords(100); + String filename = generateTestFile("tmp.avro", expected, SyncBehavior.SYNC_DEFAULT, 0, + AvroCoder.of(Bird.class), DataFileConstants.NULL_CODEC); + + AvroSource source = + AvroSource.from(filename) + .withParseFn( + new SerializableFunction() { + @Override + public Bird apply(GenericRecord input) { + return new Bird( + (long) input.get("number"), + input.get("species").toString(), + input.get("quality").toString(), + (long) input.get("quantity")); + } + }, + AvroCoder.of(Bird.class)); + List actual = SourceTestUtils.readFromSource(source, null); + assertThat(actual, containsInAnyOrder(expected.toArray())); + } + private void assertEqualsWithGeneric(List expected, List actual) { assertEquals(expected.size(), actual.size()); for (int i = 0; i < expected.size(); i++) { diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQuerySourceBase.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQuerySourceBase.java index 2b1eafef70f31..6c118a0b116cd 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQuerySourceBase.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQuerySourceBase.java @@ -183,8 +183,8 @@ public TableRow apply(GenericRecord input) { List> avroSources = Lists.newArrayList(); for (ResourceId file : files) { - avroSources.add(new TransformingSource<>( - AvroSource.from(file.toString()), function, getDefaultOutputCoder())); + avroSources.add( + AvroSource.from(file.toString()).withParseFn(function, getDefaultOutputCoder())); } return ImmutableList.copyOf(avroSources); } From c16947ec01d21ef99a5e2024d7aaead3c7a4399f Mon Sep 17 00:00:00 2001 From: Eugene Kirpichov Date: Tue, 25 Jul 2017 23:35:00 -0700 Subject: [PATCH 214/346] Removes TransformingSource that is now unused --- .../io/gcp/bigquery/TransformingSource.java | 136 ------------------ .../sdk/io/gcp/bigquery/BigQueryIOTest.java | 68 --------- 2 files changed, 204 deletions(-) delete mode 100644 sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TransformingSource.java diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TransformingSource.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TransformingSource.java deleted file mode 100644 index b8e6b396c36c0..0000000000000 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TransformingSource.java +++ /dev/null @@ -1,136 +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.Preconditions.checkNotNull; - -import com.google.common.annotations.VisibleForTesting; -import com.google.common.base.Function; -import com.google.common.collect.Lists; -import java.io.IOException; -import java.util.List; -import java.util.NoSuchElementException; -import org.apache.beam.sdk.coders.Coder; -import org.apache.beam.sdk.io.BoundedSource; -import org.apache.beam.sdk.options.PipelineOptions; -import org.apache.beam.sdk.transforms.SerializableFunction; -import org.joda.time.Instant; - -/** - * A {@link BoundedSource} that reads from {@code BoundedSource} - * and transforms elements to type {@code V}. -*/ -@VisibleForTesting -class TransformingSource extends BoundedSource { - private final BoundedSource boundedSource; - private final SerializableFunction function; - private final Coder outputCoder; - - TransformingSource( - BoundedSource boundedSource, - SerializableFunction function, - Coder outputCoder) { - this.boundedSource = checkNotNull(boundedSource, "boundedSource"); - this.function = checkNotNull(function, "function"); - this.outputCoder = checkNotNull(outputCoder, "outputCoder"); - } - - @Override - public List> split( - long desiredBundleSizeBytes, PipelineOptions options) throws Exception { - return Lists.transform( - boundedSource.split(desiredBundleSizeBytes, options), - new Function, BoundedSource>() { - @Override - public BoundedSource apply(BoundedSource input) { - return new TransformingSource<>(input, function, outputCoder); - } - }); - } - - @Override - public long getEstimatedSizeBytes(PipelineOptions options) throws Exception { - return boundedSource.getEstimatedSizeBytes(options); - } - - @Override - public BoundedReader createReader(PipelineOptions options) throws IOException { - return new TransformingReader(boundedSource.createReader(options)); - } - - @Override - public void validate() { - boundedSource.validate(); - } - - @Override - public Coder getDefaultOutputCoder() { - return outputCoder; - } - - private class TransformingReader extends BoundedReader { - private final BoundedReader boundedReader; - - private TransformingReader(BoundedReader boundedReader) { - this.boundedReader = checkNotNull(boundedReader, "boundedReader"); - } - - @Override - public synchronized BoundedSource getCurrentSource() { - return new TransformingSource<>(boundedReader.getCurrentSource(), function, outputCoder); - } - - @Override - public boolean start() throws IOException { - return boundedReader.start(); - } - - @Override - public boolean advance() throws IOException { - return boundedReader.advance(); - } - - @Override - public V getCurrent() throws NoSuchElementException { - T current = boundedReader.getCurrent(); - return function.apply(current); - } - - @Override - public void close() throws IOException { - boundedReader.close(); - } - - @Override - public synchronized BoundedSource splitAtFraction(double fraction) { - BoundedSource split = boundedReader.splitAtFraction(fraction); - return split == null ? null : new TransformingSource<>(split, function, outputCoder); - } - - @Override - public Double getFractionConsumed() { - return boundedReader.getFractionConsumed(); - } - - @Override - public Instant getCurrentTimestamp() throws NoSuchElementException { - return boundedReader.getCurrentTimestamp(); - } - } -} 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 3465b4e7f9650..8db4e9407ec11 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 @@ -86,7 +86,6 @@ import org.apache.beam.sdk.coders.StringUtf8Coder; import org.apache.beam.sdk.coders.VarIntCoder; import org.apache.beam.sdk.io.BoundedSource; -import org.apache.beam.sdk.io.CountingSource; import org.apache.beam.sdk.io.FileSystems; import org.apache.beam.sdk.io.GenerateSequence; import org.apache.beam.sdk.io.fs.ResourceId; @@ -1510,8 +1509,6 @@ public void testBigQueryTableSourceInitSplit() throws Exception { // Simulate a repeated call to split(), like a Dataflow worker will sometimes do. sources = bqSource.split(200, options); assertEquals(2, sources.size()); - BoundedSource actual = sources.get(0); - assertThat(actual, CoreMatchers.instanceOf(TransformingSource.class)); // A repeated call to split() should not have caused a duplicate extract job. assertEquals(1, fakeJobService.getNumExtractJobCalls()); @@ -1594,8 +1591,6 @@ public void testBigQueryQuerySourceInitSplit() throws Exception { List> sources = bqSource.split(100, options); assertEquals(2, sources.size()); - BoundedSource actual = sources.get(0); - assertThat(actual, CoreMatchers.instanceOf(TransformingSource.class)); } @Test @@ -1673,69 +1668,6 @@ public void testBigQueryNoTableQuerySourceInitSplit() throws Exception { List> sources = bqSource.split(100, options); assertEquals(2, sources.size()); - BoundedSource actual = sources.get(0); - assertThat(actual, CoreMatchers.instanceOf(TransformingSource.class)); - } - - @Test - public void testTransformingSource() throws Exception { - int numElements = 10000; - @SuppressWarnings("deprecation") - BoundedSource longSource = CountingSource.upTo(numElements); - SerializableFunction toStringFn = - new SerializableFunction() { - @Override - public String apply(Long input) { - return input.toString(); - }}; - BoundedSource stringSource = new TransformingSource<>( - longSource, toStringFn, StringUtf8Coder.of()); - - List expected = Lists.newArrayList(); - for (int i = 0; i < numElements; i++) { - expected.add(String.valueOf(i)); - } - - PipelineOptions options = PipelineOptionsFactory.create(); - Assert.assertThat( - SourceTestUtils.readFromSource(stringSource, options), - CoreMatchers.is(expected)); - SourceTestUtils.assertSplitAtFractionBehavior( - stringSource, 100, 0.3, ExpectedSplitOutcome.MUST_SUCCEED_AND_BE_CONSISTENT, options); - - SourceTestUtils.assertSourcesEqualReferenceSource( - stringSource, stringSource.split(100, options), options); - } - - @Test - public void testTransformingSourceUnsplittable() throws Exception { - int numElements = 10000; - @SuppressWarnings("deprecation") - BoundedSource longSource = - SourceTestUtils.toUnsplittableSource(CountingSource.upTo(numElements)); - SerializableFunction toStringFn = - new SerializableFunction() { - @Override - public String apply(Long input) { - return input.toString(); - } - }; - BoundedSource stringSource = - new TransformingSource<>(longSource, toStringFn, StringUtf8Coder.of()); - - List expected = Lists.newArrayList(); - for (int i = 0; i < numElements; i++) { - expected.add(String.valueOf(i)); - } - - PipelineOptions options = PipelineOptionsFactory.create(); - Assert.assertThat( - SourceTestUtils.readFromSource(stringSource, options), CoreMatchers.is(expected)); - SourceTestUtils.assertSplitAtFractionBehavior( - stringSource, 100, 0.3, ExpectedSplitOutcome.MUST_BE_CONSISTENT_IF_SUCCEEDS, options); - - SourceTestUtils.assertSourcesEqualReferenceSource( - stringSource, stringSource.split(100, options), options); } @Test From 6ea2eda2b5dbe4fc0cc2a84b64b76a07c7d0eda8 Mon Sep 17 00:00:00 2001 From: Thomas Groh Date: Thu, 15 Jun 2017 15:53:46 -0700 Subject: [PATCH 215/346] Perform a Multi-step combine in the DirectRunner This exercises the entire CombineFn lifecycle for simple combine fns, expressed as a collection of DoFns. --- .../beam/runners/direct/DirectRunner.java | 65 +-- .../beam/runners/direct/MultiStepCombine.java | 423 ++++++++++++++++++ .../direct/TransformEvaluatorRegistry.java | 4 + .../runners/direct/MultiStepCombineTest.java | 228 ++++++++++ 4 files changed, 690 insertions(+), 30 deletions(-) create mode 100644 runners/direct-java/src/main/java/org/apache/beam/runners/direct/MultiStepCombine.java create mode 100644 runners/direct-java/src/test/java/org/apache/beam/runners/direct/MultiStepCombineTest.java 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 c5f29e5e4a492..642ce8fc56e85 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 @@ -233,36 +233,41 @@ List defaultTransformOverrides() { PTransformMatchers.writeWithRunnerDeterminedSharding(), new WriteWithShardingFactory())); /* Uses a view internally. */ } - builder = builder.add( - PTransformOverride.of( - PTransformMatchers.urnEqualTo(PTransformTranslation.CREATE_VIEW_TRANSFORM_URN), - new ViewOverrideFactory())) /* Uses pardos and GBKs */ - .add( - PTransformOverride.of( - PTransformMatchers.urnEqualTo(PTransformTranslation.TEST_STREAM_TRANSFORM_URN), - new DirectTestStreamFactory(this))) /* primitive */ - // SplittableParMultiDo is implemented in terms of nonsplittable simple ParDos and extra - // primitives - .add( - PTransformOverride.of( - PTransformMatchers.splittableParDo(), new ParDoMultiOverrideFactory())) - // state and timer pardos are implemented in terms of simple ParDos and extra primitives - .add( - PTransformOverride.of( - PTransformMatchers.stateOrTimerParDo(), new ParDoMultiOverrideFactory())) - .add( - PTransformOverride.of( - PTransformMatchers.urnEqualTo( - SplittableParDo.SPLITTABLE_PROCESS_KEYED_ELEMENTS_URN), - new SplittableParDoViaKeyedWorkItems.OverrideFactory())) - .add( - PTransformOverride.of( - PTransformMatchers.urnEqualTo(SplittableParDo.SPLITTABLE_GBKIKWI_URN), - new DirectGBKIntoKeyedWorkItemsOverrideFactory())) /* Returns a GBKO */ - .add( - PTransformOverride.of( - PTransformMatchers.urnEqualTo(PTransformTranslation.GROUP_BY_KEY_TRANSFORM_URN), - new DirectGroupByKeyOverrideFactory())); /* returns two chained primitives. */ + builder = + builder + .add( + PTransformOverride.of( + MultiStepCombine.matcher(), MultiStepCombine.Factory.create())) + .add( + PTransformOverride.of( + PTransformMatchers.urnEqualTo(PTransformTranslation.CREATE_VIEW_TRANSFORM_URN), + new ViewOverrideFactory())) /* Uses pardos and GBKs */ + .add( + PTransformOverride.of( + PTransformMatchers.urnEqualTo(PTransformTranslation.TEST_STREAM_TRANSFORM_URN), + new DirectTestStreamFactory(this))) /* primitive */ + // SplittableParMultiDo is implemented in terms of nonsplittable simple ParDos and extra + // primitives + .add( + PTransformOverride.of( + PTransformMatchers.splittableParDo(), new ParDoMultiOverrideFactory())) + // state and timer pardos are implemented in terms of simple ParDos and extra primitives + .add( + PTransformOverride.of( + PTransformMatchers.stateOrTimerParDo(), new ParDoMultiOverrideFactory())) + .add( + PTransformOverride.of( + PTransformMatchers.urnEqualTo( + SplittableParDo.SPLITTABLE_PROCESS_KEYED_ELEMENTS_URN), + new SplittableParDoViaKeyedWorkItems.OverrideFactory())) + .add( + PTransformOverride.of( + PTransformMatchers.urnEqualTo(SplittableParDo.SPLITTABLE_GBKIKWI_URN), + new DirectGBKIntoKeyedWorkItemsOverrideFactory())) /* Returns a GBKO */ + .add( + PTransformOverride.of( + PTransformMatchers.urnEqualTo(PTransformTranslation.GROUP_BY_KEY_TRANSFORM_URN), + new DirectGroupByKeyOverrideFactory())); /* returns two chained primitives. */ return builder.build(); } diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/MultiStepCombine.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/MultiStepCombine.java new file mode 100644 index 0000000000000..6f49e9449fed5 --- /dev/null +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/MultiStepCombine.java @@ -0,0 +1,423 @@ +/* + * 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 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.collect.Iterables; +import java.io.IOException; +import java.util.Collections; +import java.util.LinkedHashMap; +import java.util.Map; +import java.util.Objects; +import javax.annotation.Nullable; +import org.apache.beam.runners.core.construction.CombineTranslation; +import org.apache.beam.runners.core.construction.PTransformTranslation; +import org.apache.beam.runners.core.construction.PTransformTranslation.RawPTransform; +import org.apache.beam.runners.core.construction.SingleInputOutputOverrideFactory; +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.AppliedPTransform; +import org.apache.beam.sdk.runners.PTransformMatcher; +import org.apache.beam.sdk.runners.PTransformOverrideFactory; +import org.apache.beam.sdk.transforms.Combine; +import org.apache.beam.sdk.transforms.Combine.CombineFn; +import org.apache.beam.sdk.transforms.Combine.PerKey; +import org.apache.beam.sdk.transforms.CombineFnBase.GlobalCombineFn; +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.windowing.BoundedWindow; +import org.apache.beam.sdk.transforms.windowing.DefaultTrigger; +import org.apache.beam.sdk.transforms.windowing.TimestampCombiner; +import org.apache.beam.sdk.util.UserCodeException; +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.PValue; +import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowingStrategy; +import org.joda.time.Instant; + +/** A {@link Combine} that performs the combine in multiple steps. */ +class MultiStepCombine + extends RawPTransform>, PCollection>> { + public static PTransformMatcher matcher() { + return new PTransformMatcher() { + @Override + public boolean matches(AppliedPTransform application) { + if (PTransformTranslation.COMBINE_TRANSFORM_URN.equals( + PTransformTranslation.urnForTransformOrNull(application.getTransform()))) { + try { + GlobalCombineFn fn = CombineTranslation.getCombineFn(application); + return isApplicable(application.getInputs(), fn); + } catch (IOException e) { + throw new RuntimeException(e); + } + } + return false; + } + + private boolean isApplicable( + Map, PValue> inputs, GlobalCombineFn fn) { + if (!(fn instanceof CombineFn)) { + return false; + } + if (inputs.size() == 1) { + PCollection> input = + (PCollection>) Iterables.getOnlyElement(inputs.values()); + WindowingStrategy windowingStrategy = input.getWindowingStrategy(); + boolean windowFnApplicable = windowingStrategy.getWindowFn().isNonMerging(); + // Triggering with count based triggers is not appropriately handled here. Disabling + // most triggers is safe, though more broad than is technically required. + boolean triggerApplicable = DefaultTrigger.of().equals(windowingStrategy.getTrigger()); + boolean accumulatorCoderAvailable; + try { + if (input.getCoder() instanceof KvCoder) { + KvCoder kvCoder = (KvCoder) input.getCoder(); + Coder accumulatorCoder = + fn.getAccumulatorCoder( + input.getPipeline().getCoderRegistry(), kvCoder.getValueCoder()); + accumulatorCoderAvailable = accumulatorCoder != null; + } else { + accumulatorCoderAvailable = false; + } + } catch (CannotProvideCoderException e) { + throw new RuntimeException( + String.format( + "Could not construct an accumulator %s for %s. Accumulator %s for a %s may be" + + " null, but may not throw an exception", + Coder.class.getSimpleName(), + fn, + Coder.class.getSimpleName(), + Combine.class.getSimpleName()), + e); + } + return windowFnApplicable && triggerApplicable && accumulatorCoderAvailable; + } + return false; + } + }; + } + + static class Factory + extends SingleInputOutputOverrideFactory< + PCollection>, PCollection>, + PTransform>, PCollection>>> { + public static PTransformOverrideFactory create() { + return new Factory<>(); + } + + private Factory() {} + + @Override + public PTransformReplacement>, PCollection>> + getReplacementTransform( + AppliedPTransform< + PCollection>, PCollection>, + PTransform>, PCollection>>> + transform) { + try { + GlobalCombineFn globalFn = CombineTranslation.getCombineFn(transform); + checkState( + globalFn instanceof CombineFn, + "%s.matcher() should only match %s instances using %s, got %s", + MultiStepCombine.class.getSimpleName(), + PerKey.class.getSimpleName(), + CombineFn.class.getSimpleName(), + globalFn.getClass().getName()); + @SuppressWarnings("unchecked") + CombineFn fn = (CombineFn) globalFn; + @SuppressWarnings("unchecked") + PCollection> input = + (PCollection>) Iterables.getOnlyElement(transform.getInputs().values()); + @SuppressWarnings("unchecked") + PCollection> output = + (PCollection>) Iterables.getOnlyElement(transform.getOutputs().values()); + return PTransformReplacement.of(input, new MultiStepCombine<>(fn, output.getCoder())); + } catch (IOException e) { + throw new RuntimeException(e); + } + } + } + + // =========================================================================================== + + private final CombineFn combineFn; + private final Coder> outputCoder; + + public static MultiStepCombine of( + CombineFn combineFn, Coder> outputCoder) { + return new MultiStepCombine<>(combineFn, outputCoder); + } + + private MultiStepCombine( + CombineFn combineFn, Coder> outputCoder) { + this.combineFn = combineFn; + this.outputCoder = outputCoder; + } + + @Nullable + @Override + public String getUrn() { + return "urn:beam:directrunner:transforms:multistepcombine:v1"; + } + + @Override + public PCollection> expand(PCollection> input) { + checkArgument( + input.getCoder() instanceof KvCoder, + "Expected input to have a %s of type %s, got %s", + Coder.class.getSimpleName(), + KvCoder.class.getSimpleName(), + input.getCoder()); + KvCoder inputCoder = (KvCoder) input.getCoder(); + Coder inputValueCoder = inputCoder.getValueCoder(); + Coder accumulatorCoder; + try { + accumulatorCoder = + combineFn.getAccumulatorCoder(input.getPipeline().getCoderRegistry(), inputValueCoder); + } catch (CannotProvideCoderException e) { + throw new IllegalStateException( + String.format( + "Could not construct an Accumulator Coder with the provided %s %s", + CombineFn.class.getSimpleName(), combineFn), + e); + } + return input + .apply( + ParDo.of( + new CombineInputs<>( + combineFn, + input.getWindowingStrategy().getTimestampCombiner(), + inputCoder.getKeyCoder()))) + .setCoder(KvCoder.of(inputCoder.getKeyCoder(), accumulatorCoder)) + .apply(GroupByKey.create()) + .apply(new MergeAndExtractAccumulatorOutput(combineFn)) + .setCoder(outputCoder); + } + + private static class CombineInputs extends DoFn, KV> { + private final CombineFn combineFn; + private final TimestampCombiner timestampCombiner; + private final Coder keyCoder; + + /** + * Per-bundle state. Accumulators and output timestamps should only be tracked while a bundle + * is being processed, and must be cleared when a bundle is completed. + */ + private transient Map, AccumT> accumulators; + private transient Map, Instant> timestamps; + + private CombineInputs( + CombineFn combineFn, + TimestampCombiner timestampCombiner, + Coder keyCoder) { + this.combineFn = combineFn; + this.timestampCombiner = timestampCombiner; + this.keyCoder = keyCoder; + } + + @StartBundle + public void startBundle() { + accumulators = new LinkedHashMap<>(); + timestamps = new LinkedHashMap<>(); + } + + @ProcessElement + public void processElement(ProcessContext context, BoundedWindow window) { + WindowedStructuralKey + key = WindowedStructuralKey.create(keyCoder, context.element().getKey(), window); + AccumT accumulator = accumulators.get(key); + Instant assignedTs = timestampCombiner.assign(window, context.timestamp()); + if (accumulator == null) { + accumulator = combineFn.createAccumulator(); + accumulators.put(key, accumulator); + timestamps.put(key, assignedTs); + } + accumulators.put(key, combineFn.addInput(accumulator, context.element().getValue())); + timestamps.put(key, timestampCombiner.combine(assignedTs, timestamps.get(key))); + } + + @FinishBundle + public void outputAccumulators(FinishBundleContext context) { + for (Map.Entry, AccumT> preCombineEntry : accumulators.entrySet()) { + context.output( + KV.of(preCombineEntry.getKey().getKey(), combineFn.compact(preCombineEntry.getValue())), + timestamps.get(preCombineEntry.getKey()), + preCombineEntry.getKey().getWindow()); + } + accumulators = null; + timestamps = null; + } + } + + static class WindowedStructuralKey { + public static WindowedStructuralKey create( + Coder keyCoder, K key, BoundedWindow window) { + return new WindowedStructuralKey<>(StructuralKey.of(key, keyCoder), window); + } + + private final StructuralKey key; + private final BoundedWindow window; + + private WindowedStructuralKey(StructuralKey key, BoundedWindow window) { + this.key = checkNotNull(key, "key cannot be null"); + this.window = checkNotNull(window, "Window cannot be null"); + } + + public K getKey() { + return key.getKey(); + } + + public BoundedWindow getWindow() { + return window; + } + + @Override + public boolean equals(Object other) { + if (!(other instanceof MultiStepCombine.WindowedStructuralKey)) { + return false; + } + WindowedStructuralKey that = (WindowedStructuralKey) other; + return this.window.equals(that.window) && this.key.equals(that.key); + } + + @Override + public int hashCode() { + return Objects.hash(window, key); + } + } + + static final String DIRECT_MERGE_ACCUMULATORS_EXTRACT_OUTPUT_URN = + "urn:beam:directrunner:transforms:merge_accumulators_extract_output:v1"; + /** + * A primitive {@link PTransform} that merges iterables of accumulators and extracts the output. + * + *

              Required to ensure that Immutability Enforcement is not applied. Accumulators + * are explicitly mutable. + */ + static class MergeAndExtractAccumulatorOutput + extends RawPTransform>>, PCollection>> { + private final CombineFn combineFn; + + private MergeAndExtractAccumulatorOutput(CombineFn combineFn) { + this.combineFn = combineFn; + } + + CombineFn getCombineFn() { + return combineFn; + } + + @Override + public PCollection> expand(PCollection>> input) { + return PCollection.createPrimitiveOutputInternal( + input.getPipeline(), input.getWindowingStrategy(), input.isBounded()); + } + + @Nullable + @Override + public String getUrn() { + return DIRECT_MERGE_ACCUMULATORS_EXTRACT_OUTPUT_URN; + } + } + + static class MergeAndExtractAccumulatorOutputEvaluatorFactory + implements TransformEvaluatorFactory { + private final EvaluationContext ctxt; + + public MergeAndExtractAccumulatorOutputEvaluatorFactory(EvaluationContext ctxt) { + this.ctxt = ctxt; + } + + @Nullable + @Override + public TransformEvaluator forApplication( + AppliedPTransform application, CommittedBundle inputBundle) throws Exception { + return createEvaluator((AppliedPTransform) application, (CommittedBundle) inputBundle); + } + + private TransformEvaluator>> createEvaluator( + AppliedPTransform< + PCollection>>, PCollection>, + MergeAndExtractAccumulatorOutput> + application, + CommittedBundle>> inputBundle) { + return new MergeAccumulatorsAndExtractOutputEvaluator<>(ctxt, application); + } + + @Override + public void cleanup() throws Exception {} + } + + private static class MergeAccumulatorsAndExtractOutputEvaluator + implements TransformEvaluator>> { + private final AppliedPTransform< + PCollection>>, PCollection>, + MergeAndExtractAccumulatorOutput> + application; + private final CombineFn combineFn; + private final UncommittedBundle> output; + + public MergeAccumulatorsAndExtractOutputEvaluator( + EvaluationContext ctxt, + AppliedPTransform< + PCollection>>, PCollection>, + MergeAndExtractAccumulatorOutput> + application) { + this.application = application; + this.combineFn = application.getTransform().getCombineFn(); + this.output = + ctxt.createBundle( + (PCollection>) + Iterables.getOnlyElement(application.getOutputs().values())); + } + + @Override + public void processElement(WindowedValue>> element) throws Exception { + checkState( + element.getWindows().size() == 1, + "Expected inputs to %s to be in exactly one window. Got %s", + MergeAccumulatorsAndExtractOutputEvaluator.class.getSimpleName(), + element.getWindows().size()); + Iterable inputAccumulators = element.getValue().getValue(); + try { + AccumT first = combineFn.createAccumulator(); + AccumT merged = combineFn.mergeAccumulators(Iterables.concat(Collections.singleton(first), + inputAccumulators, + Collections.singleton(combineFn.createAccumulator()))); + OutputT extracted = combineFn.extractOutput(merged); + output.add(element.withValue(KV.of(element.getValue().getKey(), extracted))); + } catch (Exception e) { + throw UserCodeException.wrap(e); + } + } + + @Override + public TransformResult>> finishBundle() throws Exception { + return StepTransformResult.>>withoutHold(application) + .addOutput(output) + .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 0c907dfe9f4f8..30666db2eb2b5 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 @@ -26,6 +26,7 @@ import static org.apache.beam.runners.core.construction.SplittableParDo.SPLITTABLE_PROCESS_URN; import static org.apache.beam.runners.direct.DirectGroupByKey.DIRECT_GABW_URN; import static org.apache.beam.runners.direct.DirectGroupByKey.DIRECT_GBKO_URN; +import static org.apache.beam.runners.direct.MultiStepCombine.DIRECT_MERGE_ACCUMULATORS_EXTRACT_OUTPUT_URN; import static org.apache.beam.runners.direct.ParDoMultiOverrideFactory.DIRECT_STATEFUL_PAR_DO_URN; import static org.apache.beam.runners.direct.TestStreamEvaluatorFactory.DirectTestStreamFactory.DIRECT_TEST_STREAM_URN; import static org.apache.beam.runners.direct.ViewOverrideFactory.DIRECT_WRITE_VIEW_URN; @@ -73,6 +74,9 @@ public static TransformEvaluatorRegistry defaultRegistry(EvaluationContext ctxt) .put(DIRECT_GBKO_URN, new GroupByKeyOnlyEvaluatorFactory(ctxt)) .put(DIRECT_GABW_URN, new GroupAlsoByWindowEvaluatorFactory(ctxt)) .put(DIRECT_TEST_STREAM_URN, new TestStreamEvaluatorFactory(ctxt)) + .put( + DIRECT_MERGE_ACCUMULATORS_EXTRACT_OUTPUT_URN, + new MultiStepCombine.MergeAndExtractAccumulatorOutputEvaluatorFactory(ctxt)) // Runners-core primitives .put(SPLITTABLE_PROCESS_URN, new SplittableProcessElementsEvaluatorFactory<>(ctxt)) diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/MultiStepCombineTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/MultiStepCombineTest.java new file mode 100644 index 0000000000000..0c11a8ad00a5a --- /dev/null +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/MultiStepCombineTest.java @@ -0,0 +1,228 @@ +/* + * 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.is; +import static org.junit.Assert.assertThat; + +import com.google.auto.value.AutoValue; +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.io.Serializable; +import org.apache.beam.sdk.coders.CannotProvideCoderException; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.coders.CoderException; +import org.apache.beam.sdk.coders.CoderRegistry; +import org.apache.beam.sdk.coders.CustomCoder; +import org.apache.beam.sdk.coders.KvCoder; +import org.apache.beam.sdk.coders.StringUtf8Coder; +import org.apache.beam.sdk.coders.VarLongCoder; +import org.apache.beam.sdk.testing.PAssert; +import org.apache.beam.sdk.testing.TestPipeline; +import org.apache.beam.sdk.transforms.Combine; +import org.apache.beam.sdk.transforms.Combine.CombineFn; +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.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.TimestampCombiner; +import org.apache.beam.sdk.transforms.windowing.Window; +import org.apache.beam.sdk.util.VarInt; +import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.TimestampedValue; +import org.joda.time.Duration; +import org.joda.time.Instant; +import org.junit.Rule; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +/** + * Tests for {@link MultiStepCombine}. + */ +@RunWith(JUnit4.class) +public class MultiStepCombineTest implements Serializable { + @Rule public transient TestPipeline pipeline = TestPipeline.create(); + + private transient KvCoder combinedCoder = + KvCoder.of(StringUtf8Coder.of(), VarLongCoder.of()); + + @Test + public void testMultiStepCombine() { + PCollection> combined = + pipeline + .apply( + Create.of( + KV.of("foo", 1L), + KV.of("bar", 2L), + KV.of("bizzle", 3L), + KV.of("bar", 4L), + KV.of("bizzle", 11L))) + .apply(Combine.perKey(new MultiStepCombineFn())); + + PAssert.that(combined) + .containsInAnyOrder(KV.of("foo", 1L), KV.of("bar", 6L), KV.of("bizzle", 14L)); + pipeline.run(); + } + + @Test + public void testMultiStepCombineWindowed() { + SlidingWindows windowFn = SlidingWindows.of(Duration.millis(6L)).every(Duration.millis(3L)); + PCollection> combined = + pipeline + .apply( + Create.timestamped( + TimestampedValue.of(KV.of("foo", 1L), new Instant(1L)), + TimestampedValue.of(KV.of("bar", 2L), new Instant(2L)), + TimestampedValue.of(KV.of("bizzle", 3L), new Instant(3L)), + TimestampedValue.of(KV.of("bar", 4L), new Instant(4L)), + TimestampedValue.of(KV.of("bizzle", 11L), new Instant(11L)))) + .apply(Window.>into(windowFn)) + .apply(Combine.perKey(new MultiStepCombineFn())); + + PAssert.that("Windows should combine only elements in their windows", combined) + .inWindow(new IntervalWindow(new Instant(0L), Duration.millis(6L))) + .containsInAnyOrder(KV.of("foo", 1L), KV.of("bar", 6L), KV.of("bizzle", 3L)); + PAssert.that("Elements should appear in all the windows they are assigned to", combined) + .inWindow(new IntervalWindow(new Instant(-3L), Duration.millis(6L))) + .containsInAnyOrder(KV.of("foo", 1L), KV.of("bar", 2L)); + PAssert.that(combined) + .inWindow(new IntervalWindow(new Instant(6L), Duration.millis(6L))) + .containsInAnyOrder(KV.of("bizzle", 11L)); + PAssert.that(combined) + .containsInAnyOrder( + KV.of("foo", 1L), + KV.of("foo", 1L), + KV.of("bar", 6L), + KV.of("bar", 2L), + KV.of("bar", 4L), + KV.of("bizzle", 11L), + KV.of("bizzle", 11L), + KV.of("bizzle", 3L), + KV.of("bizzle", 3L)); + pipeline.run(); + } + + @Test + public void testMultiStepCombineTimestampCombiner() { + TimestampCombiner combiner = TimestampCombiner.LATEST; + combinedCoder = KvCoder.of(StringUtf8Coder.of(), VarLongCoder.of()); + PCollection> combined = + pipeline + .apply( + Create.timestamped( + TimestampedValue.of(KV.of("foo", 4L), new Instant(1L)), + TimestampedValue.of(KV.of("foo", 1L), new Instant(4L)), + TimestampedValue.of(KV.of("bazzle", 4L), new Instant(4L)), + TimestampedValue.of(KV.of("foo", 12L), new Instant(12L)))) + .apply( + Window.>into(FixedWindows.of(Duration.millis(5L))) + .withTimestampCombiner(combiner)) + .apply(Combine.perKey(new MultiStepCombineFn())); + PCollection>> reified = + combined.apply( + ParDo.of( + new DoFn, KV>>() { + @ProcessElement + public void reifyTimestamp(ProcessContext context) { + context.output( + KV.of( + context.element().getKey(), + TimestampedValue.of( + context.element().getValue(), context.timestamp()))); + } + })); + + PAssert.that(reified) + .containsInAnyOrder( + KV.of("foo", TimestampedValue.of(5L, new Instant(4L))), + KV.of("bazzle", TimestampedValue.of(4L, new Instant(4L))), + KV.of("foo", TimestampedValue.of(12L, new Instant(12L)))); + pipeline.run(); + } + + private static class MultiStepCombineFn extends CombineFn { + @Override + public Coder getAccumulatorCoder( + CoderRegistry registry, Coder inputCoder) throws CannotProvideCoderException { + return new MultiStepAccumulatorCoder(); + } + + @Override + public MultiStepAccumulator createAccumulator() { + return MultiStepAccumulator.of(0L, false); + } + + @Override + public MultiStepAccumulator addInput(MultiStepAccumulator accumulator, Long input) { + return MultiStepAccumulator.of(accumulator.getValue() + input, accumulator.isDeserialized()); + } + + @Override + public MultiStepAccumulator mergeAccumulators(Iterable accumulators) { + MultiStepAccumulator result = MultiStepAccumulator.of(0L, false); + for (MultiStepAccumulator accumulator : accumulators) { + result = result.merge(accumulator); + } + return result; + } + + @Override + public Long extractOutput(MultiStepAccumulator accumulator) { + assertThat( + "Accumulators should have been serialized and deserialized within the Pipeline", + accumulator.isDeserialized(), + is(true)); + return accumulator.getValue(); + } + } + + @AutoValue + abstract static class MultiStepAccumulator { + private static MultiStepAccumulator of(long value, boolean deserialized) { + return new AutoValue_MultiStepCombineTest_MultiStepAccumulator(value, deserialized); + } + + MultiStepAccumulator merge(MultiStepAccumulator other) { + return MultiStepAccumulator.of( + this.getValue() + other.getValue(), this.isDeserialized() || other.isDeserialized()); + } + + abstract long getValue(); + + abstract boolean isDeserialized(); + } + + private static class MultiStepAccumulatorCoder extends CustomCoder { + @Override + public void encode(MultiStepAccumulator value, OutputStream outStream) + throws CoderException, IOException { + VarInt.encode(value.getValue(), outStream); + } + + @Override + public MultiStepAccumulator decode(InputStream inStream) throws CoderException, IOException { + return MultiStepAccumulator.of(VarInt.decodeLong(inStream), true); + } + } +} From 9f2622fa19da1284222e872fdcd63b086bdc3509 Mon Sep 17 00:00:00 2001 From: Reuven Lax Date: Thu, 6 Jul 2017 20:22:25 -0700 Subject: [PATCH 216/346] [BEAM-92] Supports DynamicDestinations in AvroIO. --- .../core/construction/ParDoTranslation.java | 2 +- .../construction/WriteFilesTranslation.java | 81 ++-- .../construction/PTransformMatchersTest.java | 10 +- .../WriteFilesTranslationTest.java | 26 +- .../direct/WriteWithShardingFactory.java | 10 +- .../direct/WriteWithShardingFactoryTest.java | 8 +- .../beam/runners/dataflow/DataflowRunner.java | 8 +- .../runners/dataflow/DataflowRunnerTest.java | 10 +- .../src/main/proto/beam_runner_api.proto | 2 + .../java/org/apache/beam/sdk/io/AvroIO.java | 436 ++++++++++++++---- .../java/org/apache/beam/sdk/io/AvroSink.java | 93 ++-- .../beam/sdk/io/ConstantAvroDestination.java | 130 ++++++ .../beam/sdk/io/DefaultFilenamePolicy.java | 1 - .../beam/sdk/io/DynamicAvroDestinations.java | 46 ++ .../beam/sdk/io/DynamicFileDestinations.java | 59 ++- .../org/apache/beam/sdk/io/FileBasedSink.java | 121 +++-- .../org/apache/beam/sdk/io/TFRecordIO.java | 23 +- .../java/org/apache/beam/sdk/io/TextIO.java | 228 +++++---- .../java/org/apache/beam/sdk/io/TextSink.java | 14 +- .../org/apache/beam/sdk/io/WriteFiles.java | 116 +++-- .../org/apache/beam/sdk/io/AvroIOTest.java | 156 ++++++- .../apache/beam/sdk/io/FileBasedSinkTest.java | 6 +- .../org/apache/beam/sdk/io/SimpleSink.java | 10 +- .../apache/beam/sdk/io/TextIOWriteTest.java | 23 +- .../apache/beam/sdk/io/WriteFilesTest.java | 74 +-- .../org/apache/beam/sdk/io/xml/XmlIO.java | 4 +- .../org/apache/beam/sdk/io/xml/XmlSink.java | 8 +- 27 files changed, 1214 insertions(+), 491 deletions(-) create mode 100644 sdks/java/core/src/main/java/org/apache/beam/sdk/io/ConstantAvroDestination.java create mode 100644 sdks/java/core/src/main/java/org/apache/beam/sdk/io/DynamicAvroDestinations.java diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ParDoTranslation.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ParDoTranslation.java index d7b0e9f4e1a51..5765c51b26fbe 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ParDoTranslation.java +++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ParDoTranslation.java @@ -484,7 +484,7 @@ protected Optional dispatchDefault(Parameter p) { }); } - private static SideInput toProto(PCollectionView view) { + public static SideInput toProto(PCollectionView view) { Builder builder = SideInput.newBuilder(); builder.setAccessPattern( FunctionSpec.newBuilder() diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/WriteFilesTranslation.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/WriteFilesTranslation.java index b1d2da435b1e8..7954b0ea811e0 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/WriteFilesTranslation.java +++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/WriteFilesTranslation.java @@ -19,29 +19,35 @@ package org.apache.beam.runners.core.construction; import static com.google.common.base.Preconditions.checkArgument; +import static com.google.common.base.Preconditions.checkNotNull; import com.google.auto.service.AutoService; import com.google.common.annotations.VisibleForTesting; +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; import com.google.protobuf.Any; import com.google.protobuf.ByteString; import com.google.protobuf.BytesValue; import java.io.IOException; import java.io.Serializable; import java.util.Collections; +import java.util.List; import java.util.Map; import org.apache.beam.runners.core.construction.PTransformTranslation.TransformPayloadTranslator; import org.apache.beam.sdk.common.runner.v1.RunnerApi; import org.apache.beam.sdk.common.runner.v1.RunnerApi.FunctionSpec; import org.apache.beam.sdk.common.runner.v1.RunnerApi.SdkFunctionSpec; +import org.apache.beam.sdk.common.runner.v1.RunnerApi.SideInput; import org.apache.beam.sdk.common.runner.v1.RunnerApi.WriteFilesPayload; import org.apache.beam.sdk.io.FileBasedSink; import org.apache.beam.sdk.io.WriteFiles; import org.apache.beam.sdk.runners.AppliedPTransform; import org.apache.beam.sdk.transforms.PTransform; -import org.apache.beam.sdk.transforms.SerializableFunction; import org.apache.beam.sdk.util.SerializableUtils; import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.PDone; +import org.apache.beam.sdk.values.TupleTag; /** * Utility methods for translating a {@link WriteFiles} to and from {@link RunnerApi} @@ -53,28 +59,25 @@ public class WriteFilesTranslation { public static final String CUSTOM_JAVA_FILE_BASED_SINK_URN = "urn:beam:file_based_sink:javasdk:0.1"; - public static final String CUSTOM_JAVA_FILE_BASED_SINK_FORMAT_FUNCTION_URN = - "urn:beam:file_based_sink_format_function:javasdk:0.1"; - @VisibleForTesting static WriteFilesPayload toProto(WriteFiles transform) { + Map sideInputs = Maps.newHashMap(); + for (PCollectionView view : transform.getSink().getDynamicDestinations().getSideInputs()) { + sideInputs.put(view.getTagInternal().getId(), ParDoTranslation.toProto(view)); + } return WriteFilesPayload.newBuilder() .setSink(toProto(transform.getSink())) - .setFormatFunction(toProto(transform.getFormatFunction())) .setWindowedWrites(transform.isWindowedWrites()) .setRunnerDeterminedSharding( transform.getNumShards() == null && transform.getSharding() == null) + .putAllSideInputs(sideInputs) .build(); } - private static SdkFunctionSpec toProto(FileBasedSink sink) { + private static SdkFunctionSpec toProto(FileBasedSink sink) { return toProto(CUSTOM_JAVA_FILE_BASED_SINK_URN, sink); } - private static SdkFunctionSpec toProto(SerializableFunction serializableFunction) { - return toProto(CUSTOM_JAVA_FILE_BASED_SINK_FORMAT_FUNCTION_URN, serializableFunction); - } - private static SdkFunctionSpec toProto(String urn, Serializable serializable) { return SdkFunctionSpec.newBuilder() .setSpec( @@ -91,7 +94,7 @@ private static SdkFunctionSpec toProto(String urn, Serializable serializable) { } @VisibleForTesting - static FileBasedSink sinkFromProto(SdkFunctionSpec sinkProto) throws IOException { + static FileBasedSink sinkFromProto(SdkFunctionSpec sinkProto) throws IOException { checkArgument( sinkProto.getSpec().getUrn().equals(CUSTOM_JAVA_FILE_BASED_SINK_URN), "Cannot extract %s instance from %s with URN %s", @@ -102,44 +105,44 @@ private static SdkFunctionSpec toProto(String urn, Serializable serializable) { byte[] serializedSink = sinkProto.getSpec().getParameter().unpack(BytesValue.class).getValue().toByteArray(); - return (FileBasedSink) + return (FileBasedSink) SerializableUtils.deserializeFromByteArray( serializedSink, FileBasedSink.class.getSimpleName()); } - @VisibleForTesting - static SerializableFunction formatFunctionFromProto( - SdkFunctionSpec sinkProto) throws IOException { - checkArgument( - sinkProto.getSpec().getUrn().equals(CUSTOM_JAVA_FILE_BASED_SINK_FORMAT_FUNCTION_URN), - "Cannot extract %s instance from %s with URN %s", - SerializableFunction.class.getSimpleName(), - FunctionSpec.class.getSimpleName(), - sinkProto.getSpec().getUrn()); - - byte[] serializedFunction = - sinkProto.getSpec().getParameter().unpack(BytesValue.class).getValue().toByteArray(); - - return (SerializableFunction) - SerializableUtils.deserializeFromByteArray( - serializedFunction, FileBasedSink.class.getSimpleName()); - } - - public static FileBasedSink getSink( + public static FileBasedSink getSink( AppliedPTransform, PDone, ? extends PTransform, PDone>> transform) throws IOException { - return (FileBasedSink) + return (FileBasedSink) sinkFromProto(getWriteFilesPayload(transform).getSink()); } - public static SerializableFunction getFormatFunction( - AppliedPTransform< - PCollection, PDone, ? extends PTransform, PDone>> - transform) - throws IOException { - return formatFunctionFromProto( - getWriteFilesPayload(transform).getFormatFunction()); + public static + List> getDynamicDestinationSideInputs( + AppliedPTransform< + PCollection, PDone, ? extends PTransform, PDone>> + transform) + throws IOException { + SdkComponents sdkComponents = SdkComponents.create(); + RunnerApi.PTransform transformProto = PTransformTranslation.toProto(transform, sdkComponents); + List> views = Lists.newArrayList(); + Map sideInputs = getWriteFilesPayload(transform).getSideInputsMap(); + for (Map.Entry entry : sideInputs.entrySet()) { + PCollection originalPCollection = + checkNotNull( + (PCollection) transform.getInputs().get(new TupleTag<>(entry.getKey())), + "no input with tag %s", + entry.getKey()); + views.add( + ParDoTranslation.viewFromProto( + entry.getValue(), + entry.getKey(), + originalPCollection, + transformProto, + RehydratedComponents.forComponents(sdkComponents.toComponents()))); + } + return views; } public static boolean isWindowedWrites( diff --git a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/PTransformMatchersTest.java b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/PTransformMatchersTest.java index 316645b2d5392..18626997000a3 100644 --- a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/PTransformMatchersTest.java +++ b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/PTransformMatchersTest.java @@ -57,7 +57,6 @@ import org.apache.beam.sdk.transforms.Materializations; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.ParDo; -import org.apache.beam.sdk.transforms.SerializableFunctions; import org.apache.beam.sdk.transforms.Sum; import org.apache.beam.sdk.transforms.View; import org.apache.beam.sdk.transforms.View.CreatePCollectionView; @@ -549,15 +548,14 @@ public void writeWithRunnerDeterminedSharding() { false); WriteFiles write = WriteFiles.to( - new FileBasedSink( + new FileBasedSink( StaticValueProvider.of(outputDirectory), - DynamicFileDestinations.constant(new FakeFilenamePolicy())) { + DynamicFileDestinations.constant(new FakeFilenamePolicy())) { @Override - public WriteOperation createWriteOperation() { + public WriteOperation createWriteOperation() { return null; } - }, - SerializableFunctions.identity()); + }); assertThat( PTransformMatchers.writeWithRunnerDeterminedSharding().matches(appliedWrite(write)), is(true)); diff --git a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/WriteFilesTranslationTest.java b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/WriteFilesTranslationTest.java index 4259ac893b64c..e067facb56923 100644 --- a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/WriteFilesTranslationTest.java +++ b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/WriteFilesTranslationTest.java @@ -38,7 +38,6 @@ import org.apache.beam.sdk.testing.TestPipeline; import org.apache.beam.sdk.transforms.Create; import org.apache.beam.sdk.transforms.ParDo; -import org.apache.beam.sdk.transforms.SerializableFunction; import org.apache.beam.sdk.transforms.SerializableFunctions; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; @@ -63,12 +62,11 @@ public class WriteFilesTranslationTest { public static class TestWriteFilesPayloadTranslation { @Parameters(name = "{index}: {0}") public static Iterable> data() { - SerializableFunction format = SerializableFunctions.constant(null); return ImmutableList.of( - WriteFiles.to(new DummySink(), format), - WriteFiles.to(new DummySink(), format).withWindowedWrites(), - WriteFiles.to(new DummySink(), format).withNumShards(17), - WriteFiles.to(new DummySink(), format).withWindowedWrites().withNumShards(42)); + WriteFiles.to(new DummySink()), + WriteFiles.to(new DummySink()).withWindowedWrites(), + WriteFiles.to(new DummySink()).withNumShards(17), + WriteFiles.to(new DummySink()).withWindowedWrites().withNumShards(42)); } @Parameter(0) @@ -87,7 +85,8 @@ public void testEncodedProto() throws Exception { assertThat(payload.getWindowedWrites(), equalTo(writeFiles.isWindowedWrites())); assertThat( - (FileBasedSink) WriteFilesTranslation.sinkFromProto(payload.getSink()), + (FileBasedSink) + WriteFilesTranslation.sinkFromProto(payload.getSink()), equalTo(writeFiles.getSink())); } @@ -118,16 +117,17 @@ public void testExtractionDirectFromTransform() throws Exception { * A simple {@link FileBasedSink} for testing serialization/deserialization. Not mocked to avoid * any issues serializing mocks. */ - private static class DummySink extends FileBasedSink { + private static class DummySink extends FileBasedSink { DummySink() { super( StaticValueProvider.of(FileSystems.matchNewResource("nowhere", false)), - DynamicFileDestinations.constant(new DummyFilenamePolicy())); + DynamicFileDestinations.constant( + new DummyFilenamePolicy(), SerializableFunctions.constant(null))); } @Override - public WriteOperation createWriteOperation() { + public WriteOperation createWriteOperation() { return new DummyWriteOperation(this); } @@ -152,13 +152,13 @@ public int hashCode() { } } - private static class DummyWriteOperation extends FileBasedSink.WriteOperation { - public DummyWriteOperation(FileBasedSink sink) { + private static class DummyWriteOperation extends FileBasedSink.WriteOperation { + public DummyWriteOperation(FileBasedSink sink) { super(sink); } @Override - public FileBasedSink.Writer createWriter() throws Exception { + public FileBasedSink.Writer createWriter() throws Exception { throw new UnsupportedOperationException("Should never be called."); } } 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 ba796ae745626..3557c5dfe3833 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 @@ -24,10 +24,12 @@ import java.io.IOException; import java.io.Serializable; import java.util.Collections; +import java.util.List; import java.util.Map; import java.util.concurrent.ThreadLocalRandom; import org.apache.beam.runners.core.construction.PTransformReplacements; import org.apache.beam.runners.core.construction.WriteFilesTranslation; +import org.apache.beam.sdk.io.FileBasedSink; import org.apache.beam.sdk.io.WriteFiles; import org.apache.beam.sdk.runners.AppliedPTransform; import org.apache.beam.sdk.runners.PTransformOverrideFactory; @@ -61,10 +63,10 @@ public PTransformReplacement, PDone> getReplacementTransform AppliedPTransform, PDone, PTransform, PDone>> transform) { try { - WriteFiles replacement = - WriteFiles.to( - WriteFilesTranslation.getSink(transform), - WriteFilesTranslation.getFormatFunction(transform)); + List> sideInputs = + WriteFilesTranslation.getDynamicDestinationSideInputs(transform); + FileBasedSink sink = WriteFilesTranslation.getSink(transform); + WriteFiles replacement = WriteFiles.to(sink).withSideInputs(sideInputs); if (WriteFilesTranslation.isWindowedWrites(transform)) { replacement = replacement.withWindowedWrites(); } 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 6dd069caa9033..d0db44ef261fc 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 @@ -55,7 +55,6 @@ import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.DoFnTester; import org.apache.beam.sdk.transforms.PTransform; -import org.apache.beam.sdk.transforms.SerializableFunctions; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.GlobalWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; @@ -143,15 +142,14 @@ public void withNoShardingSpecifiedReturnsNewTransform() { PTransform, PDone> original = WriteFiles.to( - new FileBasedSink( + new FileBasedSink( StaticValueProvider.of(outputDirectory), DynamicFileDestinations.constant(new FakeFilenamePolicy())) { @Override - public WriteOperation createWriteOperation() { + public WriteOperation createWriteOperation() { throw new IllegalArgumentException("Should not be used"); } - }, - SerializableFunctions.identity()); + }); @SuppressWarnings("unchecked") PCollection objs = (PCollection) p.apply(Create.empty(VoidCoder.of())); 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 762ac9fcd43f3..f8d2c3c12b652 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 @@ -92,6 +92,7 @@ import org.apache.beam.sdk.common.runner.v1.RunnerApi; import org.apache.beam.sdk.extensions.gcp.storage.PathValidator; import org.apache.beam.sdk.io.BoundedSource; +import org.apache.beam.sdk.io.FileBasedSink; import org.apache.beam.sdk.io.FileSystems; import org.apache.beam.sdk.io.Read; import org.apache.beam.sdk.io.UnboundedSource; @@ -1501,10 +1502,11 @@ public PTransformReplacement, PDone> getReplacementTransform( } try { + List> sideInputs = + WriteFilesTranslation.getDynamicDestinationSideInputs(transform); + FileBasedSink sink = WriteFilesTranslation.getSink(transform); WriteFiles replacement = - WriteFiles.to( - WriteFilesTranslation.getSink(transform), - WriteFilesTranslation.getFormatFunction(transform)); + WriteFiles.to(sink).withSideInputs(sideInputs); if (WriteFilesTranslation.isWindowedWrites(transform)) { replacement = replacement.withWindowedWrites(); } 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 7556a281f2c5b..9db73c6501b86 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 @@ -1271,8 +1271,7 @@ private void testStreamingWriteOverride(PipelineOptions options, int expectedNum StreamingShardedWriteFactory factory = new StreamingShardedWriteFactory<>(p.getOptions()); - WriteFiles original = - WriteFiles.to(new TestSink(tmpFolder.toString()), SerializableFunctions.identity()); + WriteFiles original = WriteFiles.to(new TestSink(tmpFolder.toString())); PCollection objs = (PCollection) p.apply(Create.empty(VoidCoder.of())); AppliedPTransform, PDone, WriteFiles> originalApplication = @@ -1290,7 +1289,7 @@ private void testStreamingWriteOverride(PipelineOptions options, int expectedNum assertThat(replacement.getNumShards().get(), equalTo(expectedNumShards)); } - private static class TestSink extends FileBasedSink { + private static class TestSink extends FileBasedSink { @Override public void validate(PipelineOptions options) {} @@ -1315,11 +1314,12 @@ public ResourceId unwindowedFilename( int shardNumber, int numShards, OutputFileHints outputFileHints) { throw new UnsupportedOperationException("should not be called"); } - })); + }, + SerializableFunctions.identity())); } @Override - public WriteOperation createWriteOperation() { + public WriteOperation createWriteOperation() { throw new IllegalArgumentException("Should not be used"); } } diff --git a/sdks/common/runner-api/src/main/proto/beam_runner_api.proto b/sdks/common/runner-api/src/main/proto/beam_runner_api.proto index 42e2601f0a31f..9afb565cca6d1 100644 --- a/sdks/common/runner-api/src/main/proto/beam_runner_api.proto +++ b/sdks/common/runner-api/src/main/proto/beam_runner_api.proto @@ -375,6 +375,8 @@ message WriteFilesPayload { bool windowed_writes = 3; bool runner_determined_sharding = 4; + + map side_inputs = 5; } // A coder, the binary format for serialization and deserialization of data in 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 27c9073b2937c..824f725fe7120 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 @@ -25,7 +25,6 @@ import com.google.common.base.Supplier; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Maps; -import com.google.common.io.BaseEncoding; import java.util.Map; import javax.annotation.Nullable; import org.apache.avro.Schema; @@ -40,7 +39,6 @@ import org.apache.beam.sdk.coders.CoderRegistry; import org.apache.beam.sdk.coders.StringUtf8Coder; import org.apache.beam.sdk.coders.VoidCoder; -import org.apache.beam.sdk.io.FileBasedSink.DynamicDestinations; import org.apache.beam.sdk.io.FileBasedSink.FilenamePolicy; import org.apache.beam.sdk.io.fs.ResourceId; import org.apache.beam.sdk.options.ValueProvider; @@ -51,7 +49,6 @@ import org.apache.beam.sdk.transforms.SerializableFunction; import org.apache.beam.sdk.transforms.SerializableFunctions; import org.apache.beam.sdk.transforms.display.DisplayData; -import org.apache.beam.sdk.transforms.display.HasDisplayData; import org.apache.beam.sdk.values.PBegin; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PDone; @@ -161,6 +158,51 @@ * .withSuffix(".avro")); * } * + *

              The following shows a more-complex example of AvroIO.Write usage, generating dynamic file + * destinations as well as a dynamic Avro schema per file. In this example, a PCollection of user + * events (e.g. actions on a website) is written out to Avro files. Each event contains the user id + * as an integer field. We want events for each user to go into a specific directory for that user, + * and each user's data should be written with a specific schema for that user; a side input is + * used, so the schema can be calculated in a different stage. + * + *

              {@code
              + * // This is the user class that controls dynamic destinations for this avro write. The input to
              + * // AvroIO.Write will be UserEvent, and we will be writing GenericRecords to the file (in order
              + * // to have dynamic schemas). Everything is per userid, so we define a dynamic destination type
              + * // of Integer.
              + * class UserDynamicAvroDestinations
              + *     extends DynamicAvroDestinations {
              + *   private final PCollectionView> userToSchemaMap;
              + *   public UserDynamicAvroDestinations( PCollectionView> userToSchemaMap) {
              + *     this.userToSchemaMap = userToSchemaMap;
              + *   }
              + *   public GenericRecord formatRecord(UserEvent record) {
              + *     return formatUserRecord(record, getSchema(record.getUserId()));
              + *   }
              + *   public Schema getSchema(Integer userId) {
              + *     return new Schema.Parser().parse(sideInput(userToSchemaMap).get(userId));
              + *   }
              + *   public Integer getDestination(UserEvent record) {
              + *     return record.getUserId();
              + *   }
              + *   public Integer getDefaultDestination() {
              + *     return 0;
              + *   }
              + *   public FilenamePolicy getFilenamePolicy(Integer userId) {
              + *     return DefaultFilenamePolicy.fromParams(new Params().withBaseFilename(baseDir + "/user-"
              + *     + userId + "/events"));
              + *   }
              + *   public List> getSideInputs() {
              + *     return ImmutableList.>of(userToSchemaMap);
              + *   }
              + * }
              + * PCollection events = ...;
              + * PCollectionView schemaMap = events.apply(
              + *     "ComputeSchemas", new ComputePerUserSchemas());
              + * events.apply("WriteAvros", AvroIO.writeCustomTypeToGenericRecords()
              + *     .to(new UserDynamicAvros()));
              + * }
              + * *

              By default, {@link AvroIO.Write} produces output files that are compressed using the {@link * org.apache.avro.file.Codec CodecFactory.deflateCodec(6)}. This default can be changed or * overridden using {@link AvroIO.Write#withCodec}. @@ -256,18 +298,53 @@ public static ParseAll parseAllGenericRecords( * pattern). */ public static Write write(Class recordClass) { - return AvroIO.defaultWriteBuilder() - .setRecordClass(recordClass) - .setSchema(ReflectData.get().getSchema(recordClass)) - .build(); + return new Write<>( + AvroIO.defaultWriteBuilder() + .setGenericRecords(false) + .setSchema(ReflectData.get().getSchema(recordClass)) + .build()); } /** Writes Avro records of the specified schema. */ public static Write writeGenericRecords(Schema schema) { - return AvroIO.defaultWriteBuilder() - .setRecordClass(GenericRecord.class) - .setSchema(schema) - .build(); + return new Write<>( + AvroIO.defaultWriteBuilder() + .setGenericRecords(true) + .setSchema(schema) + .build()); + } + + /** + * A {@link PTransform} that writes a {@link PCollection} to an avro file (or multiple avro files + * matching a sharding pattern), with each element of the input collection encoded into its own + * record of type OutputT. + * + *

              This version allows you to apply {@link AvroIO} writes to a PCollection of a custom type + * {@link UserT}. A format mechanism that converts the input type {@link UserT} to the output type + * that will be written to the file must be specified. If using a custom {@link + * DynamicAvroDestinations} object this is done using {@link + * DynamicAvroDestinations#formatRecord}, otherwise the {@link + * AvroIO.TypedWrite#withFormatFunction} can be used to specify a format function. + * + *

              The advantage of using a custom type is that is it allows a user-provided {@link + * DynamicAvroDestinations} object, set via {@link AvroIO.Write#to(DynamicAvroDestinations)} to + * examine the custom type when choosing a destination. + * + *

              If the output type is {@link GenericRecord} use {@link #writeCustomTypeToGenericRecords()} + * instead. + */ + public static TypedWrite writeCustomType() { + return AvroIO.defaultWriteBuilder().setGenericRecords(false).build(); + } + + /** + * Similar to {@link #writeCustomType()}, but specialized for the case where the output type is + * {@link GenericRecord}. A schema must be specified either in {@link + * DynamicAvroDestinations#getSchema} or if not using dynamic destinations, by using {@link + * TypedWrite#withSchema(Schema)}. + */ + public static TypedWrite writeCustomTypeToGenericRecords() { + return AvroIO.defaultWriteBuilder().setGenericRecords(true).build(); } /** @@ -277,12 +354,12 @@ public static Write writeGenericRecords(String schema) { return writeGenericRecords(new Schema.Parser().parse(schema)); } - private static Write.Builder defaultWriteBuilder() { - return new AutoValue_AvroIO_Write.Builder() + private static TypedWrite.Builder defaultWriteBuilder() { + return new AutoValue_AvroIO_TypedWrite.Builder() .setFilenameSuffix(null) .setShardTemplate(null) .setNumShards(0) - .setCodec(Write.DEFAULT_CODEC) + .setCodec(TypedWrite.DEFAULT_SERIALIZABLE_CODEC) .setMetadata(ImmutableMap.of()) .setWindowedWrites(false); } @@ -572,15 +649,18 @@ public void populateDisplayData(DisplayData.Builder builder) { } } - ///////////////////////////////////////////////////////////////////////////// + // /////////////////////////////////////////////////////////////////////////// /** Implementation of {@link #write}. */ @AutoValue - public abstract static class Write extends PTransform, PDone> { - private static final SerializableAvroCodecFactory DEFAULT_CODEC = - new SerializableAvroCodecFactory(CodecFactory.deflateCodec(6)); - // This should be a multiple of 4 to not get a partial encoded byte. - private static final int METADATA_BYTES_MAX_LENGTH = 40; + public abstract static class TypedWrite + extends PTransform, PDone> { + static final CodecFactory DEFAULT_CODEC = CodecFactory.deflateCodec(6); + static final SerializableAvroCodecFactory DEFAULT_SERIALIZABLE_CODEC = + new SerializableAvroCodecFactory(DEFAULT_CODEC); + + @Nullable + abstract SerializableFunction getFormatFunction(); @Nullable abstract ValueProvider getFilenamePrefix(); @Nullable abstract String getShardTemplate(); @@ -590,11 +670,16 @@ public abstract static class Write extends PTransform, PDone> abstract ValueProvider getTempDirectory(); abstract int getNumShards(); - @Nullable abstract Class getRecordClass(); + + abstract boolean getGenericRecords(); + @Nullable abstract Schema getSchema(); abstract boolean getWindowedWrites(); @Nullable abstract FilenamePolicy getFilenamePolicy(); + @Nullable + abstract DynamicAvroDestinations getDynamicDestinations(); + /** * The codec used to encode the blocks in the Avro file. String value drawn from those in * https://avro.apache.org/docs/1.7.7/api/java/org/apache/avro/file/CodecFactory.html @@ -603,25 +688,39 @@ public abstract static class Write extends PTransform, PDone> /** Avro file metadata. */ abstract ImmutableMap getMetadata(); - abstract Builder toBuilder(); + abstract Builder toBuilder(); @AutoValue.Builder - abstract static class Builder { - abstract Builder setFilenamePrefix(ValueProvider filenamePrefix); - abstract Builder setFilenameSuffix(String filenameSuffix); + abstract static class Builder { + abstract Builder setFormatFunction( + SerializableFunction formatFunction); - abstract Builder setTempDirectory(ValueProvider tempDirectory); + abstract Builder setFilenamePrefix(ValueProvider filenamePrefix); - abstract Builder setNumShards(int numShards); - abstract Builder setShardTemplate(String shardTemplate); - abstract Builder setRecordClass(Class recordClass); - abstract Builder setSchema(Schema schema); - abstract Builder setWindowedWrites(boolean windowedWrites); - abstract Builder setFilenamePolicy(FilenamePolicy filenamePolicy); - abstract Builder setCodec(SerializableAvroCodecFactory codec); - abstract Builder setMetadata(ImmutableMap metadata); + abstract Builder setFilenameSuffix(String filenameSuffix); + + abstract Builder setTempDirectory(ValueProvider tempDirectory); + + abstract Builder setNumShards(int numShards); + + abstract Builder setShardTemplate(String shardTemplate); + + abstract Builder setGenericRecords(boolean genericRecords); - abstract Write build(); + abstract Builder setSchema(Schema schema); + + abstract Builder setWindowedWrites(boolean windowedWrites); + + abstract Builder setFilenamePolicy(FilenamePolicy filenamePolicy); + + abstract Builder setCodec(SerializableAvroCodecFactory codec); + + abstract Builder setMetadata(ImmutableMap metadata); + + abstract Builder setDynamicDestinations( + DynamicAvroDestinations dynamicDestinations); + + abstract TypedWrite build(); } /** @@ -635,7 +734,7 @@ abstract static class Builder { * common suffix (if supplied using {@link #withSuffix(String)}). This default can be overridden * using {@link #to(FilenamePolicy)}. */ - public Write to(String outputPrefix) { + public TypedWrite to(String outputPrefix) { return to(FileBasedSink.convertToFileResourceIfPossible(outputPrefix)); } @@ -658,14 +757,12 @@ public Write to(String outputPrefix) { * infer a directory for temporary files. */ @Experimental(Kind.FILESYSTEM) - public Write to(ResourceId outputPrefix) { + public TypedWrite to(ResourceId outputPrefix) { return toResource(StaticValueProvider.of(outputPrefix)); } - /** - * Like {@link #to(String)}. - */ - public Write to(ValueProvider outputPrefix) { + /** Like {@link #to(String)}. */ + public TypedWrite to(ValueProvider outputPrefix) { return toResource(NestedValueProvider.of(outputPrefix, new SerializableFunction() { @Override @@ -675,11 +772,9 @@ public ResourceId apply(String input) { })); } - /** - * Like {@link #to(ResourceId)}. - */ + /** Like {@link #to(ResourceId)}. */ @Experimental(Kind.FILESYSTEM) - public Write toResource(ValueProvider outputPrefix) { + public TypedWrite toResource(ValueProvider outputPrefix) { return toBuilder().setFilenamePrefix(outputPrefix).build(); } @@ -687,16 +782,52 @@ public Write toResource(ValueProvider outputPrefix) { * Writes to files named according to the given {@link FileBasedSink.FilenamePolicy}. A * directory for temporary files must be specified using {@link #withTempDirectory}. */ - public Write to(FilenamePolicy filenamePolicy) { + @Experimental(Kind.FILESYSTEM) + public TypedWrite to(FilenamePolicy filenamePolicy) { return toBuilder().setFilenamePolicy(filenamePolicy).build(); } + /** + * Use a {@link DynamicAvroDestinations} object to vend {@link FilenamePolicy} objects. These + * objects can examine the input record when creating a {@link FilenamePolicy}. A directory for + * temporary files must be specified using {@link #withTempDirectory}. + */ + @Experimental(Kind.FILESYSTEM) + public TypedWrite to( + DynamicAvroDestinations dynamicDestinations) { + return toBuilder().setDynamicDestinations(dynamicDestinations).build(); + } + + /** + * Sets the the output schema. Can only be used when the output type is {@link GenericRecord} + * and when not using {@link #to(DynamicAvroDestinations)}. + */ + public TypedWrite withSchema(Schema schema) { + return toBuilder().setSchema(schema).build(); + } + + /** + * Specifies a format function to convert {@link UserT} to the output type. If {@link + * #to(DynamicAvroDestinations)} is used, {@link DynamicAvroDestinations#formatRecord} must be + * used instead. + */ + public TypedWrite withFormatFunction( + SerializableFunction formatFunction) { + return toBuilder().setFormatFunction(formatFunction).build(); + } + /** Set the base directory used to generate temporary files. */ @Experimental(Kind.FILESYSTEM) - public Write withTempDirectory(ValueProvider tempDirectory) { + public TypedWrite withTempDirectory(ValueProvider tempDirectory) { return toBuilder().setTempDirectory(tempDirectory).build(); } + /** Set the base directory used to generate temporary files. */ + @Experimental(Kind.FILESYSTEM) + public TypedWrite withTempDirectory(ResourceId tempDirectory) { + return withTempDirectory(StaticValueProvider.of(tempDirectory)); + } + /** * Uses the given {@link ShardNameTemplate} for naming output files. This option may only be * used when using one of the default filename-prefix to() overrides. @@ -704,7 +835,7 @@ public Write withTempDirectory(ValueProvider tempDirectory) { *

              See {@link DefaultFilenamePolicy} for how the prefix, shard name template, and suffix are * used. */ - public Write withShardNameTemplate(String shardTemplate) { + public TypedWrite withShardNameTemplate(String shardTemplate) { return toBuilder().setShardTemplate(shardTemplate).build(); } @@ -715,7 +846,7 @@ public Write withShardNameTemplate(String shardTemplate) { *

              See {@link DefaultFilenamePolicy} for how the prefix, shard name template, and suffix are * used. */ - public Write withSuffix(String filenameSuffix) { + public TypedWrite withSuffix(String filenameSuffix) { return toBuilder().setFilenameSuffix(filenameSuffix).build(); } @@ -729,7 +860,7 @@ public Write withSuffix(String filenameSuffix) { * * @param numShards the number of shards to use, or 0 to let the system decide. */ - public Write withNumShards(int numShards) { + public TypedWrite withNumShards(int numShards) { checkArgument(numShards >= 0); return toBuilder().setNumShards(numShards).build(); } @@ -744,7 +875,7 @@ public Write withNumShards(int numShards) { * *

              This is equivalent to {@code .withNumShards(1).withShardNameTemplate("")} */ - public Write withoutSharding() { + public TypedWrite withoutSharding() { return withNumShards(1).withShardNameTemplate(""); } @@ -754,12 +885,12 @@ public Write withoutSharding() { *

              If using {@link #to(FileBasedSink.FilenamePolicy)}. Filenames will be generated using * {@link FilenamePolicy#windowedFilename}. See also {@link WriteFiles#withWindowedWrites()}. */ - public Write withWindowedWrites() { + public TypedWrite withWindowedWrites() { return toBuilder().setWindowedWrites(true).build(); } /** Writes to Avro file(s) compressed using specified codec. */ - public Write withCodec(CodecFactory codec) { + public TypedWrite withCodec(CodecFactory codec) { return toBuilder().setCodec(new SerializableAvroCodecFactory(codec)).build(); } @@ -768,7 +899,7 @@ public Write withCodec(CodecFactory codec) { * *

              Supported value types are String, Long, and byte[]. */ - public Write withMetadata(Map metadata) { + public TypedWrite withMetadata(Map metadata) { Map badKeys = Maps.newLinkedHashMap(); for (Map.Entry entry : metadata.entrySet()) { Object v = entry.getValue(); @@ -783,18 +914,31 @@ public Write withMetadata(Map metadata) { return toBuilder().setMetadata(ImmutableMap.copyOf(metadata)).build(); } - DynamicDestinations resolveDynamicDestinations() { - FilenamePolicy usedFilenamePolicy = getFilenamePolicy(); - if (usedFilenamePolicy == null) { - usedFilenamePolicy = - DefaultFilenamePolicy.fromStandardParameters( - getFilenamePrefix(), getShardTemplate(), getFilenameSuffix(), getWindowedWrites()); + DynamicAvroDestinations resolveDynamicDestinations() { + DynamicAvroDestinations dynamicDestinations = getDynamicDestinations(); + if (dynamicDestinations == null) { + FilenamePolicy usedFilenamePolicy = getFilenamePolicy(); + if (usedFilenamePolicy == null) { + usedFilenamePolicy = + DefaultFilenamePolicy.fromStandardParameters( + getFilenamePrefix(), + getShardTemplate(), + getFilenameSuffix(), + getWindowedWrites()); + } + dynamicDestinations = + constantDestinations( + usedFilenamePolicy, + getSchema(), + getMetadata(), + getCodec().getCodec(), + getFormatFunction()); } - return DynamicFileDestinations.constant(usedFilenamePolicy); + return dynamicDestinations; } @Override - public PDone expand(PCollection input) { + public PDone expand(PCollection input) { checkArgument( getFilenamePrefix() != null || getTempDirectory() != null, "Need to set either the filename prefix or the tempDirectory of a AvroIO.Write " @@ -805,24 +949,25 @@ public PDone expand(PCollection input) { "shardTemplate and filenameSuffix should only be used with the default " + "filename policy"); } + if (getDynamicDestinations() != null) { + checkArgument( + getFormatFunction() == null, + "A format function should not be specified " + + "with DynamicDestinations. Use DynamicDestinations.formatRecord instead"); + } + return expandTyped(input, resolveDynamicDestinations()); } public PDone expandTyped( - PCollection input, DynamicDestinations dynamicDestinations) { + PCollection input, + DynamicAvroDestinations dynamicDestinations) { ValueProvider tempDirectory = getTempDirectory(); if (tempDirectory == null) { tempDirectory = getFilenamePrefix(); } - WriteFiles write = - WriteFiles.to( - new AvroSink<>( - tempDirectory, - dynamicDestinations, - AvroCoder.of(getRecordClass(), getSchema()), - getCodec(), - getMetadata()), - SerializableFunctions.identity()); + WriteFiles write = + WriteFiles.to(new AvroSink<>(tempDirectory, dynamicDestinations, getGenericRecords())); if (getNumShards() > 0) { write = write.withNumShards(getNumShards()); } @@ -845,33 +990,11 @@ public void populateDisplayData(DisplayData.Builder builder) { : getTempDirectory().toString(); } builder - .add(DisplayData.item("schema", getRecordClass()).withLabel("Record Schema")) .addIfNotDefault( DisplayData.item("numShards", getNumShards()).withLabel("Maximum Output Shards"), 0) - .addIfNotDefault( - DisplayData.item("codec", getCodec().toString()).withLabel("Avro Compression Codec"), - DEFAULT_CODEC.toString()) .addIfNotNull( DisplayData.item("tempDirectory", tempDirectory) .withLabel("Directory for temporary files")); - builder.include("Metadata", new Metadata()); - } - - private class Metadata implements HasDisplayData { - @Override - public void populateDisplayData(DisplayData.Builder builder) { - for (Map.Entry entry : getMetadata().entrySet()) { - DisplayData.Type type = DisplayData.inferType(entry.getValue()); - if (type != null) { - builder.add(DisplayData.item(entry.getKey(), type, entry.getValue())); - } else { - String base64 = BaseEncoding.base64().encode((byte[]) entry.getValue()); - String repr = base64.length() <= METADATA_BYTES_MAX_LENGTH - ? base64 : base64.substring(0, METADATA_BYTES_MAX_LENGTH) + "..."; - builder.add(DisplayData.item(entry.getKey(), repr)); - } - } - } } @Override @@ -880,6 +1003,131 @@ protected Coder getDefaultOutputCoder() { } } + /** + * This class is used as the default return value of {@link AvroIO#write} + * + *

              All methods in this class delegate to the appropriate method of {@link AvroIO.TypedWrite}. + * This class exists for backwards compatibility, and will be removed in Beam 3.0. + */ + public static class Write extends PTransform, PDone> { + @VisibleForTesting TypedWrite inner; + + Write(TypedWrite inner) { + this.inner = inner; + } + + /** See {@link TypedWrite#to(String)}. */ + public Write to(String outputPrefix) { + return new Write<>( + inner + .to(FileBasedSink.convertToFileResourceIfPossible(outputPrefix)) + .withFormatFunction(SerializableFunctions.identity())); + } + + /** See {@link TypedWrite#to(ResourceId)} . */ + @Experimental(Kind.FILESYSTEM) + public Write to(ResourceId outputPrefix) { + return new Write( + inner.to(outputPrefix).withFormatFunction(SerializableFunctions.identity())); + } + + /** See {@link TypedWrite#to(ValueProvider)}. */ + public Write to(ValueProvider outputPrefix) { + return new Write<>( + inner.to(outputPrefix).withFormatFunction(SerializableFunctions.identity())); + } + + /** See {@link TypedWrite#to(ResourceId)}. */ + @Experimental(Kind.FILESYSTEM) + public Write toResource(ValueProvider outputPrefix) { + return new Write<>( + inner.toResource(outputPrefix).withFormatFunction(SerializableFunctions.identity())); + } + + /** See {@link TypedWrite#to(FilenamePolicy)}. */ + public Write to(FilenamePolicy filenamePolicy) { + return new Write<>( + inner.to(filenamePolicy).withFormatFunction(SerializableFunctions.identity())); + } + + /** See {@link TypedWrite#to(DynamicAvroDestinations)}. */ + public Write to(DynamicAvroDestinations dynamicDestinations) { + return new Write<>(inner.to(dynamicDestinations).withFormatFunction(null)); + } + + /** See {@link TypedWrite#withSchema}. */ + public Write withSchema(Schema schema) { + return new Write<>(inner.withSchema(schema)); + } + /** See {@link TypedWrite#withTempDirectory(ValueProvider)}. */ + @Experimental(Kind.FILESYSTEM) + public Write withTempDirectory(ValueProvider tempDirectory) { + return new Write<>(inner.withTempDirectory(tempDirectory)); + } + + /** See {@link TypedWrite#withTempDirectory(ResourceId)}. */ + public Write withTempDirectory(ResourceId tempDirectory) { + return new Write<>(inner.withTempDirectory(tempDirectory)); + } + + /** See {@link TypedWrite#withShardNameTemplate}. */ + public Write withShardNameTemplate(String shardTemplate) { + return new Write<>(inner.withShardNameTemplate(shardTemplate)); + } + + /** See {@link TypedWrite#withSuffix}. */ + public Write withSuffix(String filenameSuffix) { + return new Write<>(inner.withSuffix(filenameSuffix)); + } + + /** See {@link TypedWrite#withNumShards}. */ + public Write withNumShards(int numShards) { + return new Write<>(inner.withNumShards(numShards)); + } + + /** See {@link TypedWrite#withoutSharding}. */ + public Write withoutSharding() { + return new Write<>(inner.withoutSharding()); + } + + /** See {@link TypedWrite#withWindowedWrites}. */ + public Write withWindowedWrites() { + return new Write(inner.withWindowedWrites()); + } + + /** See {@link TypedWrite#withCodec}. */ + public Write withCodec(CodecFactory codec) { + return new Write<>(inner.withCodec(codec)); + } + + /** See {@link TypedWrite#withMetadata} . */ + public Write withMetadata(Map metadata) { + return new Write<>(inner.withMetadata(metadata)); + } + + @Override + public PDone expand(PCollection input) { + return inner.expand(input); + } + + @Override + public void populateDisplayData(DisplayData.Builder builder) { + inner.populateDisplayData(builder); + } + } + + /** + * Returns a {@link DynamicAvroDestinations} that always returns the same {@link FilenamePolicy}, + * schema, metadata, and codec. + */ + public static DynamicAvroDestinations constantDestinations( + FilenamePolicy filenamePolicy, + Schema schema, + Map metadata, + CodecFactory codec, + SerializableFunction formatFunction) { + return new ConstantAvroDestination<>(filenamePolicy, schema, metadata, codec, formatFunction); + } ///////////////////////////////////////////////////////////////////////////// /** Disallow construction of utility class. */ diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/AvroSink.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/AvroSink.java index c78870b5d7ece..acd3ea6dfb3e5 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/AvroSink.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/AvroSink.java @@ -17,93 +17,90 @@ */ package org.apache.beam.sdk.io; -import com.google.common.collect.ImmutableMap; import java.nio.channels.Channels; import java.nio.channels.WritableByteChannel; import java.util.Map; +import org.apache.avro.Schema; +import org.apache.avro.file.CodecFactory; import org.apache.avro.file.DataFileWriter; import org.apache.avro.generic.GenericDatumWriter; -import org.apache.avro.generic.GenericRecord; import org.apache.avro.io.DatumWriter; import org.apache.avro.reflect.ReflectDatumWriter; -import org.apache.beam.sdk.coders.AvroCoder; import org.apache.beam.sdk.io.fs.ResourceId; import org.apache.beam.sdk.options.ValueProvider; import org.apache.beam.sdk.util.MimeTypes; /** A {@link FileBasedSink} for Avro files. */ -class AvroSink extends FileBasedSink { - private final AvroCoder coder; - private final SerializableAvroCodecFactory codec; - private final ImmutableMap metadata; +class AvroSink extends FileBasedSink { + private final DynamicAvroDestinations dynamicDestinations; + private final boolean genericRecords; AvroSink( ValueProvider outputPrefix, - DynamicDestinations dynamicDestinations, - AvroCoder coder, - SerializableAvroCodecFactory codec, - ImmutableMap metadata) { + DynamicAvroDestinations dynamicDestinations, + boolean genericRecords) { // Avro handle compression internally using the codec. super(outputPrefix, dynamicDestinations, CompressionType.UNCOMPRESSED); - this.coder = coder; - this.codec = codec; - this.metadata = metadata; + this.dynamicDestinations = dynamicDestinations; + this.genericRecords = genericRecords; } @Override - public WriteOperation createWriteOperation() { - return new AvroWriteOperation<>(this, coder, codec, metadata); + public DynamicAvroDestinations getDynamicDestinations() { + return (DynamicAvroDestinations) super.getDynamicDestinations(); + } + + @Override + public WriteOperation createWriteOperation() { + return new AvroWriteOperation<>(this, genericRecords); } /** A {@link WriteOperation WriteOperation} for Avro files. */ - private static class AvroWriteOperation extends WriteOperation { - private final AvroCoder coder; - private final SerializableAvroCodecFactory codec; - private final ImmutableMap metadata; + private static class AvroWriteOperation + extends WriteOperation { + private final DynamicAvroDestinations dynamicDestinations; + private final boolean genericRecords; - private AvroWriteOperation( - AvroSink sink, - AvroCoder coder, - SerializableAvroCodecFactory codec, - ImmutableMap metadata) { + private AvroWriteOperation(AvroSink sink, boolean genericRecords) { super(sink); - this.coder = coder; - this.codec = codec; - this.metadata = metadata; + this.dynamicDestinations = sink.getDynamicDestinations(); + this.genericRecords = genericRecords; } @Override - public Writer createWriter() throws Exception { - return new AvroWriter<>(this, coder, codec, metadata); + public Writer createWriter() throws Exception { + return new AvroWriter<>(this, dynamicDestinations, genericRecords); } } /** A {@link Writer Writer} for Avro files. */ - private static class AvroWriter extends Writer { - private final AvroCoder coder; - private DataFileWriter dataFileWriter; - private SerializableAvroCodecFactory codec; - private final ImmutableMap metadata; + private static class AvroWriter extends Writer { + private DataFileWriter dataFileWriter; + private final DynamicAvroDestinations dynamicDestinations; + private final boolean genericRecords; public AvroWriter( - WriteOperation writeOperation, - AvroCoder coder, - SerializableAvroCodecFactory codec, - ImmutableMap metadata) { + WriteOperation writeOperation, + DynamicAvroDestinations dynamicDestinations, + boolean genericRecords) { super(writeOperation, MimeTypes.BINARY); - this.coder = coder; - this.codec = codec; - this.metadata = metadata; + this.dynamicDestinations = dynamicDestinations; + this.genericRecords = genericRecords; } @SuppressWarnings("deprecation") // uses internal test functionality. @Override protected void prepareWrite(WritableByteChannel channel) throws Exception { - DatumWriter datumWriter = coder.getType().equals(GenericRecord.class) - ? new GenericDatumWriter(coder.getSchema()) - : new ReflectDatumWriter(coder.getSchema()); + DestinationT destination = getDestination(); + CodecFactory codec = dynamicDestinations.getCodec(destination); + Schema schema = dynamicDestinations.getSchema(destination); + Map metadata = dynamicDestinations.getMetadata(destination); - dataFileWriter = new DataFileWriter<>(datumWriter).setCodec(codec.getCodec()); + DatumWriter datumWriter = + genericRecords + ? new GenericDatumWriter(schema) + : new ReflectDatumWriter(schema); + dataFileWriter = new DataFileWriter<>(datumWriter).setCodec(codec); for (Map.Entry entry : metadata.entrySet()) { Object v = entry.getValue(); if (v instanceof String) { @@ -118,11 +115,11 @@ protected void prepareWrite(WritableByteChannel channel) throws Exception { + v.getClass().getSimpleName()); } } - dataFileWriter.create(coder.getSchema(), Channels.newOutputStream(channel)); + dataFileWriter.create(schema, Channels.newOutputStream(channel)); } @Override - public void write(T value) throws Exception { + public void write(OutputT value) throws Exception { dataFileWriter.append(value); } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/ConstantAvroDestination.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/ConstantAvroDestination.java new file mode 100644 index 0000000000000..b006e26c78318 --- /dev/null +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/ConstantAvroDestination.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.sdk.io; + +import com.google.common.base.Function; +import com.google.common.base.Supplier; +import com.google.common.base.Suppliers; +import com.google.common.io.BaseEncoding; +import java.io.Serializable; +import java.util.Map; +import javax.annotation.Nullable; +import org.apache.avro.Schema; +import org.apache.avro.file.CodecFactory; +import org.apache.beam.sdk.io.FileBasedSink.FilenamePolicy; +import org.apache.beam.sdk.transforms.SerializableFunction; +import org.apache.beam.sdk.transforms.display.DisplayData; +import org.apache.beam.sdk.transforms.display.HasDisplayData; + +/** Always returns a constant {@link FilenamePolicy}, {@link Schema}, metadata, and codec. */ +class ConstantAvroDestination + extends DynamicAvroDestinations { + private static class SchemaFunction implements Serializable, Function { + @Nullable + @Override + public Schema apply(@Nullable String input) { + return new Schema.Parser().parse(input); + } + } + + // This should be a multiple of 4 to not get a partial encoded byte. + private static final int METADATA_BYTES_MAX_LENGTH = 40; + private final FilenamePolicy filenamePolicy; + private final Supplier schema; + private final Map metadata; + private final SerializableAvroCodecFactory codec; + private final SerializableFunction formatFunction; + + private class Metadata implements HasDisplayData { + @Override + public void populateDisplayData(DisplayData.Builder builder) { + for (Map.Entry entry : metadata.entrySet()) { + DisplayData.Type type = DisplayData.inferType(entry.getValue()); + if (type != null) { + builder.add(DisplayData.item(entry.getKey(), type, entry.getValue())); + } else { + String base64 = BaseEncoding.base64().encode((byte[]) entry.getValue()); + String repr = + base64.length() <= METADATA_BYTES_MAX_LENGTH + ? base64 + : base64.substring(0, METADATA_BYTES_MAX_LENGTH) + "..."; + builder.add(DisplayData.item(entry.getKey(), repr)); + } + } + } + } + + public ConstantAvroDestination( + FilenamePolicy filenamePolicy, + Schema schema, + Map metadata, + CodecFactory codec, + SerializableFunction formatFunction) { + this.filenamePolicy = filenamePolicy; + this.schema = Suppliers.compose(new SchemaFunction(), Suppliers.ofInstance(schema.toString())); + this.metadata = metadata; + this.codec = new SerializableAvroCodecFactory(codec); + this.formatFunction = formatFunction; + } + + @Override + public OutputT formatRecord(UserT record) { + return formatFunction.apply(record); + } + + @Override + public Void getDestination(UserT element) { + return (Void) null; + } + + @Override + public Void getDefaultDestination() { + return (Void) null; + } + + @Override + public FilenamePolicy getFilenamePolicy(Void destination) { + return filenamePolicy; + } + + @Override + public Schema getSchema(Void destination) { + return schema.get(); + } + + @Override + public Map getMetadata(Void destination) { + return metadata; + } + + @Override + public CodecFactory getCodec(Void destination) { + return codec.getCodec(); + } + + @Override + public void populateDisplayData(DisplayData.Builder builder) { + filenamePolicy.populateDisplayData(builder); + builder.add(DisplayData.item("schema", schema.get().toString()).withLabel("Record Schema")); + builder.addIfNotDefault( + DisplayData.item("codec", codec.getCodec().toString()).withLabel("Avro Compression Codec"), + AvroIO.TypedWrite.DEFAULT_SERIALIZABLE_CODEC.toString()); + builder.include("Metadata", new Metadata()); + } +} diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/DefaultFilenamePolicy.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/DefaultFilenamePolicy.java index 4021609bf191b..1f438d57de003 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/DefaultFilenamePolicy.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/DefaultFilenamePolicy.java @@ -157,7 +157,6 @@ public boolean equals(Object o) { && shardTemplate.equals(other.shardTemplate) && suffix.equals(other.suffix); } - @Override public String toString() { return MoreObjects.toStringHelper(this) diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/DynamicAvroDestinations.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/DynamicAvroDestinations.java new file mode 100644 index 0000000000000..f4e8ee6731593 --- /dev/null +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/DynamicAvroDestinations.java @@ -0,0 +1,46 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.beam.sdk.io; + +import com.google.common.collect.ImmutableMap; +import java.util.Map; +import org.apache.avro.Schema; +import org.apache.avro.file.CodecFactory; +import org.apache.beam.sdk.io.FileBasedSink.DynamicDestinations; + +/** + * A specialization of {@link DynamicDestinations} for {@link AvroIO}. In addition to dynamic file + * destinations, this allows specifying other AVRO properties (schema, metadata, codec) per + * destination. + */ +public abstract class DynamicAvroDestinations + extends DynamicDestinations { + /** Return an AVRO schema for a given destination. */ + public abstract Schema getSchema(DestinationT destination); + + /** Return AVRO file metadata for a given destination. */ + public Map getMetadata(DestinationT destination) { + return ImmutableMap.of(); + } + + /** Return an AVRO codec for a given destination. */ + public CodecFactory getCodec(DestinationT destination) { + return AvroIO.TypedWrite.DEFAULT_CODEC; + } +} diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/DynamicFileDestinations.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/DynamicFileDestinations.java index d05a01a7e5732..b087bc5f235a3 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/DynamicFileDestinations.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/DynamicFileDestinations.java @@ -18,7 +18,6 @@ package org.apache.beam.sdk.io; -import static com.google.common.base.Preconditions.checkNotNull; import static com.google.common.base.Preconditions.checkState; import javax.annotation.Nullable; @@ -28,20 +27,30 @@ import org.apache.beam.sdk.io.FileBasedSink.DynamicDestinations; import org.apache.beam.sdk.io.FileBasedSink.FilenamePolicy; import org.apache.beam.sdk.transforms.SerializableFunction; +import org.apache.beam.sdk.transforms.SerializableFunctions; import org.apache.beam.sdk.transforms.display.DisplayData; /** Some helper classes that derive from {@link FileBasedSink.DynamicDestinations}. */ public class DynamicFileDestinations { /** Always returns a constant {@link FilenamePolicy}. */ - private static class ConstantFilenamePolicy extends DynamicDestinations { + private static class ConstantFilenamePolicy + extends DynamicDestinations { private final FilenamePolicy filenamePolicy; + private final SerializableFunction formatFunction; - public ConstantFilenamePolicy(FilenamePolicy filenamePolicy) { - this.filenamePolicy = checkNotNull(filenamePolicy); + public ConstantFilenamePolicy( + FilenamePolicy filenamePolicy, SerializableFunction formatFunction) { + this.filenamePolicy = filenamePolicy; + this.formatFunction = formatFunction; } @Override - public Void getDestination(T element) { + public OutputT formatRecord(UserT record) { + return formatFunction.apply(record); + } + + @Override + public Void getDestination(UserT element) { return (Void) null; } @@ -71,14 +80,24 @@ public void populateDisplayData(DisplayData.Builder builder) { * A base class for a {@link DynamicDestinations} object that returns differently-configured * instances of {@link DefaultFilenamePolicy}. */ - private static class DefaultPolicyDestinations extends DynamicDestinations { - SerializableFunction destinationFunction; - Params emptyDestination; + private static class DefaultPolicyDestinations + extends DynamicDestinations { + private final SerializableFunction destinationFunction; + private final Params emptyDestination; + private final SerializableFunction formatFunction; public DefaultPolicyDestinations( - SerializableFunction destinationFunction, Params emptyDestination) { + SerializableFunction destinationFunction, + Params emptyDestination, + SerializableFunction formatFunction) { this.destinationFunction = destinationFunction; this.emptyDestination = emptyDestination; + this.formatFunction = formatFunction; + } + + @Override + public OutputT formatRecord(UserT record) { + return formatFunction.apply(record); } @Override @@ -104,16 +123,28 @@ public FilenamePolicy getFilenamePolicy(DefaultFilenamePolicy.Params params) { } /** Returns a {@link DynamicDestinations} that always returns the same {@link FilenamePolicy}. */ - public static DynamicDestinations constant(FilenamePolicy filenamePolicy) { - return new ConstantFilenamePolicy<>(filenamePolicy); + public static DynamicDestinations constant( + FilenamePolicy filenamePolicy, SerializableFunction formatFunction) { + return new ConstantFilenamePolicy<>(filenamePolicy, formatFunction); + } + + /** + * A specialization of {@link #constant(FilenamePolicy, SerializableFunction)} for the case where + * UserT and OutputT are the same type and the format function is the identity. + */ + public static DynamicDestinations constant( + FilenamePolicy filenamePolicy) { + return new ConstantFilenamePolicy<>(filenamePolicy, SerializableFunctions.identity()); } /** * Returns a {@link DynamicDestinations} that returns instances of {@link DefaultFilenamePolicy} * configured with the given {@link Params}. */ - public static DynamicDestinations toDefaultPolicies( - SerializableFunction destinationFunction, Params emptyDestination) { - return new DefaultPolicyDestinations<>(destinationFunction, emptyDestination); + public static DynamicDestinations toDefaultPolicies( + SerializableFunction destinationFunction, + Params emptyDestination, + SerializableFunction formatFunction) { + return new DefaultPolicyDestinations<>(destinationFunction, emptyDestination, formatFunction); } } 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 3bf5d5b303ac3..4e2b61c34c09c 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 @@ -27,8 +27,10 @@ import com.google.common.annotations.VisibleForTesting; 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 com.google.common.collect.Maps; import com.google.common.collect.Ordering; import java.io.IOException; import java.io.InputStream; @@ -40,7 +42,6 @@ import java.util.Arrays; import java.util.Collections; import java.util.Comparator; -import java.util.HashMap; import java.util.HashSet; import java.util.List; import java.util.Map; @@ -67,6 +68,7 @@ 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.DoFn; import org.apache.beam.sdk.transforms.SerializableFunction; import org.apache.beam.sdk.transforms.display.DisplayData; import org.apache.beam.sdk.transforms.display.HasDisplayData; @@ -74,6 +76,7 @@ import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.transforms.windowing.PaneInfo.PaneInfoCoder; import org.apache.beam.sdk.util.MimeTypes; +import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TypeDescriptor; import org.apache.beam.sdk.values.TypeDescriptors.TypeVariableExtractor; import org.apache.commons.compress.compressors.bzip2.BZip2CompressorOutputStream; @@ -95,9 +98,9 @@ *

              The process of writing to file-based sink is as follows: * *

                - *
              1. An optional subclass-defined initialization, - *
              2. a parallel write of bundles to temporary files, and finally, - *
              3. these temporary files are renamed with final output filenames. + *
              4. An optional subclass-defined initialization, + *
              5. a parallel write of bundles to temporary files, and finally, + *
              6. these temporary files are renamed with final output filenames. *
              * *

              In order to ensure fault-tolerance, a bundle may be executed multiple times (e.g., in the @@ -121,7 +124,8 @@ * @param the type of values written to the sink. */ @Experimental(Kind.FILESYSTEM) -public abstract class FileBasedSink implements Serializable, HasDisplayData { +public abstract class FileBasedSink + implements Serializable, HasDisplayData { private static final Logger LOG = LoggerFactory.getLogger(FileBasedSink.class); /** Directly supported file output compression types. */ @@ -199,7 +203,7 @@ public static ResourceId convertToFileResourceIfPossible(String outputPrefix) { } } - private final DynamicDestinations dynamicDestinations; + private final DynamicDestinations dynamicDestinations; /** * The {@link WritableByteChannelFactory} that is used to wrap the raw data output to the @@ -215,8 +219,54 @@ public static ResourceId convertToFileResourceIfPossible(String outputPrefix) { * destination type into an instance of {@link FilenamePolicy}. */ @Experimental(Kind.FILESYSTEM) - public abstract static class DynamicDestinations + public abstract static class DynamicDestinations implements HasDisplayData, Serializable { + interface SideInputAccessor { + SideInputT sideInput(PCollectionView view); + } + + private SideInputAccessor sideInputAccessor; + + static class SideInputAccessorViaProcessContext implements SideInputAccessor { + private DoFn.ProcessContext processContext; + + SideInputAccessorViaProcessContext(DoFn.ProcessContext processContext) { + this.processContext = processContext; + } + + @Override + public SideInputT sideInput(PCollectionView view) { + return processContext.sideInput(view); + } + } + + /** + * Override to specify that this object needs access to one or more side inputs. This side + * inputs must be globally windowed, as they will be accessed from the global window. + */ + public List> getSideInputs() { + return ImmutableList.of(); + } + + /** + * Returns the value of a given side input. The view must be present in {@link + * #getSideInputs()}. + */ + protected final SideInputT sideInput(PCollectionView view) { + return sideInputAccessor.sideInput(view); + } + + final void setSideInputAccessor(SideInputAccessor sideInputAccessor) { + this.sideInputAccessor = sideInputAccessor; + } + + final void setSideInputAccessorFromProcessContext(DoFn.ProcessContext context) { + this.sideInputAccessor = new SideInputAccessorViaProcessContext(context); + } + + /** Convert an input record type into the output type. */ + public abstract OutputT formatRecord(UserT record); + /** * Returns an object that represents at a high level the destination being written to. May not * return null. A destination must have deterministic hash and equality methods defined. @@ -256,12 +306,13 @@ final Coder getDestinationCoderWithDefault(CoderRegistry registry) return destinationCoder; } // If dynamicDestinations doesn't provide a coder, try to find it in the coder registry. - @Nullable TypeDescriptor descriptor = + @Nullable + TypeDescriptor descriptor = extractFromTypeParameters( this, DynamicDestinations.class, new TypeVariableExtractor< - DynamicDestinations, DestinationT>() {}); + DynamicDestinations, DestinationT>() {}); checkArgument( descriptor != null, "Unable to infer a coder for DestinationT, " @@ -323,7 +374,7 @@ public ResourceId apply(ResourceId input) { @Experimental(Kind.FILESYSTEM) public FileBasedSink( ValueProvider tempDirectoryProvider, - DynamicDestinations dynamicDestinations) { + DynamicDestinations dynamicDestinations) { this(tempDirectoryProvider, dynamicDestinations, CompressionType.UNCOMPRESSED); } @@ -331,7 +382,7 @@ public FileBasedSink( @Experimental(Kind.FILESYSTEM) public FileBasedSink( ValueProvider tempDirectoryProvider, - DynamicDestinations dynamicDestinations, + DynamicDestinations dynamicDestinations, WritableByteChannelFactory writableByteChannelFactory) { this.tempDirectoryProvider = NestedValueProvider.of(tempDirectoryProvider, new ExtractDirectory()); @@ -341,8 +392,8 @@ public FileBasedSink( /** Return the {@link DynamicDestinations} used. */ @SuppressWarnings("unchecked") - public DynamicDestinations getDynamicDestinations() { - return (DynamicDestinations) dynamicDestinations; + public DynamicDestinations getDynamicDestinations() { + return (DynamicDestinations) dynamicDestinations; } /** @@ -357,7 +408,7 @@ public ValueProvider getTempDirectoryProvider() { public void validate(PipelineOptions options) {} /** Return a subclass of {@link WriteOperation} that will manage the write to the sink. */ - public abstract WriteOperation createWriteOperation(); + public abstract WriteOperation createWriteOperation(); public void populateDisplayData(DisplayData.Builder builder) { getDynamicDestinations().populateDisplayData(builder); @@ -371,11 +422,11 @@ public void populateDisplayData(DisplayData.Builder builder) { * written, * *

                - *
              1. {@link WriteOperation#finalize} is given a list of the temporary files containing the - * output bundles. - *
              2. During finalize, these temporary files are copied to final output locations and named - * according to a file naming template. - *
              3. Finally, any temporary files that were created during the write are removed. + *
              4. {@link WriteOperation#finalize} is given a list of the temporary files containing the + * output bundles. + *
              5. During finalize, these temporary files are copied to final output locations and named + * according to a file naming template. + *
              6. Finally, any temporary files that were created during the write are removed. *
              * *

              Subclass implementations of WriteOperation must implement {@link @@ -400,9 +451,9 @@ public void populateDisplayData(DisplayData.Builder builder) { * * @param the type of values written to the sink. */ - public abstract static class WriteOperation implements Serializable { + public abstract static class WriteOperation implements Serializable { /** The Sink that this WriteOperation will write to. */ - protected final FileBasedSink sink; + protected final FileBasedSink sink; /** Directory for temporary output files. */ protected final ValueProvider tempDirectory; @@ -428,7 +479,7 @@ protected static ResourceId buildTemporaryFilename(ResourceId tempDirectory, Str * * @param sink the FileBasedSink that will be used to configure this write operation. */ - public WriteOperation(FileBasedSink sink) { + public WriteOperation(FileBasedSink sink) { this( sink, NestedValueProvider.of(sink.getTempDirectoryProvider(), new TemporaryDirectoryBuilder())); @@ -463,12 +514,12 @@ public ResourceId apply(ResourceId tempDirectory) { * @param tempDirectory the base directory to be used for temporary output files. */ @Experimental(Kind.FILESYSTEM) - public WriteOperation(FileBasedSink sink, ResourceId tempDirectory) { + public WriteOperation(FileBasedSink sink, ResourceId tempDirectory) { this(sink, StaticValueProvider.of(tempDirectory)); } private WriteOperation( - FileBasedSink sink, ValueProvider tempDirectory) { + FileBasedSink sink, ValueProvider tempDirectory) { this.sink = sink; this.tempDirectory = tempDirectory; this.windowedWrites = false; @@ -478,7 +529,7 @@ private WriteOperation( * Clients must implement to return a subclass of {@link Writer}. This method must not mutate * the state of the object. */ - public abstract Writer createWriter() throws Exception; + public abstract Writer createWriter() throws Exception; /** Indicates that the operation will be performing windowed writes. */ public void setWindowedWrites(boolean windowedWrites) { @@ -533,7 +584,7 @@ public void removeTemporaryFiles(Set filenames) throws IOException { protected final Map buildOutputFilenames( Iterable> writerResults) { int numShards = Iterables.size(writerResults); - Map outputFilenames = new HashMap<>(); + Map outputFilenames = Maps.newHashMap(); // Either all results have a shard number set (if the sink is configured with a fixed // number of shards), or they all don't (otherwise). @@ -597,7 +648,6 @@ public int compare( "Only generated %s distinct file names for %s files.", numDistinctShards, outputFilenames.size()); - return outputFilenames; } @@ -691,7 +741,7 @@ final void removeTemporaryFiles( } /** Returns the FileBasedSink for this write operation. */ - public FileBasedSink getSink() { + public FileBasedSink getSink() { return sink; } @@ -727,10 +777,10 @@ protected final WritableByteChannelFactory getWritableByteChannelFactory() { * * @param the type of values to write. */ - public abstract static class Writer { + public abstract static class Writer { private static final Logger LOG = LoggerFactory.getLogger(Writer.class); - private final WriteOperation writeOperation; + private final WriteOperation writeOperation; /** Unique id for this output bundle. */ private String id; @@ -757,7 +807,7 @@ public abstract static class Writer { private final String mimeType; /** Construct a new {@link Writer} that will produce files of the given MIME type. */ - public Writer(WriteOperation writeOperation, String mimeType) { + public Writer(WriteOperation writeOperation, String mimeType) { checkNotNull(writeOperation); this.writeOperation = writeOperation; this.mimeType = mimeType; @@ -930,9 +980,14 @@ public final FileResult close() throws Exception { } /** Return the WriteOperation that this Writer belongs to. */ - public WriteOperation getWriteOperation() { + public WriteOperation getWriteOperation() { return writeOperation; } + + /** Return the user destination object for this writer. */ + public DestinationT getDestination() { + return destination; + } } /** @@ -987,7 +1042,7 @@ public DestinationT getDestination() { @Experimental(Kind.FILESYSTEM) public ResourceId getDestinationFile( - DynamicDestinations dynamicDestinations, + DynamicDestinations dynamicDestinations, int numShards, OutputFileHints outputFileHints) { checkArgument(getShard() != UNKNOWN_SHARDNUM); diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/TFRecordIO.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/TFRecordIO.java index 6e7b243b6284d..29b3e295e3b3b 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/TFRecordIO.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/TFRecordIO.java @@ -45,7 +45,6 @@ import org.apache.beam.sdk.options.ValueProvider.StaticValueProvider; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.SerializableFunction; -import org.apache.beam.sdk.transforms.SerializableFunctions; import org.apache.beam.sdk.transforms.display.DisplayData; import org.apache.beam.sdk.util.MimeTypes; import org.apache.beam.sdk.values.PBegin; @@ -357,10 +356,12 @@ public PDone expand(PCollection input) { checkState(getOutputPrefix() != null, "need to set the output prefix of a TFRecordIO.Write transform"); WriteFiles write = - WriteFiles.to( + WriteFiles.to( new TFRecordSink( - getOutputPrefix(), getShardTemplate(), getFilenameSuffix(), getCompressionType()), - SerializableFunctions.identity()); + getOutputPrefix(), + getShardTemplate(), + getFilenameSuffix(), + getCompressionType())); if (getNumShards() > 0) { write = write.withNumShards(getNumShards()); } @@ -548,7 +549,7 @@ protected boolean readNextRecord() throws IOException { /** A {@link FileBasedSink} for TFRecord files. Produces TFRecord files. */ @VisibleForTesting - static class TFRecordSink extends FileBasedSink { + static class TFRecordSink extends FileBasedSink { @VisibleForTesting TFRecordSink( ValueProvider outputPrefix, @@ -557,7 +558,7 @@ static class TFRecordSink extends FileBasedSink { TFRecordIO.CompressionType compressionType) { super( outputPrefix, - DynamicFileDestinations.constant( + DynamicFileDestinations.constant( DefaultFilenamePolicy.fromStandardParameters( outputPrefix, shardTemplate, suffix, false)), writableByteChannelFactory(compressionType)); @@ -571,7 +572,7 @@ public ResourceId apply(ResourceId input) { } @Override - public WriteOperation createWriteOperation() { + public WriteOperation createWriteOperation() { return new TFRecordWriteOperation(this); } @@ -591,23 +592,23 @@ private static WritableByteChannelFactory writableByteChannelFactory( } /** A {@link WriteOperation WriteOperation} for TFRecord files. */ - private static class TFRecordWriteOperation extends WriteOperation { + private static class TFRecordWriteOperation extends WriteOperation { private TFRecordWriteOperation(TFRecordSink sink) { super(sink); } @Override - public Writer createWriter() throws Exception { + public Writer createWriter() throws Exception { return new TFRecordWriter(this); } } /** A {@link Writer Writer} for TFRecord files. */ - private static class TFRecordWriter extends Writer { + private static class TFRecordWriter extends Writer { private WritableByteChannel outChannel; private TFRecordCodec codec; - private TFRecordWriter(WriteOperation writeOperation) { + private TFRecordWriter(WriteOperation writeOperation) { super(writeOperation, MimeTypes.BINARY); } 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 765a84243d635..312dc075ac8bc 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 @@ -23,6 +23,10 @@ import com.google.auto.value.AutoValue; import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Predicates; +import com.google.common.collect.Iterables; +import com.google.common.collect.Lists; +import java.util.List; import javax.annotation.Nullable; import org.apache.beam.sdk.annotations.Experimental; import org.apache.beam.sdk.annotations.Experimental.Kind; @@ -52,8 +56,8 @@ * *

              To read a {@link PCollection} from one or more text files, use {@code TextIO.read()} to * instantiate a transform and use {@link TextIO.Read#from(String)} to specify the path of the - * file(s) to be read. Alternatively, if the filenames to be read are themselves in a - * {@link PCollection}, apply {@link TextIO#readAll()}. + * file(s) to be read. Alternatively, if the filenames to be read are themselves in a {@link + * PCollection}, apply {@link TextIO#readAll()}. * *

              {@link TextIO.Read} returns a {@link PCollection} of {@link String Strings}, each * corresponding to one line of an input UTF-8 text file (split into lines delimited by '\n', '\r', @@ -70,8 +74,8 @@ * *

              If it is known that the filepattern will match a very large number of files (e.g. tens of * thousands or more), use {@link Read#withHintMatchesManyFiles} for better performance and - * scalability. Note that it may decrease performance if the filepattern matches only a small - * number of files. + * scalability. Note that it may decrease performance if the filepattern matches only a small number + * of files. * *

              Example 2: reading a PCollection of filenames. * @@ -121,9 +125,9 @@ * allows you to convert any input value into a custom destination object, and map that destination * object to a {@link FilenamePolicy}. This allows using different filename policies (or more * commonly, differently-configured instances of the same policy) based on the input record. Often - * this is used in conjunction with {@link TextIO#writeCustomType(SerializableFunction)}, which - * allows your {@link DynamicDestinations} object to examine the input type and takes a format - * function to convert that type to a string for writing. + * this is used in conjunction with {@link TextIO#writeCustomType}, which allows your {@link + * DynamicDestinations} object to examine the input type and takes a format function to convert that + * type to a string for writing. * *

              A convenience shortcut is provided for the case where the default naming policy is used, but * different configurations of this policy are wanted based on the input record. Default naming @@ -189,20 +193,23 @@ public static Write write() { * line. * *

              This version allows you to apply {@link TextIO} writes to a PCollection of a custom type - * {@link T}, along with a format function that converts the input type {@link T} to the String - * that will be written to the file. The advantage of this is it allows a user-provided {@link + * {@link UserT}. A format mechanism that converts the input type {@link UserT} to the String that + * will be written to the file must be specified. If using a custom {@link DynamicDestinations} + * object this is done using {@link DynamicDestinations#formatRecord}, otherwise the {@link + * TypedWrite#withFormatFunction} can be used to specify a format function. + * + *

              The advantage of using a custom type is that is it allows a user-provided {@link * DynamicDestinations} object, set via {@link Write#to(DynamicDestinations)} to examine the - * user's custom type when choosing a destination. + * custom type when choosing a destination. */ - public static TypedWrite writeCustomType(SerializableFunction formatFunction) { - return new AutoValue_TextIO_TypedWrite.Builder() + public static TypedWrite writeCustomType() { + return new AutoValue_TextIO_TypedWrite.Builder() .setFilenamePrefix(null) .setTempDirectory(null) .setShardTemplate(null) .setFilenameSuffix(null) .setFilenamePolicy(null) .setDynamicDestinations(null) - .setFormatFunction(formatFunction) .setWritableByteChannelFactory(FileBasedSink.CompressionType.UNCOMPRESSED) .setWindowedWrites(false) .setNumShards(0) @@ -417,11 +424,11 @@ public Boolean apply(String filename) { } } - ///////////////////////////////////////////////////////////////////////////// + // /////////////////////////////////////////////////////////////////////////// /** Implementation of {@link #write}. */ @AutoValue - public abstract static class TypedWrite extends PTransform, PDone> { + public abstract static class TypedWrite extends PTransform, PDone> { /** The prefix of each file written, combined with suffix and shardTemplate. */ @Nullable abstract ValueProvider getFilenamePrefix(); @@ -449,10 +456,19 @@ public abstract static class TypedWrite extends PTransform, PD /** Allows for value-dependent {@link DynamicDestinations} to be vended. */ @Nullable - abstract DynamicDestinations getDynamicDestinations(); + abstract DynamicDestinations getDynamicDestinations(); + + @Nullable + /** A destination function for using {@link DefaultFilenamePolicy} */ + abstract SerializableFunction getDestinationFunction(); - /** A function that converts T to a String, for writing to the file. */ - abstract SerializableFunction getFormatFunction(); + @Nullable + /** A default destination for empty PCollections. */ + abstract Params getEmptyDestination(); + + /** A function that converts UserT to a String, for writing to the file. */ + @Nullable + abstract SerializableFunction getFormatFunction(); /** Whether to write windowed output files. */ abstract boolean getWindowedWrites(); @@ -463,37 +479,42 @@ public abstract static class TypedWrite extends PTransform, PD */ abstract WritableByteChannelFactory getWritableByteChannelFactory(); - abstract Builder toBuilder(); + abstract Builder toBuilder(); @AutoValue.Builder - abstract static class Builder { - abstract Builder setFilenamePrefix(ValueProvider filenamePrefix); + abstract static class Builder { + abstract Builder setFilenamePrefix(ValueProvider filenamePrefix); + + abstract Builder setTempDirectory(ValueProvider tempDirectory); - abstract Builder setTempDirectory(ValueProvider tempDirectory); + abstract Builder setShardTemplate(@Nullable String shardTemplate); - abstract Builder setShardTemplate(@Nullable String shardTemplate); + abstract Builder setFilenameSuffix(@Nullable String filenameSuffix); - abstract Builder setFilenameSuffix(@Nullable String filenameSuffix); + abstract Builder setHeader(@Nullable String header); - abstract Builder setHeader(@Nullable String header); + abstract Builder setFooter(@Nullable String footer); - abstract Builder setFooter(@Nullable String footer); + abstract Builder setFilenamePolicy(@Nullable FilenamePolicy filenamePolicy); - abstract Builder setFilenamePolicy(@Nullable FilenamePolicy filenamePolicy); + abstract Builder setDynamicDestinations( + @Nullable DynamicDestinations dynamicDestinations); - abstract Builder setDynamicDestinations( - @Nullable DynamicDestinations dynamicDestinations); + abstract Builder setDestinationFunction( + @Nullable SerializableFunction destinationFunction); - abstract Builder setFormatFunction(SerializableFunction formatFunction); + abstract Builder setEmptyDestination(Params emptyDestination); - abstract Builder setNumShards(int numShards); + abstract Builder setFormatFunction(SerializableFunction formatFunction); - abstract Builder setWindowedWrites(boolean windowedWrites); + abstract Builder setNumShards(int numShards); - abstract Builder setWritableByteChannelFactory( + abstract Builder setWindowedWrites(boolean windowedWrites); + + abstract Builder setWritableByteChannelFactory( WritableByteChannelFactory writableByteChannelFactory); - abstract TypedWrite build(); + abstract TypedWrite build(); } /** @@ -513,18 +534,18 @@ abstract Builder setWritableByteChannelFactory( *

              If {@link #withTempDirectory} has not been called, this filename prefix will be used to * infer a directory for temporary files. */ - public TypedWrite to(String filenamePrefix) { + public TypedWrite to(String filenamePrefix) { return to(FileBasedSink.convertToFileResourceIfPossible(filenamePrefix)); } /** Like {@link #to(String)}. */ @Experimental(Kind.FILESYSTEM) - public TypedWrite to(ResourceId filenamePrefix) { + public TypedWrite to(ResourceId filenamePrefix) { return toResource(StaticValueProvider.of(filenamePrefix)); } /** Like {@link #to(String)}. */ - public TypedWrite to(ValueProvider outputPrefix) { + public TypedWrite to(ValueProvider outputPrefix) { return toResource(NestedValueProvider.of(outputPrefix, new SerializableFunction() { @Override @@ -538,7 +559,7 @@ public ResourceId apply(String input) { * Writes to files named according to the given {@link FileBasedSink.FilenamePolicy}. A * directory for temporary files must be specified using {@link #withTempDirectory}. */ - public TypedWrite to(FilenamePolicy filenamePolicy) { + public TypedWrite to(FilenamePolicy filenamePolicy) { return toBuilder().setFilenamePolicy(filenamePolicy).build(); } @@ -547,7 +568,7 @@ public TypedWrite to(FilenamePolicy filenamePolicy) { * objects can examine the input record when creating a {@link FilenamePolicy}. A directory for * temporary files must be specified using {@link #withTempDirectory}. */ - public TypedWrite to(DynamicDestinations dynamicDestinations) { + public TypedWrite to(DynamicDestinations dynamicDestinations) { return toBuilder().setDynamicDestinations(dynamicDestinations).build(); } @@ -558,26 +579,39 @@ public TypedWrite to(DynamicDestinations dynamicDestinations) { * emptyDestination parameter specified where empty files should be written for when the written * {@link PCollection} is empty. */ - public TypedWrite to( - SerializableFunction destinationFunction, Params emptyDestination) { - return to(DynamicFileDestinations.toDefaultPolicies(destinationFunction, emptyDestination)); + public TypedWrite to( + SerializableFunction destinationFunction, Params emptyDestination) { + return toBuilder() + .setDestinationFunction(destinationFunction) + .setEmptyDestination(emptyDestination) + .build(); } /** Like {@link #to(ResourceId)}. */ @Experimental(Kind.FILESYSTEM) - public TypedWrite toResource(ValueProvider filenamePrefix) { + public TypedWrite toResource(ValueProvider filenamePrefix) { return toBuilder().setFilenamePrefix(filenamePrefix).build(); } + /** + * Specifies a format function to convert {@link UserT} to the output type. If {@link + * #to(DynamicDestinations)} is used, {@link DynamicDestinations#formatRecord(Object)} must be + * used instead. + */ + public TypedWrite withFormatFunction( + SerializableFunction formatFunction) { + return toBuilder().setFormatFunction(formatFunction).build(); + } + /** Set the base directory used to generate temporary files. */ @Experimental(Kind.FILESYSTEM) - public TypedWrite withTempDirectory(ValueProvider tempDirectory) { + public TypedWrite withTempDirectory(ValueProvider tempDirectory) { return toBuilder().setTempDirectory(tempDirectory).build(); } /** Set the base directory used to generate temporary files. */ @Experimental(Kind.FILESYSTEM) - public TypedWrite withTempDirectory(ResourceId tempDirectory) { + public TypedWrite withTempDirectory(ResourceId tempDirectory) { return withTempDirectory(StaticValueProvider.of(tempDirectory)); } @@ -589,7 +623,7 @@ public TypedWrite withTempDirectory(ResourceId tempDirectory) { *

              See {@link DefaultFilenamePolicy} for how the prefix, shard name template, and suffix are * used. */ - public TypedWrite withShardNameTemplate(String shardTemplate) { + public TypedWrite withShardNameTemplate(String shardTemplate) { return toBuilder().setShardTemplate(shardTemplate).build(); } @@ -601,7 +635,7 @@ public TypedWrite withShardNameTemplate(String shardTemplate) { *

              See {@link DefaultFilenamePolicy} for how the prefix, shard name template, and suffix are * used. */ - public TypedWrite withSuffix(String filenameSuffix) { + public TypedWrite withSuffix(String filenameSuffix) { return toBuilder().setFilenameSuffix(filenameSuffix).build(); } @@ -615,7 +649,7 @@ public TypedWrite withSuffix(String filenameSuffix) { * * @param numShards the number of shards to use, or 0 to let the system decide. */ - public TypedWrite withNumShards(int numShards) { + public TypedWrite withNumShards(int numShards) { checkArgument(numShards >= 0); return toBuilder().setNumShards(numShards).build(); } @@ -629,7 +663,7 @@ public TypedWrite withNumShards(int numShards) { * *

              This is equivalent to {@code .withNumShards(1).withShardNameTemplate("")} */ - public TypedWrite withoutSharding() { + public TypedWrite withoutSharding() { return withNumShards(1).withShardNameTemplate(""); } @@ -638,7 +672,7 @@ public TypedWrite withoutSharding() { * *

              A {@code null} value will clear any previously configured header. */ - public TypedWrite withHeader(@Nullable String header) { + public TypedWrite withHeader(@Nullable String header) { return toBuilder().setHeader(header).build(); } @@ -647,7 +681,7 @@ public TypedWrite withHeader(@Nullable String header) { * *

              A {@code null} value will clear any previously configured footer. */ - public TypedWrite withFooter(@Nullable String footer) { + public TypedWrite withFooter(@Nullable String footer) { return toBuilder().setFooter(footer).build(); } @@ -658,7 +692,7 @@ public TypedWrite withFooter(@Nullable String footer) { * *

              A {@code null} value will reset the value to the default value mentioned above. */ - public TypedWrite withWritableByteChannelFactory( + public TypedWrite withWritableByteChannelFactory( WritableByteChannelFactory writableByteChannelFactory) { return toBuilder().setWritableByteChannelFactory(writableByteChannelFactory).build(); } @@ -669,36 +703,58 @@ public TypedWrite withWritableByteChannelFactory( *

              If using {@link #to(FileBasedSink.FilenamePolicy)}. Filenames will be generated using * {@link FilenamePolicy#windowedFilename}. See also {@link WriteFiles#withWindowedWrites()}. */ - public TypedWrite withWindowedWrites() { + public TypedWrite withWindowedWrites() { return toBuilder().setWindowedWrites(true).build(); } - private DynamicDestinations resolveDynamicDestinations() { - DynamicDestinations dynamicDestinations = getDynamicDestinations(); + private DynamicDestinations resolveDynamicDestinations() { + DynamicDestinations dynamicDestinations = getDynamicDestinations(); if (dynamicDestinations == null) { - FilenamePolicy usedFilenamePolicy = getFilenamePolicy(); - if (usedFilenamePolicy == null) { - usedFilenamePolicy = - DefaultFilenamePolicy.fromStandardParameters( - getFilenamePrefix(), - getShardTemplate(), - getFilenameSuffix(), - getWindowedWrites()); + if (getDestinationFunction() != null) { + dynamicDestinations = + DynamicFileDestinations.toDefaultPolicies( + getDestinationFunction(), getEmptyDestination(), getFormatFunction()); + } else { + FilenamePolicy usedFilenamePolicy = getFilenamePolicy(); + if (usedFilenamePolicy == null) { + usedFilenamePolicy = + DefaultFilenamePolicy.fromStandardParameters( + getFilenamePrefix(), + getShardTemplate(), + getFilenameSuffix(), + getWindowedWrites()); + } + dynamicDestinations = + DynamicFileDestinations.constant(usedFilenamePolicy, getFormatFunction()); } - dynamicDestinations = DynamicFileDestinations.constant(usedFilenamePolicy); } return dynamicDestinations; } @Override - public PDone expand(PCollection input) { + public PDone expand(PCollection input) { checkState( getFilenamePrefix() != null || getTempDirectory() != null, "Need to set either the filename prefix or the tempDirectory of a TextIO.Write " + "transform."); - checkState( - getFilenamePolicy() == null || getDynamicDestinations() == null, - "Cannot specify both a filename policy and dynamic destinations"); + + List allToArgs = + Lists.newArrayList( + getFilenamePolicy(), + getDynamicDestinations(), + getFilenamePrefix(), + getDestinationFunction()); + checkArgument( + 1 == Iterables.size(Iterables.filter(allToArgs, Predicates.notNull())), + "Exactly one of filename policy, dynamic destinations, filename prefix, or destination " + + "function must be set"); + + if (getDynamicDestinations() != null) { + checkArgument( + getFormatFunction() == null, + "A format function should not be specified " + + "with DynamicDestinations. Use DynamicDestinations.formatRecord instead"); + } if (getFilenamePolicy() != null || getDynamicDestinations() != null) { checkState( getShardTemplate() == null && getFilenameSuffix() == null, @@ -709,20 +765,20 @@ public PDone expand(PCollection input) { } public PDone expandTyped( - PCollection input, DynamicDestinations dynamicDestinations) { + PCollection input, + DynamicDestinations dynamicDestinations) { ValueProvider tempDirectory = getTempDirectory(); if (tempDirectory == null) { tempDirectory = getFilenamePrefix(); } - WriteFiles write = + WriteFiles write = WriteFiles.to( new TextSink<>( tempDirectory, dynamicDestinations, getHeader(), getFooter(), - getWritableByteChannelFactory()), - getFormatFunction()); + getWritableByteChannelFactory())); if (getNumShards() > 0) { write = write.withNumShards(getNumShards()); } @@ -774,7 +830,7 @@ public static class Write extends PTransform, PDone> { @VisibleForTesting TypedWrite inner; Write() { - this(TextIO.writeCustomType(SerializableFunctions.identity())); + this(TextIO.writeCustomType()); } Write(TypedWrite inner) { @@ -783,43 +839,53 @@ public static class Write extends PTransform, PDone> { /** See {@link TypedWrite#to(String)}. */ public Write to(String filenamePrefix) { - return new Write(inner.to(filenamePrefix)); + return new Write( + inner.to(filenamePrefix).withFormatFunction(SerializableFunctions.identity())); } /** See {@link TypedWrite#to(ResourceId)}. */ @Experimental(Kind.FILESYSTEM) public Write to(ResourceId filenamePrefix) { - return new Write(inner.to(filenamePrefix)); + return new Write( + inner.to(filenamePrefix).withFormatFunction(SerializableFunctions.identity())); } /** See {@link TypedWrite#to(ValueProvider)}. */ public Write to(ValueProvider outputPrefix) { - return new Write(inner.to(outputPrefix)); + return new Write( + inner.to(outputPrefix).withFormatFunction(SerializableFunctions.identity())); } /** See {@link TypedWrite#toResource(ValueProvider)}. */ @Experimental(Kind.FILESYSTEM) public Write toResource(ValueProvider filenamePrefix) { - return new Write(inner.toResource(filenamePrefix)); + return new Write( + inner + .toResource(filenamePrefix) + .withFormatFunction(SerializableFunctions.identity())); } /** See {@link TypedWrite#to(FilenamePolicy)}. */ @Experimental(Kind.FILESYSTEM) public Write to(FilenamePolicy filenamePolicy) { - return new Write(inner.to(filenamePolicy)); + return new Write( + inner.to(filenamePolicy).withFormatFunction(SerializableFunctions.identity())); } /** See {@link TypedWrite#to(DynamicDestinations)}. */ @Experimental(Kind.FILESYSTEM) - public Write to(DynamicDestinations dynamicDestinations) { - return new Write(inner.to(dynamicDestinations)); + public Write to(DynamicDestinations dynamicDestinations) { + return new Write(inner.to(dynamicDestinations).withFormatFunction(null)); } /** See {@link TypedWrite#to(SerializableFunction, Params)}. */ @Experimental(Kind.FILESYSTEM) public Write to( SerializableFunction destinationFunction, Params emptyDestination) { - return new Write(inner.to(destinationFunction, emptyDestination)); + return new Write( + inner + .to(destinationFunction, emptyDestination) + .withFormatFunction(SerializableFunctions.identity())); } /** See {@link TypedWrite#withTempDirectory(ValueProvider)}. */ diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/TextSink.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/TextSink.java index b57b28c5c0310..387e0acb9ca78 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/TextSink.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/TextSink.java @@ -34,13 +34,13 @@ * '\n'} represented in {@code UTF-8} format as the record separator. Each record (including the * last) is terminated. */ -class TextSink extends FileBasedSink { +class TextSink extends FileBasedSink { @Nullable private final String header; @Nullable private final String footer; TextSink( ValueProvider baseOutputFilename, - DynamicDestinations dynamicDestinations, + DynamicDestinations dynamicDestinations, @Nullable String header, @Nullable String footer, WritableByteChannelFactory writableByteChannelFactory) { @@ -50,13 +50,13 @@ class TextSink extends FileBasedSink } @Override - public WriteOperation createWriteOperation() { + public WriteOperation createWriteOperation() { return new TextWriteOperation<>(this, header, footer); } /** A {@link WriteOperation WriteOperation} for text files. */ private static class TextWriteOperation - extends WriteOperation { + extends WriteOperation { @Nullable private final String header; @Nullable private final String footer; @@ -67,20 +67,20 @@ private TextWriteOperation(TextSink sink, @Nullable String header, @Nullable Str } @Override - public Writer createWriter() throws Exception { + public Writer createWriter() throws Exception { return new TextWriter<>(this, header, footer); } } /** A {@link Writer Writer} for text files. */ - private static class TextWriter extends Writer { + private static class TextWriter extends Writer { private static final String NEWLINE = "\n"; @Nullable private final String header; @Nullable private final String footer; private OutputStreamWriter out; public TextWriter( - WriteOperation writeOperation, + WriteOperation writeOperation, @Nullable String header, @Nullable String footer) { super(writeOperation, MimeTypes.TEXT); diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/WriteFiles.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/WriteFiles.java index d8d7478223365..85c5652db4950 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/WriteFiles.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/WriteFiles.java @@ -60,7 +60,6 @@ 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; import org.apache.beam.sdk.transforms.View; import org.apache.beam.sdk.transforms.WithKeys; import org.apache.beam.sdk.transforms.display.DisplayData; @@ -76,7 +75,9 @@ import org.apache.beam.sdk.values.PCollectionList; import org.apache.beam.sdk.values.PCollectionTuple; import org.apache.beam.sdk.values.PCollectionView; +import org.apache.beam.sdk.values.PCollectionViews; import org.apache.beam.sdk.values.PDone; +import org.apache.beam.sdk.values.PValue; import org.apache.beam.sdk.values.ShardedKey; import org.apache.beam.sdk.values.TupleTag; import org.apache.beam.sdk.values.TupleTagList; @@ -121,9 +122,8 @@ public class WriteFiles private static final int SPILLED_RECORD_SHARDING_FACTOR = 10; static final int UNKNOWN_SHARDNUM = -1; - private FileBasedSink sink; - private SerializableFunction formatFunction; - private WriteOperation writeOperation; + private FileBasedSink sink; + private WriteOperation writeOperation; // This allows the number of shards to be dynamically computed based on the input // PCollection. @Nullable private final PTransform, PCollectionView> computeNumShards; @@ -133,37 +133,44 @@ public class WriteFiles private final ValueProvider numShardsProvider; private final boolean windowedWrites; private int maxNumWritersPerBundle; + // This is the set of side inputs used by this transform. This is usually populated by the users's + // DynamicDestinations object. + private final List> sideInputs; /** * Creates a {@link WriteFiles} transform that writes to the given {@link FileBasedSink}, letting * the runner control how many different shards are produced. */ public static WriteFiles to( - FileBasedSink sink, - SerializableFunction formatFunction) { + FileBasedSink sink) { checkNotNull(sink, "sink"); return new WriteFiles<>( sink, - formatFunction, null /* runner-determined sharding */, null, false, - DEFAULT_MAX_NUM_WRITERS_PER_BUNDLE); + DEFAULT_MAX_NUM_WRITERS_PER_BUNDLE, + sink.getDynamicDestinations().getSideInputs()); } private WriteFiles( - FileBasedSink sink, - SerializableFunction formatFunction, + FileBasedSink sink, @Nullable PTransform, PCollectionView> computeNumShards, @Nullable ValueProvider numShardsProvider, boolean windowedWrites, - int maxNumWritersPerBundle) { + int maxNumWritersPerBundle, + List> sideInputs) { this.sink = sink; - this.formatFunction = checkNotNull(formatFunction); this.computeNumShards = computeNumShards; this.numShardsProvider = numShardsProvider; this.windowedWrites = windowedWrites; this.maxNumWritersPerBundle = maxNumWritersPerBundle; + this.sideInputs = sideInputs; + } + + @Override + public Map, PValue> getAdditionalInputs() { + return PCollectionViews.toAdditionalInputs(sideInputs); } @Override @@ -207,15 +214,10 @@ public void populateDisplayData(DisplayData.Builder builder) { } /** Returns the {@link FileBasedSink} associated with this PTransform. */ - public FileBasedSink getSink() { + public FileBasedSink getSink() { return sink; } - /** Returns the the format function that maps the user type to the record written to files. */ - public SerializableFunction getFormatFunction() { - return formatFunction; - } - /** * Returns whether or not to perform windowed writes. */ @@ -266,11 +268,11 @@ public WriteFiles withNumShards( ValueProvider numShardsProvider) { return new WriteFiles<>( sink, - formatFunction, computeNumShards, numShardsProvider, windowedWrites, - maxNumWritersPerBundle); + maxNumWritersPerBundle, + sideInputs); } /** Set the maximum number of writers created in a bundle before spilling to shuffle. */ @@ -278,11 +280,22 @@ public WriteFiles withMaxNumWritersPerBundle( int maxNumWritersPerBundle) { return new WriteFiles<>( sink, - formatFunction, computeNumShards, numShardsProvider, windowedWrites, - maxNumWritersPerBundle); + maxNumWritersPerBundle, + sideInputs); + } + + public WriteFiles withSideInputs( + List> sideInputs) { + return new WriteFiles<>( + sink, + computeNumShards, + numShardsProvider, + windowedWrites, + maxNumWritersPerBundle, + sideInputs); } /** @@ -297,7 +310,7 @@ public WriteFiles withSharding( checkNotNull( sharding, "Cannot provide null sharding. Use withRunnerDeterminedSharding() instead"); return new WriteFiles<>( - sink, formatFunction, sharding, null, windowedWrites, maxNumWritersPerBundle); + sink, sharding, null, windowedWrites, maxNumWritersPerBundle, sideInputs); } /** @@ -305,8 +318,7 @@ public WriteFiles withSharding( * runner-determined sharding. */ public WriteFiles withRunnerDeterminedSharding() { - return new WriteFiles<>( - sink, formatFunction, null, null, windowedWrites, maxNumWritersPerBundle); + return new WriteFiles<>(sink, null, null, windowedWrites, maxNumWritersPerBundle, sideInputs); } /** @@ -323,7 +335,7 @@ public WriteFiles withRunnerDeterminedSharding() { */ public WriteFiles withWindowedWrites() { return new WriteFiles<>( - sink, formatFunction, computeNumShards, numShardsProvider, true, maxNumWritersPerBundle); + sink, computeNumShards, numShardsProvider, true, maxNumWritersPerBundle, sideInputs); } private static class WriterKey { @@ -374,7 +386,7 @@ private class WriteBundles extends DoFn> { private final Coder destinationCoder; private final boolean windowedWrites; - private Map, Writer> writers; + private Map, Writer> writers; private int spilledShardNum = UNKNOWN_SHARDNUM; WriteBundles( @@ -394,6 +406,7 @@ public void startBundle(StartBundleContext c) { @ProcessElement public void processElement(ProcessContext c, BoundedWindow window) throws Exception { + sink.getDynamicDestinations().setSideInputAccessorFromProcessContext(c); PaneInfo paneInfo = c.pane(); // If we are doing windowed writes, we need to ensure that we have separate files for // data in different windows/panes. Similar for dynamic writes, make sure that different @@ -402,7 +415,7 @@ public void processElement(ProcessContext c, BoundedWindow window) throws Except // the map will only have a single element. DestinationT destination = sink.getDynamicDestinations().getDestination(c.element()); WriterKey key = new WriterKey<>(window, c.pane(), destination); - Writer writer = writers.get(key); + Writer writer = writers.get(key); if (writer == null) { if (writers.size() <= maxNumWritersPerBundle) { String uuid = UUID.randomUUID().toString(); @@ -436,14 +449,14 @@ public void processElement(ProcessContext c, BoundedWindow window) throws Except return; } } - writeOrClose(writer, formatFunction.apply(c.element())); + writeOrClose(writer, getSink().getDynamicDestinations().formatRecord(c.element())); } @FinishBundle public void finishBundle(FinishBundleContext c) throws Exception { - for (Map.Entry, Writer> entry : + for (Map.Entry, Writer> entry : writers.entrySet()) { - Writer writer = entry.getValue(); + Writer writer = entry.getValue(); FileResult result; try { result = writer.close(); @@ -478,13 +491,14 @@ private class WriteShardedBundles @ProcessElement public void processElement(ProcessContext c, BoundedWindow window) throws Exception { + sink.getDynamicDestinations().setSideInputAccessorFromProcessContext(c); // Since we key by a 32-bit hash of the destination, there might be multiple destinations // in this iterable. The number of destinations is generally very small (1000s or less), so // there will rarely be hash collisions. - Map> writers = Maps.newHashMap(); + Map> writers = Maps.newHashMap(); for (UserT input : c.element().getValue()) { DestinationT destination = sink.getDynamicDestinations().getDestination(input); - Writer writer = writers.get(destination); + Writer writer = writers.get(destination); if (writer == null) { LOG.debug("Opening writer for write operation {}", writeOperation); writer = writeOperation.createWriter(); @@ -501,12 +515,12 @@ public void processElement(ProcessContext c, BoundedWindow window) throws Except LOG.debug("Done opening writer"); writers.put(destination, writer); } - writeOrClose(writer, formatFunction.apply(input)); - } + writeOrClose(writer, getSink().getDynamicDestinations().formatRecord(input)); + } // Close all writers. - for (Map.Entry> entry : writers.entrySet()) { - Writer writer = entry.getValue(); + for (Map.Entry> entry : writers.entrySet()) { + Writer writer = entry.getValue(); FileResult result; try { // Close the writer; if this throws let the error propagate. @@ -526,8 +540,8 @@ public void populateDisplayData(DisplayData.Builder builder) { } } - private static void writeOrClose( - Writer writer, OutputT t) throws Exception { + private static void writeOrClose( + Writer writer, OutputT t) throws Exception { try { writer.write(t); } catch (Exception e) { @@ -678,6 +692,7 @@ private PDone createWrite(PCollection input) { input.apply( writeName, ParDo.of(new WriteBundles(windowedWrites, unwrittedRecordsTag, destinationCoder)) + .withSideInputs(sideInputs) .withOutputTags(writtenRecordsTag, TupleTagList.of(unwrittedRecordsTag))); PCollection> writtenBundleFiles = writeTuple @@ -692,17 +707,18 @@ private PDone createWrite(PCollection input) { .apply("GroupUnwritten", GroupByKey., UserT>create()) .apply( "WriteUnwritten", - ParDo.of(new WriteShardedBundles(ShardAssignment.ASSIGN_IN_FINALIZE))) + ParDo.of(new WriteShardedBundles(ShardAssignment.ASSIGN_IN_FINALIZE)) + .withSideInputs(sideInputs)) .setCoder(FileResultCoder.of(shardedWindowCoder, destinationCoder)); results = PCollectionList.of(writtenBundleFiles) .and(writtenGroupedFiles) .apply(Flatten.>pCollections()); } else { - List> sideInputs = Lists.newArrayList(); + List> shardingSideInputs = Lists.newArrayList(); if (computeNumShards != null) { numShardsView = input.apply(computeNumShards); - sideInputs.add(numShardsView); + shardingSideInputs.add(numShardsView); } else { numShardsView = null; } @@ -715,7 +731,7 @@ private PDone createWrite(PCollection input) { numShardsView, (numShardsView != null) ? null : numShardsProvider, destinationCoder)) - .withSideInputs(sideInputs)) + .withSideInputs(shardingSideInputs)) .setCoder(KvCoder.of(ShardedKeyCoder.of(VarIntCoder.of()), input.getCoder())) .apply("GroupIntoShards", GroupByKey., UserT>create()); shardedWindowCoder = @@ -728,7 +744,8 @@ private PDone createWrite(PCollection input) { results = sharded.apply( "WriteShardedBundles", - ParDo.of(new WriteShardedBundles(ShardAssignment.ASSIGN_WHEN_WRITING))); + ParDo.of(new WriteShardedBundles(ShardAssignment.ASSIGN_WHEN_WRITING)) + .withSideInputs(sideInputs)); } results.setCoder(FileResultCoder.of(shardedWindowCoder, destinationCoder)); @@ -773,11 +790,12 @@ public void processElement(ProcessContext c) throws Exception { } else { final PCollectionView>> resultsView = results.apply(View.>asIterable()); - ImmutableList.Builder> sideInputs = + ImmutableList.Builder> finalizeSideInputs = ImmutableList.>builder().add(resultsView); if (numShardsView != null) { - sideInputs.add(numShardsView); + finalizeSideInputs.add(numShardsView); } + finalizeSideInputs.addAll(sideInputs); // Finalize the write in another do-once ParDo on the singleton collection containing the // Writer. The results from the per-bundle writes are given as an Iterable side input. @@ -794,7 +812,7 @@ public void processElement(ProcessContext c) throws Exception { new DoFn() { @ProcessElement public void processElement(ProcessContext c) throws Exception { - LOG.info("Finalizing write operation {}.", writeOperation); + sink.getDynamicDestinations().setSideInputAccessorFromProcessContext(c); // We must always output at least 1 shard, and honor user-specified numShards // if // set. @@ -827,7 +845,7 @@ public void processElement(ProcessContext c) throws Exception { writeOperation.removeTemporaryFiles(tempFiles); } }) - .withSideInputs(sideInputs.build())); + .withSideInputs(finalizeSideInputs.build())); } return PDone.in(input.getPipeline()); } @@ -857,7 +875,7 @@ private Set finalizeForDestinationFillEmptyShards( minShardsNeeded, destination); for (int i = 0; i < extraShardsNeeded; ++i) { - Writer writer = writeOperation.createWriter(); + Writer writer = writeOperation.createWriter(); // Currently this code path is only called in the unwindowed case. writer.openUnwindowed(UUID.randomUUID().toString(), UNKNOWN_SHARDNUM, destination); FileResult emptyWrite = writer.close(); 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 154ff5a59e0ef..a96b6bed2741c 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 @@ -30,9 +30,11 @@ import com.google.common.base.MoreObjects; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; +import com.google.common.collect.ImmutableSet; import com.google.common.collect.Iterables; import com.google.common.collect.Iterators; import com.google.common.collect.Lists; +import com.google.common.collect.Maps; import java.io.File; import java.io.FileInputStream; import java.io.IOException; @@ -41,6 +43,7 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.List; +import java.util.Map; import java.util.Objects; import java.util.Random; import java.util.Set; @@ -48,6 +51,7 @@ import org.apache.avro.file.CodecFactory; import org.apache.avro.file.DataFileReader; import org.apache.avro.file.DataFileStream; +import org.apache.avro.generic.GenericData; import org.apache.avro.generic.GenericDatumReader; import org.apache.avro.generic.GenericRecord; import org.apache.avro.reflect.Nullable; @@ -55,6 +59,7 @@ import org.apache.avro.reflect.ReflectDatumReader; import org.apache.beam.sdk.coders.AvroCoder; import org.apache.beam.sdk.coders.DefaultCoder; +import org.apache.beam.sdk.coders.StringUtf8Coder; import org.apache.beam.sdk.io.FileBasedSink.FilenamePolicy; import org.apache.beam.sdk.io.FileBasedSink.OutputFileHints; import org.apache.beam.sdk.io.fs.ResolveOptions.StandardResolveOptions; @@ -68,6 +73,7 @@ import org.apache.beam.sdk.testing.ValidatesRunner; import org.apache.beam.sdk.transforms.Create; import org.apache.beam.sdk.transforms.SerializableFunction; +import org.apache.beam.sdk.transforms.View; import org.apache.beam.sdk.transforms.display.DisplayData; import org.apache.beam.sdk.transforms.display.DisplayDataEvaluator; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; @@ -77,6 +83,7 @@ import org.apache.beam.sdk.transforms.windowing.Window; import org.apache.beam.sdk.util.SerializableUtils; import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TimestampedValue; import org.joda.time.Duration; import org.joda.time.Instant; @@ -535,17 +542,147 @@ public void testWindowedAvroIOWrite() throws Throwable { assertThat(actualElements, containsInAnyOrder(allElements.toArray())); } + private static final String SCHEMA_TEMPLATE_STRING = + "{\"namespace\": \"example.avro\",\n" + + " \"type\": \"record\",\n" + + " \"name\": \"TestTemplateSchema$$\",\n" + + " \"fields\": [\n" + + " {\"name\": \"$$full\", \"type\": \"string\"},\n" + + " {\"name\": \"$$suffix\", \"type\": [\"string\", \"null\"]}\n" + + " ]\n" + + "}"; + + private static String schemaFromPrefix(String prefix) { + return SCHEMA_TEMPLATE_STRING.replace("$$", prefix); + } + + private static GenericRecord createRecord(String record, String prefix, Schema schema) { + GenericRecord genericRecord = new GenericData.Record(schema); + genericRecord.put(prefix + "full", record); + genericRecord.put(prefix + "suffix", record.substring(1)); + return genericRecord; + } + + private static class TestDynamicDestinations + extends DynamicAvroDestinations { + ResourceId baseDir; + PCollectionView> schemaView; + + TestDynamicDestinations(ResourceId baseDir, PCollectionView> schemaView) { + this.baseDir = baseDir; + this.schemaView = schemaView; + } + + @Override + public Schema getSchema(String destination) { + // Return a per-destination schema. + String schema = sideInput(schemaView).get(destination); + return new Schema.Parser().parse(schema); + } + + @Override + public List> getSideInputs() { + return ImmutableList.>of(schemaView); + } + + @Override + public GenericRecord formatRecord(String record) { + String prefix = record.substring(0, 1); + return createRecord(record, prefix, getSchema(prefix)); + } + + @Override + public String getDestination(String element) { + // Destination is based on first character of string. + return element.substring(0, 1); + } + + @Override + public String getDefaultDestination() { + return ""; + } + + @Override + public FilenamePolicy getFilenamePolicy(String destination) { + return DefaultFilenamePolicy.fromStandardParameters( + StaticValueProvider.of( + baseDir.resolve("file_" + destination + ".txt", StandardResolveOptions.RESOLVE_FILE)), + null, + null, + false); + } + } + + @Test + @Category(NeedsRunner.class) + public void testDynamicDestinations() throws Exception { + ResourceId baseDir = + FileSystems.matchNewResource( + Files.createTempDirectory(tmpFolder.getRoot().toPath(), "testDynamicDestinations") + .toString(), + true); + + List elements = Lists.newArrayList("aaaa", "aaab", "baaa", "baab", "caaa", "caab"); + List expectedElements = Lists.newArrayListWithExpectedSize(elements.size()); + Map schemaMap = Maps.newHashMap(); + for (String element : elements) { + String prefix = element.substring(0, 1); + String jsonSchema = schemaFromPrefix(prefix); + schemaMap.put(prefix, jsonSchema); + expectedElements.add(createRecord(element, prefix, new Schema.Parser().parse(jsonSchema))); + } + PCollectionView> schemaView = + writePipeline + .apply("createSchemaView", Create.of(schemaMap)) + .apply(View.asMap()); + + PCollection input = + writePipeline.apply("createInput", Create.of(elements).withCoder(StringUtf8Coder.of())); + input.apply( + AvroIO.writeCustomTypeToGenericRecords() + .to(new TestDynamicDestinations(baseDir, schemaView)) + .withoutSharding() + .withTempDirectory(baseDir)); + writePipeline.run(); + + // Validate that the data written matches the expected elements in the expected order. + + List prefixes = Lists.newArrayList(); + for (String element : elements) { + prefixes.add(element.substring(0, 1)); + } + prefixes = ImmutableSet.copyOf(prefixes).asList(); + + List actualElements = new ArrayList<>(); + for (String prefix : prefixes) { + File expectedFile = + new File( + baseDir + .resolve( + "file_" + prefix + ".txt-00000-of-00001", StandardResolveOptions.RESOLVE_FILE) + .toString()); + assertTrue("Expected output file " + expectedFile.getAbsolutePath(), expectedFile.exists()); + Schema schema = new Schema.Parser().parse(schemaFromPrefix(prefix)); + try (DataFileReader reader = + new DataFileReader<>(expectedFile, new GenericDatumReader(schema))) { + Iterators.addAll(actualElements, reader); + } + expectedFile.delete(); + } + assertThat(actualElements, containsInAnyOrder(expectedElements.toArray())); + } + @Test public void testWriteWithDefaultCodec() throws Exception { AvroIO.Write write = AvroIO.write(String.class).to("/tmp/foo/baz"); - assertEquals(CodecFactory.deflateCodec(6).toString(), write.getCodec().toString()); + assertEquals(CodecFactory.deflateCodec(6).toString(), write.inner.getCodec().toString()); } @Test public void testWriteWithCustomCodec() throws Exception { AvroIO.Write write = AvroIO.write(String.class).to("/tmp/foo/baz").withCodec(CodecFactory.snappyCodec()); - assertEquals(SNAPPY_CODEC, write.getCodec().toString()); + assertEquals(SNAPPY_CODEC, write.inner.getCodec().toString()); } @Test @@ -556,7 +693,7 @@ public void testWriteWithSerDeCustomDeflateCodec() throws Exception { assertEquals( CodecFactory.deflateCodec(9).toString(), - SerializableUtils.clone(write.getCodec()).getCodec().toString()); + SerializableUtils.clone(write.inner.getCodec()).getCodec().toString()); } @Test @@ -567,7 +704,7 @@ public void testWriteWithSerDeCustomXZCodec() throws Exception { assertEquals( CodecFactory.xzCodec(9).toString(), - SerializableUtils.clone(write.getCodec()).getCodec().toString()); + SerializableUtils.clone(write.inner.getCodec()).getCodec().toString()); } @Test @@ -618,7 +755,8 @@ private void runTestWrite(String[] expectedElements, int numShards) throws IOExc String shardNameTemplate = firstNonNull( - write.getShardTemplate(), DefaultFilenamePolicy.DEFAULT_UNWINDOWED_SHARD_TEMPLATE); + write.inner.getShardTemplate(), + DefaultFilenamePolicy.DEFAULT_UNWINDOWED_SHARD_TEMPLATE); assertTestOutputs(expectedElements, numShards, outputFilePrefix, shardNameTemplate); } @@ -710,7 +848,13 @@ public void testWriteDisplayData() { assertThat(displayData, hasDisplayItem("filePrefix", "/foo")); assertThat(displayData, hasDisplayItem("shardNameTemplate", "-SS-of-NN-")); assertThat(displayData, hasDisplayItem("fileSuffix", "bar")); - assertThat(displayData, hasDisplayItem("schema", GenericClass.class)); + assertThat( + displayData, + hasDisplayItem( + "schema", + "{\"type\":\"record\",\"name\":\"GenericClass\",\"namespace\":\"org.apache.beam.sdk.io" + + ".AvroIOTest$\",\"fields\":[{\"name\":\"intField\",\"type\":\"int\"}," + + "{\"name\":\"stringField\",\"type\":\"string\"}]}")); assertThat(displayData, hasDisplayItem("numShards", 100)); assertThat(displayData, hasDisplayItem("codec", CodecFactory.snappyCodec().toString())); } 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 a6ad7464597d0..ff30e33c19f9f 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,7 @@ private void testRemoveTemporaryFiles(int numFiles, ResourceId tempDirectory) th SimpleSink.makeSimpleSink( getBaseOutputDirectory(), prefix, "", "", CompressionType.UNCOMPRESSED); - WriteOperation writeOp = + WriteOperation writeOp = new SimpleSink.SimpleWriteOperation<>(sink, tempDirectory); List temporaryFiles = new ArrayList<>(); @@ -482,11 +482,11 @@ private File writeValuesWithWritableByteChannelFactory( public void testFileBasedWriterWithWritableByteChannelFactory() throws Exception { final String testUid = "testId"; ResourceId root = getBaseOutputDirectory(); - WriteOperation writeOp = + WriteOperation writeOp = SimpleSink.makeSimpleSink( root, "file", "-SS-of-NN", "txt", new DrunkWritableByteChannelFactory()) .createWriteOperation(); - final Writer writer = writeOp.createWriter(); + final Writer writer = writeOp.createWriter(); final ResourceId expectedFile = writeOp.tempDirectory.get().resolve(testUid, StandardResolveOptions.RESOLVE_FILE); diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/SimpleSink.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/SimpleSink.java index 9196178104dbd..382898d304046 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/SimpleSink.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/SimpleSink.java @@ -28,10 +28,10 @@ /** * A simple {@link FileBasedSink} that writes {@link String} values as lines with header and footer. */ -class SimpleSink extends FileBasedSink { +class SimpleSink extends FileBasedSink { public SimpleSink( ResourceId tempDirectory, - DynamicDestinations dynamicDestinations, + DynamicDestinations dynamicDestinations, WritableByteChannelFactory writableByteChannelFactory) { super(StaticValueProvider.of(tempDirectory), dynamicDestinations, writableByteChannelFactory); } @@ -50,7 +50,7 @@ public static SimpleSink makeSimpleSink( String shardTemplate, String suffix, WritableByteChannelFactory writableByteChannelFactory) { - DynamicDestinations dynamicDestinations = + DynamicDestinations dynamicDestinations = DynamicFileDestinations.constant( DefaultFilenamePolicy.fromParams( new Params() @@ -67,7 +67,7 @@ public SimpleWriteOperation createWriteOperation() { } static final class SimpleWriteOperation - extends WriteOperation { + extends WriteOperation { public SimpleWriteOperation(SimpleSink sink, ResourceId tempOutputDirectory) { super(sink, tempOutputDirectory); } @@ -82,7 +82,7 @@ public SimpleWriter createWriter() throws Exception { } } - static final class SimpleWriter extends Writer { + static final class SimpleWriter extends Writer { static final String HEADER = "header"; static final String FOOTER = "footer"; diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/TextIOWriteTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/TextIOWriteTest.java index a73ed7d0df4b7..7f80c265f9e86 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/TextIOWriteTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/TextIOWriteTest.java @@ -110,13 +110,19 @@ public FileVisitResult postVisitDirectory(Path dir, IOException exc) throws IOEx }); } - static class TestDynamicDestinations extends FileBasedSink.DynamicDestinations { + static class TestDynamicDestinations + extends FileBasedSink.DynamicDestinations { ResourceId baseDir; TestDynamicDestinations(ResourceId baseDir) { this.baseDir = baseDir; } + @Override + public String formatRecord(String record) { + return record; + } + @Override public String getDestination(String element) { // Destination is based on first character of string. @@ -169,10 +175,7 @@ public void testDynamicDestinations() throws Exception { List elements = Lists.newArrayList("aaaa", "aaab", "baaa", "baab", "caaa", "caab"); PCollection input = p.apply(Create.of(elements).withCoder(StringUtf8Coder.of())); - input.apply( - TextIO.write() - .to(new TestDynamicDestinations(baseDir)) - .withTempDirectory(FileSystems.matchNewResource(baseDir.toString(), true))); + input.apply(TextIO.write().to(new TestDynamicDestinations(baseDir)).withTempDirectory(baseDir)); p.run(); assertOutputFiles( @@ -268,8 +271,14 @@ public void testDynamicDefaultFilenamePolicy() throws Exception { new UserWriteType("caab", "sixth")); PCollection input = p.apply(Create.of(elements)); input.apply( - TextIO.writeCustomType(new SerializeUserWrite()) - .to(new UserWriteDestination(baseDir), new DefaultFilenamePolicy.Params()) + TextIO.writeCustomType() + .to( + new UserWriteDestination(baseDir), + new DefaultFilenamePolicy.Params() + .withBaseFilename( + baseDir.resolve( + "empty", ResolveOptions.StandardResolveOptions.RESOLVE_FILE))) + .withFormatFunction(new SerializeUserWrite()) .withTempDirectory(FileSystems.matchNewResource(baseDir.toString(), true))); p.run(); diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/WriteFilesTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/WriteFilesTest.java index 60088de1bc005..1d4ce08f70cd6 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/WriteFilesTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/WriteFilesTest.java @@ -68,8 +68,6 @@ 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.SerializableFunction; -import org.apache.beam.sdk.transforms.SerializableFunctions; import org.apache.beam.sdk.transforms.SimpleFunction; import org.apache.beam.sdk.transforms.Top; import org.apache.beam.sdk.transforms.View; @@ -178,11 +176,7 @@ public void testWrite() throws IOException { "Intimidating pigeon", "Pedantic gull", "Frisky finch"); - runWrite( - inputs, - IDENTITY_MAP, - getBaseOutputFilename(), - WriteFiles.to(makeSimpleSink(), SerializableFunctions.identity())); + runWrite(inputs, IDENTITY_MAP, getBaseOutputFilename(), WriteFiles.to(makeSimpleSink())); } /** Test that WriteFiles with an empty input still produces one shard. */ @@ -193,7 +187,7 @@ public void testEmptyWrite() throws IOException { Collections.emptyList(), IDENTITY_MAP, getBaseOutputFilename(), - WriteFiles.to(makeSimpleSink(), SerializableFunctions.identity())); + WriteFiles.to(makeSimpleSink())); checkFileContents(getBaseOutputFilename(), Collections.emptyList(), Optional.of(1)); } @@ -208,7 +202,7 @@ public void testShardedWrite() throws IOException { Arrays.asList("one", "two", "three", "four", "five", "six"), IDENTITY_MAP, getBaseOutputFilename(), - WriteFiles.to(makeSimpleSink(), SerializableFunctions.identity()).withNumShards(1)); + WriteFiles.to(makeSimpleSink())); } private ResourceId getBaseOutputDirectory() { @@ -241,9 +235,7 @@ public void testCustomShardedWrite() throws IOException { } SimpleSink sink = makeSimpleSink(); - WriteFiles write = - WriteFiles.to(sink, SerializableFunctions.identity()) - .withSharding(new LargestInt()); + WriteFiles write = WriteFiles.to(sink).withSharding(new LargestInt()); p.apply(Create.timestamped(inputs, timestamps).withCoder(StringUtf8Coder.of())) .apply(IDENTITY_MAP) .apply(write); @@ -264,8 +256,7 @@ public void testExpandShardedWrite() throws IOException { Arrays.asList("one", "two", "three", "four", "five", "six"), IDENTITY_MAP, getBaseOutputFilename(), - WriteFiles.to(makeSimpleSink(), SerializableFunctions.identity()) - .withNumShards(20)); + WriteFiles.to(makeSimpleSink()).withNumShards(20)); } /** Test a WriteFiles transform with an empty PCollection. */ @@ -273,11 +264,7 @@ public void testExpandShardedWrite() throws IOException { @Category(NeedsRunner.class) public void testWriteWithEmptyPCollection() throws IOException { List inputs = new ArrayList<>(); - runWrite( - inputs, - IDENTITY_MAP, - getBaseOutputFilename(), - WriteFiles.to(makeSimpleSink(), SerializableFunctions.identity())); + runWrite(inputs, IDENTITY_MAP, getBaseOutputFilename(), WriteFiles.to(makeSimpleSink())); } /** Test a WriteFiles with a windowed PCollection. */ @@ -295,7 +282,7 @@ public void testWriteWindowed() throws IOException { inputs, new WindowAndReshuffle<>(Window.into(FixedWindows.of(Duration.millis(2)))), getBaseOutputFilename(), - WriteFiles.to(makeSimpleSink(), SerializableFunctions.identity())); + WriteFiles.to(makeSimpleSink())); } /** Test a WriteFiles with sessions. */ @@ -314,7 +301,7 @@ public void testWriteWithSessions() throws IOException { inputs, new WindowAndReshuffle<>(Window.into(Sessions.withGapDuration(Duration.millis(1)))), getBaseOutputFilename(), - WriteFiles.to(makeSimpleSink(), SerializableFunctions.identity())); + WriteFiles.to(makeSimpleSink())); } @Test @@ -328,15 +315,12 @@ public void testWriteSpilling() throws IOException { inputs, Window.into(FixedWindows.of(Duration.millis(2))), getBaseOutputFilename(), - WriteFiles.to(makeSimpleSink(), SerializableFunctions.identity()) - .withMaxNumWritersPerBundle(2) - .withWindowedWrites()); + WriteFiles.to(makeSimpleSink()).withMaxNumWritersPerBundle(2).withWindowedWrites()); } public void testBuildWrite() { SimpleSink sink = makeSimpleSink(); - WriteFiles write = - WriteFiles.to(sink, SerializableFunctions.identity()).withNumShards(3); + WriteFiles write = WriteFiles.to(sink).withNumShards(3); assertThat((SimpleSink) write.getSink(), is(sink)); PTransform, PCollectionView> originalSharding = write.getSharding(); @@ -358,7 +342,7 @@ public void testBuildWrite() { @Test public void testDisplayData() { - DynamicDestinations dynamicDestinations = + DynamicDestinations dynamicDestinations = DynamicFileDestinations.constant( DefaultFilenamePolicy.fromParams( new Params() @@ -374,8 +358,7 @@ public void populateDisplayData(DisplayData.Builder builder) { builder.add(DisplayData.item("foo", "bar")); } }; - WriteFiles write = - WriteFiles.to(sink, SerializableFunctions.identity()); + WriteFiles write = WriteFiles.to(sink); DisplayData displayData = DisplayData.from(write); @@ -391,9 +374,7 @@ public void testUnboundedNeedsWindowed() { "Must use windowed writes when applying WriteFiles to an unbounded PCollection"); SimpleSink sink = makeSimpleSink(); - p.apply(Create.of("foo")) - .setIsBoundedInternal(IsBounded.UNBOUNDED) - .apply(WriteFiles.to(sink, SerializableFunctions.identity())); + p.apply(Create.of("foo")).setIsBoundedInternal(IsBounded.UNBOUNDED).apply(WriteFiles.to(sink)); p.run(); } @@ -408,19 +389,24 @@ public void testUnboundedNeedsSharding() { SimpleSink sink = makeSimpleSink(); p.apply(Create.of("foo")) .setIsBoundedInternal(IsBounded.UNBOUNDED) - .apply(WriteFiles.to(sink, SerializableFunctions.identity()).withWindowedWrites()); + .apply(WriteFiles.to(sink).withWindowedWrites()); p.run(); } // Test DynamicDestinations class. Expects user values to be string-encoded integers. // Stores the integer mod 5 as the destination, and uses that in the file prefix. - static class TestDestinations extends DynamicDestinations { + static class TestDestinations extends DynamicDestinations { private ResourceId baseOutputDirectory; TestDestinations(ResourceId baseOutputDirectory) { this.baseOutputDirectory = baseOutputDirectory; } + @Override + public String formatRecord(String record) { + return "record_" + record; + } + @Override public Integer getDestination(String element) { return Integer.valueOf(element) % 5; @@ -444,14 +430,6 @@ public void populateDisplayData(Builder builder) { } } - // Test format function. Prepend a string to each record before writing. - static class TestDynamicFormatFunction implements SerializableFunction { - @Override - public String apply(String input) { - return "record_" + input; - } - } - @Test @Category(NeedsRunner.class) public void testDynamicDestinationsBounded() throws Exception { @@ -495,8 +473,7 @@ private void testDynamicDestinationsHelper(boolean bounded, boolean emptyShards) // If emptyShards==true make numShards larger than the number of elements per destination. // This will force every destination to generate some empty shards. int numShards = emptyShards ? 2 * numInputs / 5 : 2; - WriteFiles writeFiles = - WriteFiles.to(sink, new TestDynamicFormatFunction()).withNumShards(numShards); + WriteFiles writeFiles = WriteFiles.to(sink).withNumShards(numShards); PCollection input = p.apply(Create.timestamped(inputs, timestamps)); if (!bounded) { @@ -521,7 +498,7 @@ private void testDynamicDestinationsHelper(boolean bounded, boolean emptyShards) @Test public void testShardedDisplayData() { - DynamicDestinations dynamicDestinations = + DynamicDestinations dynamicDestinations = DynamicFileDestinations.constant( DefaultFilenamePolicy.fromParams( new Params() @@ -537,8 +514,7 @@ public void populateDisplayData(DisplayData.Builder builder) { builder.add(DisplayData.item("foo", "bar")); } }; - WriteFiles write = - WriteFiles.to(sink, SerializableFunctions.identity()).withNumShards(1); + WriteFiles write = WriteFiles.to(sink).withNumShards(1); DisplayData displayData = DisplayData.from(write); assertThat(displayData, hasDisplayItem("sink", sink.getClass())); assertThat(displayData, includesDisplayDataFor("sink", sink)); @@ -547,7 +523,7 @@ public void populateDisplayData(DisplayData.Builder builder) { @Test public void testCustomShardStrategyDisplayData() { - DynamicDestinations dynamicDestinations = + DynamicDestinations dynamicDestinations = DynamicFileDestinations.constant( DefaultFilenamePolicy.fromParams( new Params() @@ -564,7 +540,7 @@ public void populateDisplayData(DisplayData.Builder builder) { } }; WriteFiles write = - WriteFiles.to(sink, SerializableFunctions.identity()) + WriteFiles.to(sink) .withSharding( new PTransform, PCollectionView>() { @Override diff --git a/sdks/java/io/xml/src/main/java/org/apache/beam/sdk/io/xml/XmlIO.java b/sdks/java/io/xml/src/main/java/org/apache/beam/sdk/io/xml/XmlIO.java index 442fba5c0bfcb..7255a94357eb4 100644 --- a/sdks/java/io/xml/src/main/java/org/apache/beam/sdk/io/xml/XmlIO.java +++ b/sdks/java/io/xml/src/main/java/org/apache/beam/sdk/io/xml/XmlIO.java @@ -36,7 +36,6 @@ 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.SerializableFunctions; import org.apache.beam.sdk.transforms.display.DisplayData; import org.apache.beam.sdk.values.PBegin; import org.apache.beam.sdk.values.PCollection; @@ -522,8 +521,7 @@ public void validate(PipelineOptions options) { @Override public PDone expand(PCollection input) { - return input.apply( - org.apache.beam.sdk.io.WriteFiles.to(createSink(), SerializableFunctions.identity())); + return input.apply(org.apache.beam.sdk.io.WriteFiles.to(createSink())); } @VisibleForTesting diff --git a/sdks/java/io/xml/src/main/java/org/apache/beam/sdk/io/xml/XmlSink.java b/sdks/java/io/xml/src/main/java/org/apache/beam/sdk/io/xml/XmlSink.java index 74e0bda25e77e..b66354489e6e7 100644 --- a/sdks/java/io/xml/src/main/java/org/apache/beam/sdk/io/xml/XmlSink.java +++ b/sdks/java/io/xml/src/main/java/org/apache/beam/sdk/io/xml/XmlSink.java @@ -35,7 +35,7 @@ import org.apache.beam.sdk.util.MimeTypes; /** Implementation of {@link XmlIO#write}. */ -class XmlSink extends FileBasedSink { +class XmlSink extends FileBasedSink { private static final String XML_EXTENSION = ".xml"; private final XmlIO.Write spec; @@ -46,7 +46,7 @@ private static DefaultFilenamePolicy makeFilenamePolicy(XmlIO.Write spec) } XmlSink(XmlIO.Write spec) { - super(spec.getFilenamePrefix(), DynamicFileDestinations.constant(makeFilenamePolicy(spec))); + super(spec.getFilenamePrefix(), DynamicFileDestinations.constant(makeFilenamePolicy(spec))); this.spec = spec; } @@ -77,7 +77,7 @@ void populateFileBasedDisplayData(DisplayData.Builder builder) { } /** {@link WriteOperation} for XML {@link FileBasedSink}s. */ - protected static final class XmlWriteOperation extends WriteOperation { + protected static final class XmlWriteOperation extends WriteOperation { public XmlWriteOperation(XmlSink sink) { super(sink); } @@ -112,7 +112,7 @@ ResourceId getTemporaryDirectory() { } /** A {@link Writer} that can write objects as XML elements. */ - protected static final class XmlWriter extends Writer { + protected static final class XmlWriter extends Writer { final Marshaller marshaller; private OutputStream os = null; From 04f27007968ae0c0be1fe2cb7c921c3dcfac27cf Mon Sep 17 00:00:00 2001 From: Eugene Kirpichov Date: Wed, 19 Jul 2017 19:50:03 -0700 Subject: [PATCH 217/346] [BEAM-2623] Introduces Watch transform The transform watches for new elements in a family of growing sets. See design at http://s.apache.org/beam-watch-transform As part of the implementation, I found and fixed a bug in tracking the watermark in OutputAndTimeBoundedSplittableProcessElementInvoker. The watermark must be captured at the moment checkpoint is taken, because it describes timestamps of elements output from the checkpoint. I also made direct runner by default checkpoint SDF's every 100 elements rather than every 10000, to make it more aggressive - that's what uncovered the bug above. --- ...oundedSplittableProcessElementInvoker.java | 44 +- ...ttableProcessElementsEvaluatorFactory.java | 6 +- .../org/apache/beam/sdk/transforms/Watch.java | 1009 +++++++++++++++++ .../apache/beam/sdk/transforms/WatchTest.java | 763 +++++++++++++ 4 files changed, 1803 insertions(+), 19 deletions(-) create mode 100644 sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Watch.java create mode 100644 sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/WatchTest.java diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/OutputAndTimeBoundedSplittableProcessElementInvoker.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/OutputAndTimeBoundedSplittableProcessElementInvoker.java index 0c956d53af9da..d830db503b758 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/OutputAndTimeBoundedSplittableProcessElementInvoker.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/OutputAndTimeBoundedSplittableProcessElementInvoker.java @@ -18,6 +18,7 @@ package org.apache.beam.runners.core; import static com.google.common.base.Preconditions.checkNotNull; +import static com.google.common.base.Preconditions.checkState; import com.google.common.collect.Iterables; import com.google.common.util.concurrent.Futures; @@ -37,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.WindowedValue; +import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TupleTag; import org.joda.time.Duration; @@ -158,7 +160,8 @@ public Timer timer(String timerId) { // TODO: verify that if there was a failed tryClaim() call, then cont.shouldResume() is false. // Currently we can't verify this because there are no hooks into tryClaim(). // See https://issues.apache.org/jira/browse/BEAM-2607 - RestrictionT residual = processContext.extractCheckpoint(); + processContext.cancelScheduledCheckpoint(); + KV residual = processContext.getTakenCheckpoint(); if (cont.shouldResume()) { if (residual == null) { // No checkpoint had been taken by the runner while the ProcessElement call ran, however @@ -166,7 +169,7 @@ public Timer timer(String timerId) { // a checkpoint now: checkpoint() guarantees that the primary restriction describes exactly // the work that was done in the current ProcessElement call, and returns a residual // restriction that describes exactly the work that wasn't done in the current call. - residual = tracker.checkpoint(); + residual = checkNotNull(processContext.takeCheckpointNow()); } else { // A checkpoint was taken by the runner, and then the ProcessElement call returned resume() // without making more tryClaim() calls (since no tryClaim() calls can succeed after @@ -185,7 +188,13 @@ public Timer timer(String timerId) { // special needs to be done. } tracker.checkDone(); - return new Result(residual, cont, processContext.getLastReportedWatermark()); + if (residual == null) { + // Can only be true if cont.shouldResume() is false and no checkpoint was taken. + // This means the restriction has been fully processed. + checkState(!cont.shouldResume()); + return new Result(null, cont, BoundedWindow.TIMESTAMP_MAX_VALUE); + } + return new Result(residual.getKey(), cont, residual.getValue()); } private class ProcessContext extends DoFn.ProcessContext { @@ -199,6 +208,9 @@ private class ProcessContext extends DoFn.ProcessContext { // This is either the result of the sole tracker.checkpoint() call, or null if // the call completed before reaching the given number of outputs or duration. private RestrictionT checkpoint; + // Watermark captured at the moment before checkpoint was taken, describing a lower bound + // on the output from "checkpoint". + private Instant residualWatermark; // A handle on the scheduled action to take a checkpoint. private Future scheduledCheckpoint; private Instant lastReportedWatermark; @@ -213,34 +225,36 @@ public ProcessContext(WindowedValue element, TrackerT tracker) { new Runnable() { @Override public void run() { - initiateCheckpoint(); + takeCheckpointNow(); } }, maxDuration.getMillis(), TimeUnit.MILLISECONDS); } - @Nullable - RestrictionT extractCheckpoint() { + void cancelScheduledCheckpoint() { scheduledCheckpoint.cancel(true); try { Futures.getUnchecked(scheduledCheckpoint); } catch (CancellationException e) { // This is expected if the call took less than the maximum duration. } - // By now, a checkpoint may or may not have been taken; - // via .output() or via scheduledCheckpoint. - synchronized (this) { - return checkpoint; - } } - private synchronized void initiateCheckpoint() { + synchronized KV takeCheckpointNow() { // This method may be entered either via .output(), or via scheduledCheckpoint. // Only one of them "wins" - tracker.checkpoint() must be called only once. if (checkpoint == null) { + residualWatermark = lastReportedWatermark; checkpoint = checkNotNull(tracker.checkpoint()); } + return getTakenCheckpoint(); + } + + @Nullable + synchronized KV getTakenCheckpoint() { + // The checkpoint may or may not have been taken. + return (checkpoint == null) ? null : KV.of(checkpoint, residualWatermark); } @Override @@ -271,10 +285,6 @@ public synchronized void updateWatermark(Instant watermark) { lastReportedWatermark = watermark; } - public synchronized Instant getLastReportedWatermark() { - return lastReportedWatermark; - } - @Override public PipelineOptions getPipelineOptions() { return pipelineOptions; @@ -306,7 +316,7 @@ public void outputWithTimestamp(TupleTag tag, T value, Instant timestamp) private void noteOutput() { ++numOutputs; if (numOutputs >= maxNumOutputs) { - initiateCheckpoint(); + takeCheckpointNow(); } } } 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 e6b51b79ce7f3..bc7b193e75301 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 @@ -178,8 +178,10 @@ public void outputWindowedValue( .setDaemon(true) .setNameFormat("direct-splittable-process-element-checkpoint-executor") .build()), - 10000, - Duration.standardSeconds(10))); + // Setting small values here to stimulate frequent checkpointing and better exercise + // splittable DoFn's in that respect. + 100, + Duration.standardSeconds(1))); return DoFnLifecycleManagerRemovingTransformEvaluator.wrapping(parDoEvaluator, fnManager); } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Watch.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Watch.java new file mode 100644 index 0000000000000..b21eb62226609 --- /dev/null +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Watch.java @@ -0,0 +1,1009 @@ +/* + * 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.transforms; + +import static com.google.common.base.Preconditions.checkArgument; +import static com.google.common.base.Preconditions.checkNotNull; +import static com.google.common.base.Preconditions.checkState; +import static org.apache.beam.sdk.transforms.DoFn.ProcessContinuation.resume; +import static org.apache.beam.sdk.transforms.DoFn.ProcessContinuation.stop; + +import com.google.auto.value.AutoValue; +import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Function; +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; +import com.google.common.collect.Ordering; +import com.google.common.hash.Funnel; +import com.google.common.hash.Funnels; +import com.google.common.hash.HashCode; +import com.google.common.hash.Hashing; +import com.google.common.hash.PrimitiveSink; +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.io.Serializable; +import java.lang.reflect.TypeVariable; +import java.util.Arrays; +import java.util.Collections; +import java.util.LinkedList; +import java.util.List; +import java.util.Map; +import javax.annotation.Nullable; +import org.apache.beam.sdk.annotations.Experimental; +import org.apache.beam.sdk.coders.AtomicCoder; +import org.apache.beam.sdk.coders.CannotProvideCoderException; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.coders.DurationCoder; +import org.apache.beam.sdk.coders.InstantCoder; +import org.apache.beam.sdk.coders.KvCoder; +import org.apache.beam.sdk.coders.ListCoder; +import org.apache.beam.sdk.coders.MapCoder; +import org.apache.beam.sdk.coders.NullableCoder; +import org.apache.beam.sdk.coders.StructuredCoder; +import org.apache.beam.sdk.coders.VarIntCoder; +import org.apache.beam.sdk.transforms.splittabledofn.RestrictionTracker; +import org.apache.beam.sdk.transforms.windowing.BoundedWindow; +import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.TimestampedValue; +import org.apache.beam.sdk.values.TypeDescriptor; +import org.joda.time.Duration; +import org.joda.time.Instant; +import org.joda.time.ReadableDuration; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Given a "poll function" that produces a potentially growing set of outputs for an input, this + * transform simultaneously continuously watches the growth of output sets of all inputs, until a + * per-input termination condition is reached. + * + *

              The output is returned as an unbounded {@link PCollection} of {@code KV}, + * where each {@code OutputT} is associated with the {@code InputT} that produced it, and is + * assigned with the timestamp that the poll function returned when this output was detected for the + * first time. + * + *

              Hypothetical usage example for watching new files in a collection of directories, where for + * each directory we assume that new files will not appear if the directory contains a file named + * ".complete": + * + *

              {@code
              + * PCollection directories = ...;  // E.g. Create.of(single directory)
              + * PCollection> matches = filepatterns.apply(Watch.growthOf(
              + *   new PollFn() {
              + *     public PollResult apply(TimestampedValue input) {
              + *       String directory = input.getValue();
              + *       List> outputs = new ArrayList<>();
              + *       ... List the directory and get creation times of all files ...
              + *       boolean isComplete = ... does a file ".complete" exist in the directory ...
              + *       return isComplete ? PollResult.complete(outputs) : PollResult.incomplete(outputs);
              + *     }
              + *   })
              + *   // Poll each directory every 5 seconds
              + *   .withPollInterval(Duration.standardSeconds(5))
              + *   // Stop watching each directory 12 hours after it's seen even if it's incomplete
              + *   .withTerminationPerInput(afterTotalOf(Duration.standardHours(12)));
              + * }
              + * + *

              By default, the watermark for a particular input is computed from a poll result as "earliest + * timestamp of new elements in this poll result". It can also be set explicitly via {@link + * Growth.PollResult#withWatermark} if the {@link Growth.PollFn} can provide a more optimistic + * estimate. + * + *

              Note: This transform works only in runners supporting Splittable DoFn: see capability matrix. + */ +@Experimental(Experimental.Kind.SPLITTABLE_DO_FN) +public class Watch { + private static final Logger LOG = LoggerFactory.getLogger(Watch.class); + + /** Watches the growth of the given poll function. See class documentation for more details. */ + public static Growth growthOf( + Growth.PollFn pollFn) { + return new AutoValue_Watch_Growth.Builder() + .setTerminationPerInput(Watch.Growth.never()) + .setPollFn(pollFn) + .build(); + } + + /** Implementation of {@link #growthOf}. */ + @AutoValue + public abstract static class Growth + extends PTransform, PCollection>> { + /** The result of a single invocation of a {@link PollFn}. */ + public static final class PollResult { + private final List> outputs; + // null means unspecified (infer automatically). + @Nullable private final Instant watermark; + + private PollResult(List> outputs, @Nullable Instant watermark) { + this.outputs = outputs; + this.watermark = watermark; + } + + List> getOutputs() { + return outputs; + } + + @Nullable + Instant getWatermark() { + return watermark; + } + + /** + * Sets the watermark - an approximate lower bound on timestamps of future new outputs from + * this {@link PollFn}. + */ + public PollResult withWatermark(Instant watermark) { + checkNotNull(watermark, "watermark"); + return new PollResult<>(outputs, watermark); + } + + /** + * Constructs a {@link PollResult} with the given outputs and declares that there will be no + * new outputs for the current input. The {@link PollFn} will not be called again for this + * input. + */ + public static PollResult complete( + List> outputs) { + return new PollResult<>(outputs, BoundedWindow.TIMESTAMP_MAX_VALUE); + } + + /** Like {@link #complete(List)}, but assigns the same timestamp to all new outputs. */ + public static PollResult complete( + Instant timestamp, List outputs) { + return new PollResult<>( + addTimestamp(timestamp, outputs), BoundedWindow.TIMESTAMP_MAX_VALUE); + } + + /** + * Constructs a {@link PollResult} with the given outputs and declares that new outputs might + * appear for the current input. By default, {@link Watch} will estimate the watermark for + * future new outputs as equal to the earliest of the new outputs from this {@link + * PollResult}. To specify a more exact watermark, use {@link #withWatermark(Instant)}. + */ + public static PollResult incomplete( + List> outputs) { + return new PollResult<>(outputs, null); + } + + /** Like {@link #incomplete(List)}, but assigns the same timestamp to all new outputs. */ + public static PollResult incomplete( + Instant timestamp, List outputs) { + return new PollResult<>(addTimestamp(timestamp, outputs), null); + } + + private static List> addTimestamp( + Instant timestamp, List outputs) { + List> res = Lists.newArrayListWithExpectedSize(outputs.size()); + for (OutputT output : outputs) { + res.add(TimestampedValue.of(output, timestamp)); + } + return res; + } + } + + /** + * A function that computes the current set of outputs for the given input (given as a {@link + * TimestampedValue}), in the form of a {@link PollResult}. + */ + public interface PollFn extends Serializable { + PollResult apply(InputT input, Instant timestamp) throws Exception; + } + + /** + * A strategy for determining whether it is time to stop polling the current input regardless of + * whether its output is complete or not. + * + *

              Some built-in termination conditions are {@link #never}, {@link #afterTotalOf} and {@link + * #afterTimeSinceNewOutput}. Conditions can be combined using {@link #eitherOf} and {@link + * #allOf}. Users can also develop custom termination conditions, for example, one might imagine + * a condition that terminates after a given time after the first output appears for the input + * (unlike {@link #afterTotalOf} which operates relative to when the input itself arrives). + * + *

              A {@link TerminationCondition} is provided to {@link + * Growth#withTerminationPerInput(TerminationCondition)} and is used to maintain an independent + * state of the termination condition for every input, represented as {@code StateT} which must + * be immutable, non-null, and encodable via {@link #getStateCoder()}. + * + *

              All functions take the wall-clock timestamp as {@link Instant} for convenience of + * unit-testing custom termination conditions. + */ + public interface TerminationCondition extends Serializable { + /** Used to encode the state of this {@link TerminationCondition}. */ + Coder getStateCoder(); + + /** + * Called by the {@link Watch} transform to create a new independent termination state for a + * newly arrived {@code InputT}. + */ + StateT forNewInput(Instant now, InputT input); + + /** + * Called by the {@link Watch} transform to compute a new termination state, in case after + * calling the {@link PollFn} for the current input, the {@link PollResult} included a + * previously unseen {@code OutputT}. + */ + StateT onSeenNewOutput(Instant now, StateT state); + + /** + * Called by the {@link Watch} transform to determine whether the given termination state + * signals that {@link Watch} should stop calling {@link PollFn} for the current input, + * regardless of whether the last {@link PollResult} was complete or incomplete. + */ + boolean canStopPolling(Instant now, StateT state); + + /** Creates a human-readable representation of the given state of this condition. */ + String toString(StateT state); + } + + /** + * Returns a {@link TerminationCondition} that never holds (i.e., poll each input until its + * output is complete). + */ + public static Never never() { + return new Never<>(); + } + + /** + * Returns a {@link TerminationCondition} that holds after the given time has elapsed after the + * current input was seen. + */ + public static AfterTotalOf afterTotalOf(ReadableDuration timeSinceInput) { + return afterTotalOf(SerializableFunctions.constant(timeSinceInput)); + } + + /** Like {@link #afterTotalOf(ReadableDuration)}, but the duration is input-dependent. */ + public static AfterTotalOf afterTotalOf( + SerializableFunction timeSinceInput) { + return new AfterTotalOf<>(timeSinceInput); + } + + /** + * Returns a {@link TerminationCondition} that holds after the given time has elapsed after the + * last time the {@link PollResult} for the current input contained a previously unseen output. + */ + public static AfterTimeSinceNewOutput afterTimeSinceNewOutput( + ReadableDuration timeSinceNewOutput) { + return afterTimeSinceNewOutput( + SerializableFunctions.constant(timeSinceNewOutput)); + } + + /** + * Like {@link #afterTimeSinceNewOutput(ReadableDuration)}, but the duration is input-dependent. + */ + public static AfterTimeSinceNewOutput afterTimeSinceNewOutput( + SerializableFunction timeSinceNewOutput) { + return new AfterTimeSinceNewOutput<>(timeSinceNewOutput); + } + + /** + * Returns a {@link TerminationCondition} that holds when at least one of the given two + * conditions holds. + */ + public static + BinaryCombined eitherOf( + TerminationCondition first, + TerminationCondition second) { + return new BinaryCombined<>(BinaryCombined.Operation.OR, first, second); + } + + /** + * Returns a {@link TerminationCondition} that holds when both of the given two conditions hold. + */ + public static + BinaryCombined allOf( + TerminationCondition first, + TerminationCondition second) { + return new BinaryCombined<>(BinaryCombined.Operation.AND, first, second); + } + + // Uses Integer rather than Void for state, because termination state must be non-null. + static class Never implements TerminationCondition { + @Override + public Coder getStateCoder() { + return VarIntCoder.of(); + } + + @Override + public Integer forNewInput(Instant now, InputT input) { + return 0; + } + + @Override + public Integer onSeenNewOutput(Instant now, Integer state) { + return state; + } + + @Override + public boolean canStopPolling(Instant now, Integer state) { + return false; + } + + @Override + public String toString(Integer state) { + return "Never"; + } + } + + static class AfterTotalOf + implements TerminationCondition< + InputT, KV> { + private final SerializableFunction maxTimeSinceInput; + + private AfterTotalOf(SerializableFunction maxTimeSinceInput) { + this.maxTimeSinceInput = maxTimeSinceInput; + } + + @Override + public Coder> getStateCoder() { + return KvCoder.of(InstantCoder.of(), DurationCoder.of()); + } + + @Override + public KV forNewInput(Instant now, InputT input) { + return KV.of(now, maxTimeSinceInput.apply(input)); + } + + @Override + public KV onSeenNewOutput( + Instant now, KV state) { + return state; + } + + @Override + public boolean canStopPolling(Instant now, KV state) { + return new Duration(state.getKey(), now).isLongerThan(state.getValue()); + } + + @Override + public String toString(KV state) { + return "AfterTotalOf{" + + "timeStarted=" + + state.getKey() + + ", maxTimeSinceInput=" + + state.getValue() + + '}'; + } + } + + static class AfterTimeSinceNewOutput + implements TerminationCondition< + InputT, + KV> { + private final SerializableFunction maxTimeSinceNewOutput; + + private AfterTimeSinceNewOutput( + SerializableFunction maxTimeSinceNewOutput) { + this.maxTimeSinceNewOutput = maxTimeSinceNewOutput; + } + + @Override + public Coder> getStateCoder() { + return KvCoder.of(NullableCoder.of(InstantCoder.of()), DurationCoder.of()); + } + + @Override + public KV forNewInput(Instant now, InputT input) { + return KV.of(null, maxTimeSinceNewOutput.apply(input)); + } + + @Override + public KV onSeenNewOutput( + Instant now, KV state) { + return KV.of(now, state.getValue()); + } + + @Override + public boolean canStopPolling(Instant now, KV state) { + Instant timeOfLastNewOutput = state.getKey(); + ReadableDuration maxTimeSinceNewOutput = state.getValue(); + return timeOfLastNewOutput != null + && new Duration(timeOfLastNewOutput, now).isLongerThan(maxTimeSinceNewOutput); + } + + @Override + public String toString(KV state) { + return "AfterTimeSinceNewOutput{" + + "timeOfLastNewOutput=" + + state.getKey() + + ", maxTimeSinceNewOutput=" + + state.getValue() + + '}'; + } + } + + static class BinaryCombined + implements TerminationCondition> { + private enum Operation { + OR, + AND + } + + private final Operation operation; + private final TerminationCondition first; + private final TerminationCondition second; + + public BinaryCombined( + Operation operation, + TerminationCondition first, + TerminationCondition second) { + this.operation = operation; + this.first = first; + this.second = second; + } + + @Override + public Coder> getStateCoder() { + return KvCoder.of(first.getStateCoder(), second.getStateCoder()); + } + + @Override + public KV forNewInput(Instant now, InputT input) { + return KV.of(first.forNewInput(now, input), second.forNewInput(now, input)); + } + + @Override + public KV onSeenNewOutput( + Instant now, KV state) { + return KV.of( + first.onSeenNewOutput(now, state.getKey()), + second.onSeenNewOutput(now, state.getValue())); + } + + @Override + public boolean canStopPolling(Instant now, KV state) { + switch (operation) { + case OR: + return first.canStopPolling(now, state.getKey()) + || second.canStopPolling(now, state.getValue()); + case AND: + return first.canStopPolling(now, state.getKey()) + && second.canStopPolling(now, state.getValue()); + default: + throw new UnsupportedOperationException("Unexpected operation " + operation); + } + } + + @Override + public String toString(KV state) { + return operation + + "{first=" + + first.toString(state.getKey()) + + ", second=" + + second.toString(state.getValue()) + + '}'; + } + } + + abstract PollFn getPollFn(); + + @Nullable + abstract Duration getPollInterval(); + + @Nullable + abstract TerminationCondition getTerminationPerInput(); + + @Nullable + abstract Coder getOutputCoder(); + + abstract Builder toBuilder(); + + @AutoValue.Builder + abstract static class Builder { + abstract Builder setPollFn(PollFn pollFn); + + abstract Builder setTerminationPerInput( + TerminationCondition terminationPerInput); + + abstract Builder setPollInterval(Duration pollInterval); + + abstract Builder setOutputCoder(Coder outputCoder); + + abstract Growth build(); + } + + /** Specifies a {@link TerminationCondition} that will be independently used for every input. */ + public Growth withTerminationPerInput( + TerminationCondition terminationPerInput) { + return toBuilder().setTerminationPerInput(terminationPerInput).build(); + } + + /** + * Specifies how long to wait after a call to {@link PollFn} before calling it again (if at all + * - according to {@link PollResult} and the {@link TerminationCondition}). + */ + public Growth withPollInterval(Duration pollInterval) { + return toBuilder().setPollInterval(pollInterval).build(); + } + + /** + * Specifies a {@link Coder} to use for the outputs. If unspecified, it will be inferred from + * the output type of {@link PollFn} whenever possible. + * + *

              The coder must be deterministic, because the transform will compare encoded outputs for + * deduplication between polling rounds. + */ + public Growth withOutputCoder(Coder outputCoder) { + return toBuilder().setOutputCoder(outputCoder).build(); + } + + @Override + public PCollection> expand(PCollection input) { + checkNotNull(getPollInterval(), "pollInterval"); + checkNotNull(getTerminationPerInput(), "terminationPerInput"); + + Coder outputCoder = getOutputCoder(); + if (outputCoder == null) { + // If a coder was not specified explicitly, infer it from the OutputT type parameter + // of the PollFn. + TypeDescriptor superDescriptor = + TypeDescriptor.of(getPollFn().getClass()).getSupertype(PollFn.class); + TypeVariable typeVariable = superDescriptor.getTypeParameter("OutputT"); + @SuppressWarnings("unchecked") + TypeDescriptor descriptor = + (TypeDescriptor) superDescriptor.resolveType(typeVariable); + try { + outputCoder = input.getPipeline().getCoderRegistry().getCoder(descriptor); + } catch (CannotProvideCoderException e) { + throw new RuntimeException( + "Unable to infer coder for OutputT. Specify it explicitly using withOutputCoder()."); + } + } + try { + outputCoder.verifyDeterministic(); + } catch (Coder.NonDeterministicException e) { + throw new IllegalArgumentException( + "Output coder " + outputCoder + " must be deterministic"); + } + + return input + .apply(ParDo.of(new WatchGrowthFn<>(this, outputCoder))) + .setCoder(KvCoder.of(input.getCoder(), outputCoder)); + } + } + + private static class WatchGrowthFn + extends DoFn> { + private final Watch.Growth spec; + private final Coder outputCoder; + + private WatchGrowthFn(Growth spec, Coder outputCoder) { + this.spec = spec; + this.outputCoder = outputCoder; + } + + @ProcessElement + public ProcessContinuation process( + ProcessContext c, final GrowthTracker tracker) + throws Exception { + if (!tracker.hasPending() && !tracker.currentRestriction().isOutputComplete) { + LOG.debug("{} - polling input", c.element()); + Growth.PollResult res = spec.getPollFn().apply(c.element(), c.timestamp()); + // TODO (https://issues.apache.org/jira/browse/BEAM-2680): + // Consider truncating the pending outputs if there are too many, to avoid blowing + // up the state. In that case, we'd rely on the next poll cycle to provide more outputs. + // All outputs would still have to be stored in state.completed, but it is more compact + // because it stores hashes and because it could potentially be garbage-collected. + int numPending = tracker.addNewAsPending(res); + if (numPending > 0) { + LOG.info( + "{} - polling returned {} results, of which {} were new. The output is {}.", + c.element(), + res.getOutputs().size(), + numPending, + BoundedWindow.TIMESTAMP_MAX_VALUE.equals(res.getWatermark()) + ? "complete" + : "incomplete"); + } + } + while (tracker.hasPending()) { + c.updateWatermark(tracker.getWatermark()); + + TimestampedValue nextPending = tracker.tryClaimNextPending(); + if (nextPending == null) { + return stop(); + } + c.outputWithTimestamp( + KV.of(c.element(), nextPending.getValue()), nextPending.getTimestamp()); + } + Instant watermark = tracker.getWatermark(); + if (watermark != null) { + // Null means the poll result did not provide a watermark and there were no new elements, + // so we have no information to update the watermark and should keep it as-is. + c.updateWatermark(watermark); + } + // No more pending outputs - future output will come from more polling, + // unless output is complete or termination condition is reached. + if (tracker.shouldPollMore()) { + return resume().withResumeDelay(spec.getPollInterval()); + } + return stop(); + } + + private Growth.TerminationCondition getTerminationCondition() { + return ((Growth.TerminationCondition) + spec.getTerminationPerInput()); + } + + @GetInitialRestriction + public GrowthState getInitialRestriction(InputT element) { + return new GrowthState<>(getTerminationCondition().forNewInput(Instant.now(), element)); + } + + @NewTracker + public GrowthTracker newTracker( + GrowthState restriction) { + return new GrowthTracker<>(outputCoder, restriction, getTerminationCondition()); + } + + @GetRestrictionCoder + @SuppressWarnings({"unchecked", "rawtypes"}) + public Coder> getRestrictionCoder() { + return GrowthStateCoder.of( + outputCoder, (Coder) spec.getTerminationPerInput().getStateCoder()); + } + } + + @VisibleForTesting + static class GrowthState { + // Hashes and timestamps of outputs that have already been output and should be omitted + // from future polls. Timestamps are preserved to allow garbage-collecting this state + // in the future, e.g. dropping elements from "completed" and from addNewAsPending() if their + // timestamp is more than X behind the watermark. + // As of writing, we don't do this, but preserve the information for forward compatibility + // in case of pipeline update. TODO: do this. + private final Map completed; + // Outputs that are known to be present in a poll result, but have not yet been returned + // from a ProcessElement call, sorted by timestamp to help smooth watermark progress. + private final List> pending; + // If true, processing of this restriction should only output "pending". Otherwise, it should + // also continue polling. + private final boolean isOutputComplete; + // Can be null only if isOutputComplete is true. + @Nullable private final TerminationStateT terminationState; + // A lower bound on timestamps of future outputs from PollFn, excluding completed and pending. + @Nullable private final Instant pollWatermark; + + GrowthState(TerminationStateT terminationState) { + this.completed = Collections.emptyMap(); + this.pending = Collections.emptyList(); + this.isOutputComplete = false; + this.terminationState = checkNotNull(terminationState); + this.pollWatermark = BoundedWindow.TIMESTAMP_MIN_VALUE; + } + + GrowthState( + Map completed, + List> pending, + boolean isOutputComplete, + @Nullable TerminationStateT terminationState, + @Nullable Instant pollWatermark) { + if (!isOutputComplete) { + checkNotNull(terminationState); + } + this.completed = Collections.unmodifiableMap(completed); + this.pending = Collections.unmodifiableList(pending); + this.isOutputComplete = isOutputComplete; + this.terminationState = terminationState; + this.pollWatermark = pollWatermark; + } + + public String toString(Growth.TerminationCondition terminationCondition) { + return "GrowthState{" + + "completed=<" + + completed.size() + + " elements>, pending=<" + + pending.size() + + " elements" + + (pending.isEmpty() ? "" : (", earliest " + pending.get(0))) + + ">, isOutputComplete=" + + isOutputComplete + + ", terminationState=" + + terminationCondition.toString(terminationState) + + ", pollWatermark=" + + pollWatermark + + '}'; + } + } + + @VisibleForTesting + static class GrowthTracker + implements RestrictionTracker> { + private final Funnel coderFunnel; + private final Growth.TerminationCondition terminationCondition; + + // The restriction describing the entire work to be done by the current ProcessElement call. + // Changes only in checkpoint(). + private GrowthState state; + + // Mutable state changed by the ProcessElement call itself, and used to compute the primary + // and residual restrictions in checkpoint(). + + // Remaining pending outputs; initialized from state.pending (if non-empty) or in + // addNewAsPending(); drained via tryClaimNextPending(). + private LinkedList> pending; + // Outputs that have been claimed in the current ProcessElement call. A prefix of "pending". + private List> claimed = Lists.newArrayList(); + private boolean isOutputComplete; + private TerminationStateT terminationState; + @Nullable private Instant pollWatermark; + private boolean shouldStop = false; + + GrowthTracker(final Coder outputCoder, GrowthState state, + Growth.TerminationCondition terminationCondition) { + this.coderFunnel = + new Funnel() { + @Override + public void funnel(OutputT from, PrimitiveSink into) { + try { + outputCoder.encode(from, Funnels.asOutputStream(into)); + } catch (IOException e) { + throw new RuntimeException(e); + } + } + }; + this.terminationCondition = terminationCondition; + this.state = state; + this.isOutputComplete = state.isOutputComplete; + this.pollWatermark = state.pollWatermark; + this.terminationState = state.terminationState; + this.pending = Lists.newLinkedList(state.pending); + } + + @Override + public synchronized GrowthState currentRestriction() { + return state; + } + + @Override + public synchronized GrowthState checkpoint() { + // primary should contain exactly the work claimed in the current ProcessElement call - i.e. + // claimed outputs become pending, and it shouldn't poll again. + GrowthState primary = + new GrowthState<>( + state.completed /* completed */, + claimed /* pending */, + true /* isOutputComplete */, + null /* terminationState */, + BoundedWindow.TIMESTAMP_MAX_VALUE /* pollWatermark */); + + // residual should contain exactly the work *not* claimed in the current ProcessElement call - + // unclaimed pending outputs plus future polling outputs. + Map newCompleted = Maps.newHashMap(state.completed); + for (TimestampedValue claimedOutput : claimed) { + newCompleted.put(hash128(claimedOutput.getValue()), claimedOutput.getTimestamp()); + } + GrowthState residual = + new GrowthState<>( + newCompleted /* completed */, + pending /* pending */, + isOutputComplete /* isOutputComplete */, + terminationState, + pollWatermark); + + // Morph ourselves into primary, except for "pending" - the current call has already claimed + // everything from it. + this.state = primary; + this.isOutputComplete = primary.isOutputComplete; + this.pollWatermark = primary.pollWatermark; + this.terminationState = null; + this.pending = Lists.newLinkedList(); + + this.shouldStop = true; + return residual; + } + + private HashCode hash128(OutputT value) { + return Hashing.murmur3_128().hashObject(value, coderFunnel); + } + + @Override + public synchronized void checkDone() throws IllegalStateException { + if (shouldStop) { + return; + } + checkState(!shouldPollMore(), "Polling is still allowed to continue"); + checkState(pending.isEmpty(), "There are %s unclaimed pending outputs", pending.size()); + } + + @VisibleForTesting + synchronized boolean hasPending() { + return !pending.isEmpty(); + } + + @VisibleForTesting + @Nullable + synchronized TimestampedValue tryClaimNextPending() { + if (shouldStop) { + return null; + } + checkState(!pending.isEmpty(), "No more unclaimed pending outputs"); + TimestampedValue value = pending.removeFirst(); + claimed.add(value); + return value; + } + + @VisibleForTesting + synchronized boolean shouldPollMore() { + return !isOutputComplete + && !terminationCondition.canStopPolling(Instant.now(), terminationState); + } + + @VisibleForTesting + synchronized int addNewAsPending(Growth.PollResult pollResult) { + checkState( + state.pending.isEmpty(), + "Should have drained all old pending outputs before adding new, " + + "but there are %s old pending outputs", + state.pending.size()); + List> newPending = Lists.newArrayList(); + for (TimestampedValue output : pollResult.getOutputs()) { + OutputT value = output.getValue(); + if (state.completed.containsKey(hash128(value))) { + continue; + } + // TODO (https://issues.apache.org/jira/browse/BEAM-2680): + // Consider adding only at most N pending elements and ignoring others, + // instead relying on future poll rounds to provide them, in order to avoid + // blowing up the state. Combined with garbage collection of GrowthState.completed, + // this would make the transform scalable to very large poll results. + newPending.add(TimestampedValue.of(value, output.getTimestamp())); + } + if (!newPending.isEmpty()) { + terminationState = terminationCondition.onSeenNewOutput(Instant.now(), terminationState); + } + this.pending = + Lists.newLinkedList( + Ordering.natural() + .onResultOf( + new Function, Instant>() { + @Override + public Instant apply(TimestampedValue output) { + return output.getTimestamp(); + } + }) + .sortedCopy(newPending)); + // If poll result doesn't provide a watermark, assume that future new outputs may + // arrive with about the same timestamps as the current new outputs. + if (pollResult.getWatermark() != null) { + this.pollWatermark = pollResult.getWatermark(); + } else if (!pending.isEmpty()) { + this.pollWatermark = pending.getFirst().getTimestamp(); + } + if (BoundedWindow.TIMESTAMP_MAX_VALUE.equals(pollWatermark)) { + isOutputComplete = true; + } + return pending.size(); + } + + @VisibleForTesting + synchronized Instant getWatermark() { + // Future elements that can be claimed in this restriction come either from + // "pending" or from future polls, so the total watermark is + // min(watermark for future polling, earliest remaining pending element) + return Ordering.natural() + .nullsLast() + .min(pollWatermark, pending.isEmpty() ? null : pending.getFirst().getTimestamp()); + } + + @Override + public synchronized String toString() { + return "GrowthTracker{" + + "state=" + + state.toString(terminationCondition) + + ", pending=<" + + pending.size() + + " elements" + + (pending.isEmpty() ? "" : (", earliest " + pending.get(0))) + + ">, claimed=<" + + claimed.size() + + " elements>, isOutputComplete=" + + isOutputComplete + + ", terminationState=" + + terminationState + + ", pollWatermark=" + + pollWatermark + + ", shouldStop=" + + shouldStop + + '}'; + } + } + + private static class HashCode128Coder extends AtomicCoder { + private static final HashCode128Coder INSTANCE = new HashCode128Coder(); + + public static HashCode128Coder of() { + return INSTANCE; + } + + @Override + public void encode(HashCode value, OutputStream os) throws IOException { + checkArgument( + value.bits() == 128, "Expected a 128-bit hash code, but got %s bits", value.bits()); + byte[] res = new byte[16]; + value.writeBytesTo(res, 0, 16); + os.write(res); + } + + @Override + public HashCode decode(InputStream is) throws IOException { + byte[] res = new byte[16]; + int numRead = is.read(res, 0, 16); + checkArgument(numRead == 16, "Expected to read 16 bytes, but read %s", numRead); + return HashCode.fromBytes(res); + } + } + + private static class GrowthStateCoder + extends StructuredCoder> { + public static GrowthStateCoder of( + Coder outputCoder, Coder terminationStateCoder) { + return new GrowthStateCoder<>(outputCoder, terminationStateCoder); + } + + private static final Coder INT_CODER = VarIntCoder.of(); + private static final Coder INSTANT_CODER = NullableCoder.of(InstantCoder.of()); + private static final Coder HASH_CODE_CODER = HashCode128Coder.of(); + + private final Coder outputCoder; + private final Coder> completedCoder; + private final Coder>> pendingCoder; + private final Coder terminationStateCoder; + + private GrowthStateCoder( + Coder outputCoder, Coder terminationStateCoder) { + this.outputCoder = outputCoder; + this.terminationStateCoder = terminationStateCoder; + this.completedCoder = MapCoder.of(HASH_CODE_CODER, INSTANT_CODER); + this.pendingCoder = ListCoder.of(TimestampedValue.TimestampedValueCoder.of(outputCoder)); + } + + @Override + public void encode(GrowthState value, OutputStream os) + throws IOException { + completedCoder.encode(value.completed, os); + pendingCoder.encode(value.pending, os); + INT_CODER.encode(value.isOutputComplete ? 1 : 0, os); + terminationStateCoder.encode(value.terminationState, os); + INSTANT_CODER.encode(value.pollWatermark, os); + } + + @Override + public GrowthState decode(InputStream is) throws IOException { + Map completed = completedCoder.decode(is); + List> pending = pendingCoder.decode(is); + boolean isOutputComplete = (INT_CODER.decode(is) == 1); + TerminationStateT terminationState = terminationStateCoder.decode(is); + Instant pollWatermark = INSTANT_CODER.decode(is); + return new GrowthState<>( + completed, pending, isOutputComplete, terminationState, pollWatermark); + } + + @Override + public List> getCoderArguments() { + return Arrays.asList(outputCoder, terminationStateCoder); + } + + @Override + public void verifyDeterministic() throws NonDeterministicException { + outputCoder.verifyDeterministic(); + } + } +} diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/WatchTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/WatchTest.java new file mode 100644 index 0000000000000..132a1ff582ad4 --- /dev/null +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/WatchTest.java @@ -0,0 +1,763 @@ +/* + * 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.transforms; + +import static org.apache.beam.sdk.transforms.Watch.Growth.afterTimeSinceNewOutput; +import static org.apache.beam.sdk.transforms.Watch.Growth.afterTotalOf; +import static org.apache.beam.sdk.transforms.Watch.Growth.allOf; +import static org.apache.beam.sdk.transforms.Watch.Growth.eitherOf; +import static org.apache.beam.sdk.transforms.Watch.Growth.never; +import static org.hamcrest.Matchers.greaterThan; +import static org.joda.time.Duration.standardSeconds; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +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 com.google.common.base.Function; +import com.google.common.collect.Iterables; +import com.google.common.collect.Lists; +import com.google.common.collect.Ordering; +import com.google.common.collect.Sets; +import com.google.common.hash.HashCode; +import java.io.Serializable; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.UUID; +import javax.annotation.Nullable; +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.testing.NeedsRunner; +import org.apache.beam.sdk.testing.PAssert; +import org.apache.beam.sdk.testing.TestPipeline; +import org.apache.beam.sdk.testing.UsesSplittableParDo; +import org.apache.beam.sdk.transforms.Watch.Growth.PollFn; +import org.apache.beam.sdk.transforms.Watch.Growth.PollResult; +import org.apache.beam.sdk.transforms.Watch.GrowthState; +import org.apache.beam.sdk.transforms.Watch.GrowthTracker; +import org.apache.beam.sdk.transforms.windowing.BoundedWindow; +import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.TimestampedValue; +import org.joda.time.Duration; +import org.joda.time.Instant; +import org.joda.time.ReadableDuration; +import org.junit.Rule; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +/** Tests for {@link Watch}. */ +@RunWith(JUnit4.class) +public class WatchTest implements Serializable { + @Rule public transient TestPipeline p = TestPipeline.create(); + + @Test + @Category({NeedsRunner.class, UsesSplittableParDo.class}) + public void testSinglePollMultipleInputs() { + PCollection> res = + p.apply(Create.of("a", "b")) + .apply( + Watch.growthOf( + new PollFn() { + @Override + public PollResult apply(String input, Instant time) { + return PollResult.complete( + time, Arrays.asList(input + ".foo", input + ".bar")); + } + }) + .withPollInterval(Duration.ZERO)); + + PAssert.that(res) + .containsInAnyOrder( + Arrays.asList( + KV.of("a", "a.foo"), KV.of("a", "a.bar"), + KV.of("b", "b.foo"), KV.of("b", "b.bar"))); + + p.run(); + } + + @Test + @Category({NeedsRunner.class, UsesSplittableParDo.class}) + public void testMultiplePollsWithTerminationBecauseOutputIsFinal() { + testMultiplePolls(false); + } + + @Test + @Category({NeedsRunner.class, UsesSplittableParDo.class}) + public void testMultiplePollsWithTerminationDueToTerminationCondition() { + testMultiplePolls(true); + } + + private void testMultiplePolls(boolean terminationConditionElapsesBeforeOutputIsFinal) { + List all = Arrays.asList(0, 1, 2, 3, 4, 5, 6, 7, 8, 9); + + PCollection res = + p.apply(Create.of("a")) + .apply( + Watch.growthOf( + new TimedPollFn( + all, + standardSeconds(1) /* timeToOutputEverything */, + standardSeconds(3) /* timeToDeclareOutputFinal */, + standardSeconds(30) /* timeToFail */)) + .withTerminationPerInput( + Watch.Growth.afterTotalOf( + standardSeconds( + // At 2 seconds, all output has been yielded, but not yet + // declared final - so polling should terminate per termination + // condition. + // At 3 seconds, all output has been yielded (and declared final), + // so polling should terminate because of that without waiting for + // 100 seconds. + terminationConditionElapsesBeforeOutputIsFinal ? 2 : 100))) + .withPollInterval(Duration.millis(300)) + .withOutputCoder(VarIntCoder.of())) + .apply("Drop input", Values.create()); + + PAssert.that(res).containsInAnyOrder(all); + + p.run(); + } + + @Test + @Category({NeedsRunner.class, UsesSplittableParDo.class}) + public void testMultiplePollsStopAfterTimeSinceNewOutput() { + List all = Arrays.asList(0, 1, 2, 3, 4, 5, 6, 7, 8, 9); + + PCollection res = + p.apply(Create.of("a")) + .apply( + Watch.growthOf( + new TimedPollFn( + all, + standardSeconds(1) /* timeToOutputEverything */, + // Never declare output final + standardSeconds(1000) /* timeToDeclareOutputFinal */, + standardSeconds(30) /* timeToFail */)) + // Should terminate after 4 seconds - earlier than timeToFail + .withTerminationPerInput( + Watch.Growth.afterTimeSinceNewOutput(standardSeconds(3))) + .withPollInterval(Duration.millis(300)) + .withOutputCoder(VarIntCoder.of())) + .apply("Drop input", Values.create()); + + PAssert.that(res).containsInAnyOrder(all); + + p.run(); + } + + @Test + @Category({NeedsRunner.class, UsesSplittableParDo.class}) + public void testSinglePollWithManyResults() { + // More than the default 100 elements per checkpoint for direct runner. + final long numResults = 3000; + PCollection> res = + p.apply(Create.of("a")) + .apply( + Watch.growthOf( + new PollFn>() { + @Override + public PollResult> apply(String input, Instant time) { + String pollId = UUID.randomUUID().toString(); + List> output = Lists.newArrayList(); + for (int i = 0; i < numResults; ++i) { + output.add(KV.of(pollId, i)); + } + return PollResult.complete(time, output); + } + }) + .withTerminationPerInput(Watch.Growth.afterTotalOf(standardSeconds(1))) + .withPollInterval(Duration.millis(1)) + .withOutputCoder(KvCoder.of(StringUtf8Coder.of(), VarIntCoder.of()))) + .apply("Drop input", Values.>create()); + + PAssert.that("Poll called only once", res.apply(Keys.create())) + .satisfies( + new SerializableFunction, Void>() { + @Override + public Void apply(Iterable pollIds) { + assertEquals(1, Sets.newHashSet(pollIds).size()); + return null; + } + }); + PAssert.that("Yields all expected results", res.apply("Drop poll id", Values.create())) + .satisfies( + new SerializableFunction, Void>() { + @Override + public Void apply(Iterable input) { + assertEquals( + "Total number of results mismatches", + numResults, + Lists.newArrayList(input).size()); + assertEquals("Results are not unique", numResults, Sets.newHashSet(input).size()); + return null; + } + }); + + p.run(); + } + + @Test + @Category({NeedsRunner.class, UsesSplittableParDo.class}) + public void testMultiplePollsWithManyResults() { + final long numResults = 3000; + List all = Lists.newArrayList(); + for (int i = 0; i < numResults; ++i) { + all.add(i); + } + + PCollection> res = + p.apply(Create.of("a")) + .apply( + Watch.growthOf( + new TimedPollFn( + all, + standardSeconds(1) /* timeToOutputEverything */, + standardSeconds(3) /* timeToDeclareOutputFinal */, + standardSeconds(30) /* timeToFail */)) + .withPollInterval(Duration.millis(500)) + .withOutputCoder(VarIntCoder.of())) + .apply(ReifyTimestamps.inValues()) + .apply("Drop timestamped input", Values.>create()); + + PAssert.that(res) + .satisfies( + new SerializableFunction>, Void>() { + @Override + public Void apply(Iterable> outputs) { + Function, Integer> extractValueFn = + new Function, Integer>() { + @Nullable + @Override + public Integer apply(@Nullable TimestampedValue input) { + return input.getValue(); + } + }; + Function, Instant> extractTimestampFn = + new Function, Instant>() { + @Nullable + @Override + public Instant apply(@Nullable TimestampedValue input) { + return input.getTimestamp(); + } + }; + + Ordering> byValue = + Ordering.natural().onResultOf(extractValueFn); + Ordering> byTimestamp = + Ordering.natural().onResultOf(extractTimestampFn); + // New outputs appear in timestamp order because each output's assigned timestamp + // is Instant.now() at the time of poll. + assertTrue( + "Outputs must be in timestamp order", + byTimestamp.isOrdered(byValue.sortedCopy(outputs))); + assertEquals( + "Yields all expected values", + Sets.newHashSet(Iterables.transform(outputs, extractValueFn)).size(), + numResults); + assertThat( + "Poll called more than once", + Sets.newHashSet(Iterables.transform(outputs, extractTimestampFn)).size(), + greaterThan(1)); + return null; + } + }); + + p.run(); + } + + /** + * Gradually emits all items from the given list, pairing each one with a UUID that identifies the + * round of polling, so a client can check how many rounds of polling there were. + */ + private static class TimedPollFn implements PollFn { + private final Instant baseTime; + private final List outputs; + private final Duration timeToOutputEverything; + private final Duration timeToDeclareOutputFinal; + private final Duration timeToFail; + + public TimedPollFn( + List outputs, + Duration timeToOutputEverything, + Duration timeToDeclareOutputFinal, + Duration timeToFail) { + this.baseTime = Instant.now(); + this.outputs = outputs; + this.timeToOutputEverything = timeToOutputEverything; + this.timeToDeclareOutputFinal = timeToDeclareOutputFinal; + this.timeToFail = timeToFail; + } + + @Override + public PollResult apply(InputT input, Instant time) { + Instant now = Instant.now(); + Duration elapsed = new Duration(baseTime, Instant.now()); + if (elapsed.isLongerThan(timeToFail)) { + fail( + String.format( + "Poll called %s after base time, which is longer than the threshold of %s", + elapsed, timeToFail)); + } + + double fractionElapsed = 1.0 * elapsed.getMillis() / timeToOutputEverything.getMillis(); + int numToEmit = (int) Math.min(outputs.size(), fractionElapsed * outputs.size()); + List> toEmit = Lists.newArrayList(); + for (int i = 0; i < numToEmit; ++i) { + toEmit.add(TimestampedValue.of(outputs.get(i), now)); + } + return elapsed.isLongerThan(timeToDeclareOutputFinal) + ? PollResult.complete(toEmit) + : PollResult.incomplete(toEmit).withWatermark(now); + } + } + + @Test + public void testTerminationConditionsNever() { + Watch.Growth.Never c = never(); + Integer state = c.forNewInput(Instant.now(), null); + assertFalse(c.canStopPolling(Instant.now(), state)); + } + + @Test + public void testTerminationConditionsAfterTotalOf() { + Instant now = Instant.now(); + Watch.Growth.AfterTotalOf c = afterTotalOf(standardSeconds(5)); + KV state = c.forNewInput(now, null); + assertFalse(c.canStopPolling(now, state)); + assertFalse(c.canStopPolling(now.plus(standardSeconds(3)), state)); + assertTrue(c.canStopPolling(now.plus(standardSeconds(6)), state)); + } + + @Test + public void testTerminationConditionsAfterTimeSinceNewOutput() { + Instant now = Instant.now(); + Watch.Growth.AfterTimeSinceNewOutput c = afterTimeSinceNewOutput(standardSeconds(5)); + KV state = c.forNewInput(now, null); + assertFalse(c.canStopPolling(now, state)); + assertFalse(c.canStopPolling(now.plus(standardSeconds(3)), state)); + assertFalse(c.canStopPolling(now.plus(standardSeconds(6)), state)); + + state = c.onSeenNewOutput(now.plus(standardSeconds(3)), state); + assertFalse(c.canStopPolling(now.plus(standardSeconds(3)), state)); + assertFalse(c.canStopPolling(now.plus(standardSeconds(6)), state)); + assertTrue(c.canStopPolling(now.plus(standardSeconds(9)), state)); + + state = c.onSeenNewOutput(now.plus(standardSeconds(5)), state); + assertFalse(c.canStopPolling(now.plus(standardSeconds(3)), state)); + assertFalse(c.canStopPolling(now.plus(standardSeconds(6)), state)); + assertFalse(c.canStopPolling(now.plus(standardSeconds(9)), state)); + assertTrue(c.canStopPolling(now.plus(standardSeconds(11)), state)); + } + + @Test + public void testTerminationConditionsEitherOf() { + Instant now = Instant.now(); + Watch.Growth.AfterTotalOf a = afterTotalOf(standardSeconds(5)); + Watch.Growth.AfterTotalOf b = afterTotalOf(standardSeconds(10)); + + Watch.Growth.BinaryCombined< + Object, KV, KV> + c = eitherOf(a, b); + KV, KV> state = + c.forNewInput(now, null); + assertFalse(c.canStopPolling(now.plus(standardSeconds(3)), state)); + assertTrue(c.canStopPolling(now.plus(standardSeconds(7)), state)); + assertTrue(c.canStopPolling(now.plus(standardSeconds(12)), state)); + } + + @Test + public void testTerminationConditionsAllOf() { + Instant now = Instant.now(); + Watch.Growth.AfterTotalOf a = afterTotalOf(standardSeconds(5)); + Watch.Growth.AfterTotalOf b = afterTotalOf(standardSeconds(10)); + + Watch.Growth.BinaryCombined< + Object, KV, KV> + c = allOf(a, b); + KV, KV> state = + c.forNewInput(now, null); + assertFalse(c.canStopPolling(now.plus(standardSeconds(3)), state)); + assertFalse(c.canStopPolling(now.plus(standardSeconds(7)), state)); + assertTrue(c.canStopPolling(now.plus(standardSeconds(12)), state)); + } + + private static GrowthTracker newTracker(GrowthState state) { + return new GrowthTracker<>(StringUtf8Coder.of(), state, never()); + } + + private static GrowthTracker newTracker() { + return newTracker(new GrowthState(never().forNewInput(Instant.now(), null))); + } + + @Test + public void testGrowthTrackerCheckpointEmpty() { + // Checkpoint an empty tracker. + GrowthTracker tracker = newTracker(); + GrowthState residual = tracker.checkpoint(); + GrowthState primary = tracker.currentRestriction(); + Watch.Growth.Never condition = never(); + assertEquals( + primary.toString(condition), + new GrowthState<>( + Collections.emptyMap() /* completed */, + Collections.>emptyList() /* pending */, + true /* isOutputFinal */, + (Integer) null /* terminationState */, + BoundedWindow.TIMESTAMP_MAX_VALUE /* pollWatermark */) + .toString(condition)); + assertEquals( + residual.toString(condition), + new GrowthState<>( + Collections.emptyMap() /* completed */, + Collections.>emptyList() /* pending */, + false /* isOutputFinal */, + 0 /* terminationState */, + BoundedWindow.TIMESTAMP_MIN_VALUE /* pollWatermark */) + .toString(condition)); + } + + @Test + public void testGrowthTrackerCheckpointNonEmpty() { + Instant now = Instant.now(); + GrowthTracker tracker = newTracker(); + tracker.addNewAsPending( + PollResult.incomplete( + Arrays.asList( + TimestampedValue.of("d", now.plus(standardSeconds(4))), + TimestampedValue.of("c", now.plus(standardSeconds(3))), + TimestampedValue.of("a", now.plus(standardSeconds(1))), + TimestampedValue.of("b", now.plus(standardSeconds(2))))) + .withWatermark(now.plus(standardSeconds(7)))); + + assertEquals(now.plus(standardSeconds(1)), tracker.getWatermark()); + assertTrue(tracker.hasPending()); + assertEquals("a", tracker.tryClaimNextPending().getValue()); + assertTrue(tracker.hasPending()); + assertEquals("b", tracker.tryClaimNextPending().getValue()); + assertTrue(tracker.hasPending()); + assertEquals(now.plus(standardSeconds(3)), tracker.getWatermark()); + + GrowthTracker residualTracker = newTracker(tracker.checkpoint()); + GrowthTracker primaryTracker = newTracker(tracker.currentRestriction()); + + // Verify primary: should contain what the current tracker claimed, and nothing else. + assertEquals(now.plus(standardSeconds(1)), primaryTracker.getWatermark()); + assertTrue(primaryTracker.hasPending()); + assertEquals("a", primaryTracker.tryClaimNextPending().getValue()); + assertTrue(primaryTracker.hasPending()); + assertEquals("b", primaryTracker.tryClaimNextPending().getValue()); + assertFalse(primaryTracker.hasPending()); + assertFalse(primaryTracker.shouldPollMore()); + // No more pending elements in primary restriction, and no polling. + primaryTracker.checkDone(); + assertEquals(BoundedWindow.TIMESTAMP_MAX_VALUE, primaryTracker.getWatermark()); + + // Verify residual: should contain what the current tracker didn't claim. + assertEquals(now.plus(standardSeconds(3)), residualTracker.getWatermark()); + assertTrue(residualTracker.hasPending()); + assertEquals("c", residualTracker.tryClaimNextPending().getValue()); + assertTrue(residualTracker.hasPending()); + assertEquals("d", residualTracker.tryClaimNextPending().getValue()); + assertFalse(residualTracker.hasPending()); + assertTrue(residualTracker.shouldPollMore()); + // No more pending elements in residual restriction, but poll watermark still holds. + assertEquals(now.plus(standardSeconds(7)), residualTracker.getWatermark()); + + // Verify current tracker: it was checkpointed, so should contain nothing else. + assertNull(tracker.tryClaimNextPending()); + tracker.checkDone(); + assertFalse(tracker.hasPending()); + assertFalse(tracker.shouldPollMore()); + assertEquals(BoundedWindow.TIMESTAMP_MAX_VALUE, tracker.getWatermark()); + } + + @Test + public void testGrowthTrackerOutputFullyBeforeCheckpointIncomplete() { + Instant now = Instant.now(); + GrowthTracker tracker = newTracker(); + tracker.addNewAsPending( + PollResult.incomplete( + Arrays.asList( + TimestampedValue.of("d", now.plus(standardSeconds(4))), + TimestampedValue.of("c", now.plus(standardSeconds(3))), + TimestampedValue.of("a", now.plus(standardSeconds(1))), + TimestampedValue.of("b", now.plus(standardSeconds(2))))) + .withWatermark(now.plus(standardSeconds(7)))); + + assertEquals("a", tracker.tryClaimNextPending().getValue()); + assertEquals("b", tracker.tryClaimNextPending().getValue()); + assertEquals("c", tracker.tryClaimNextPending().getValue()); + assertEquals("d", tracker.tryClaimNextPending().getValue()); + assertFalse(tracker.hasPending()); + assertEquals(now.plus(standardSeconds(7)), tracker.getWatermark()); + + GrowthTracker residualTracker = newTracker(tracker.checkpoint()); + GrowthTracker primaryTracker = newTracker(tracker.currentRestriction()); + + // Verify primary: should contain what the current tracker claimed, and nothing else. + assertEquals(now.plus(standardSeconds(1)), primaryTracker.getWatermark()); + assertTrue(primaryTracker.hasPending()); + assertEquals("a", primaryTracker.tryClaimNextPending().getValue()); + assertTrue(primaryTracker.hasPending()); + assertEquals("b", primaryTracker.tryClaimNextPending().getValue()); + assertTrue(primaryTracker.hasPending()); + assertEquals("c", primaryTracker.tryClaimNextPending().getValue()); + assertTrue(primaryTracker.hasPending()); + assertEquals("d", primaryTracker.tryClaimNextPending().getValue()); + assertFalse(primaryTracker.hasPending()); + assertFalse(primaryTracker.shouldPollMore()); + // No more pending elements in primary restriction, and no polling. + primaryTracker.checkDone(); + assertEquals(BoundedWindow.TIMESTAMP_MAX_VALUE, primaryTracker.getWatermark()); + + // Verify residual: should contain what the current tracker didn't claim. + assertFalse(residualTracker.hasPending()); + assertTrue(residualTracker.shouldPollMore()); + // No more pending elements in residual restriction, but poll watermark still holds. + assertEquals(now.plus(standardSeconds(7)), residualTracker.getWatermark()); + + // Verify current tracker: it was checkpointed, so should contain nothing else. + tracker.checkDone(); + assertFalse(tracker.hasPending()); + assertFalse(tracker.shouldPollMore()); + assertEquals(BoundedWindow.TIMESTAMP_MAX_VALUE, tracker.getWatermark()); + } + + @Test + public void testGrowthTrackerPollAfterCheckpointIncompleteWithNewOutputs() { + Instant now = Instant.now(); + GrowthTracker tracker = newTracker(); + tracker.addNewAsPending( + PollResult.incomplete( + Arrays.asList( + TimestampedValue.of("d", now.plus(standardSeconds(4))), + TimestampedValue.of("c", now.plus(standardSeconds(3))), + TimestampedValue.of("a", now.plus(standardSeconds(1))), + TimestampedValue.of("b", now.plus(standardSeconds(2))))) + .withWatermark(now.plus(standardSeconds(7)))); + + assertEquals("a", tracker.tryClaimNextPending().getValue()); + assertEquals("b", tracker.tryClaimNextPending().getValue()); + assertEquals("c", tracker.tryClaimNextPending().getValue()); + assertEquals("d", tracker.tryClaimNextPending().getValue()); + + GrowthState checkpoint = tracker.checkpoint(); + // Simulate resuming from the checkpoint and adding more elements. + { + GrowthTracker residualTracker = newTracker(checkpoint); + residualTracker.addNewAsPending( + PollResult.incomplete( + Arrays.asList( + TimestampedValue.of("e", now.plus(standardSeconds(5))), + TimestampedValue.of("d", now.plus(standardSeconds(4))), + TimestampedValue.of("c", now.plus(standardSeconds(3))), + TimestampedValue.of("a", now.plus(standardSeconds(1))), + TimestampedValue.of("b", now.plus(standardSeconds(2))), + TimestampedValue.of("f", now.plus(standardSeconds(8))))) + .withWatermark(now.plus(standardSeconds(12)))); + + assertEquals(now.plus(standardSeconds(5)), residualTracker.getWatermark()); + assertEquals("e", residualTracker.tryClaimNextPending().getValue()); + assertEquals(now.plus(standardSeconds(8)), residualTracker.getWatermark()); + assertEquals("f", residualTracker.tryClaimNextPending().getValue()); + + assertFalse(residualTracker.hasPending()); + assertTrue(residualTracker.shouldPollMore()); + assertEquals(now.plus(standardSeconds(12)), residualTracker.getWatermark()); + } + // Try same without an explicitly specified watermark. + { + GrowthTracker residualTracker = newTracker(checkpoint); + residualTracker.addNewAsPending( + PollResult.incomplete( + Arrays.asList( + TimestampedValue.of("e", now.plus(standardSeconds(5))), + TimestampedValue.of("d", now.plus(standardSeconds(4))), + TimestampedValue.of("c", now.plus(standardSeconds(3))), + TimestampedValue.of("a", now.plus(standardSeconds(1))), + TimestampedValue.of("b", now.plus(standardSeconds(2))), + TimestampedValue.of("f", now.plus(standardSeconds(8)))))); + + assertEquals(now.plus(standardSeconds(5)), residualTracker.getWatermark()); + assertEquals("e", residualTracker.tryClaimNextPending().getValue()); + assertEquals(now.plus(standardSeconds(5)), residualTracker.getWatermark()); + assertEquals("f", residualTracker.tryClaimNextPending().getValue()); + + assertFalse(residualTracker.hasPending()); + assertTrue(residualTracker.shouldPollMore()); + assertEquals(now.plus(standardSeconds(5)), residualTracker.getWatermark()); + } + } + + @Test + public void testGrowthTrackerPollAfterCheckpointWithoutNewOutputs() { + Instant now = Instant.now(); + GrowthTracker tracker = newTracker(); + tracker.addNewAsPending( + PollResult.incomplete( + Arrays.asList( + TimestampedValue.of("d", now.plus(standardSeconds(4))), + TimestampedValue.of("c", now.plus(standardSeconds(3))), + TimestampedValue.of("a", now.plus(standardSeconds(1))), + TimestampedValue.of("b", now.plus(standardSeconds(2))))) + .withWatermark(now.plus(standardSeconds(7)))); + + assertEquals("a", tracker.tryClaimNextPending().getValue()); + assertEquals("b", tracker.tryClaimNextPending().getValue()); + assertEquals("c", tracker.tryClaimNextPending().getValue()); + assertEquals("d", tracker.tryClaimNextPending().getValue()); + + // Simulate resuming from the checkpoint but there are no new elements. + GrowthState checkpoint = tracker.checkpoint(); + { + GrowthTracker residualTracker = newTracker(checkpoint); + residualTracker.addNewAsPending( + PollResult.incomplete( + Arrays.asList( + TimestampedValue.of("c", now.plus(standardSeconds(3))), + TimestampedValue.of("d", now.plus(standardSeconds(4))), + TimestampedValue.of("a", now.plus(standardSeconds(1))), + TimestampedValue.of("b", now.plus(standardSeconds(2))))) + .withWatermark(now.plus(standardSeconds(12)))); + + assertFalse(residualTracker.hasPending()); + assertTrue(residualTracker.shouldPollMore()); + assertEquals(now.plus(standardSeconds(12)), residualTracker.getWatermark()); + } + // Try the same without an explicitly specified watermark + { + GrowthTracker residualTracker = newTracker(checkpoint); + residualTracker.addNewAsPending( + PollResult.incomplete( + Arrays.asList( + TimestampedValue.of("c", now.plus(standardSeconds(3))), + TimestampedValue.of("d", now.plus(standardSeconds(4))), + TimestampedValue.of("a", now.plus(standardSeconds(1))), + TimestampedValue.of("b", now.plus(standardSeconds(2)))))); + // No new elements and no explicit watermark supplied - should reuse old watermark. + assertEquals(now.plus(standardSeconds(7)), residualTracker.getWatermark()); + } + } + + @Test + public void testGrowthTrackerPollAfterCheckpointWithoutNewOutputsNoWatermark() { + Instant now = Instant.now(); + GrowthTracker tracker = newTracker(); + tracker.addNewAsPending( + PollResult.incomplete( + Arrays.asList( + TimestampedValue.of("d", now.plus(standardSeconds(4))), + TimestampedValue.of("c", now.plus(standardSeconds(3))), + TimestampedValue.of("a", now.plus(standardSeconds(1))), + TimestampedValue.of("b", now.plus(standardSeconds(2)))))); + assertEquals("a", tracker.tryClaimNextPending().getValue()); + assertEquals("b", tracker.tryClaimNextPending().getValue()); + assertEquals("c", tracker.tryClaimNextPending().getValue()); + assertEquals("d", tracker.tryClaimNextPending().getValue()); + assertEquals(now.plus(standardSeconds(1)), tracker.getWatermark()); + + // Simulate resuming from the checkpoint but there are no new elements. + GrowthState checkpoint = tracker.checkpoint(); + GrowthTracker residualTracker = newTracker(checkpoint); + residualTracker.addNewAsPending( + PollResult.incomplete( + Arrays.asList( + TimestampedValue.of("c", now.plus(standardSeconds(3))), + TimestampedValue.of("d", now.plus(standardSeconds(4))), + TimestampedValue.of("a", now.plus(standardSeconds(1))), + TimestampedValue.of("b", now.plus(standardSeconds(2)))))); + // No new elements and no explicit watermark supplied - should keep old watermark. + assertEquals(now.plus(standardSeconds(1)), residualTracker.getWatermark()); + } + + @Test + public void testGrowthTrackerRepeatedEmptyPollWatermark() { + // Empty poll result with no watermark + { + GrowthTracker tracker = newTracker(); + tracker.addNewAsPending( + PollResult.incomplete(Collections.>emptyList())); + assertEquals(BoundedWindow.TIMESTAMP_MIN_VALUE, tracker.getWatermark()); + + // Simulate resuming from the checkpoint but there are still no new elements. + GrowthTracker residualTracker = newTracker(tracker.checkpoint()); + tracker.addNewAsPending( + PollResult.incomplete(Collections.>emptyList())); + // No new elements and no explicit watermark supplied - still no watermark. + assertEquals(BoundedWindow.TIMESTAMP_MIN_VALUE, residualTracker.getWatermark()); + } + // Empty poll result with watermark + { + Instant now = Instant.now(); + GrowthTracker tracker = newTracker(); + tracker.addNewAsPending( + PollResult.incomplete(Collections.>emptyList()) + .withWatermark(now)); + assertEquals(now, tracker.getWatermark()); + + // Simulate resuming from the checkpoint but there are still no new elements. + GrowthTracker residualTracker = newTracker(tracker.checkpoint()); + tracker.addNewAsPending( + PollResult.incomplete(Collections.>emptyList())); + // No new elements and no explicit watermark supplied - should keep old watermark. + assertEquals(now, residualTracker.getWatermark()); + } + } + + @Test + public void testGrowthTrackerOutputFullyBeforeCheckpointComplete() { + Instant now = Instant.now(); + GrowthTracker tracker = newTracker(); + tracker.addNewAsPending( + PollResult.complete( + Arrays.asList( + TimestampedValue.of("d", now.plus(standardSeconds(4))), + TimestampedValue.of("c", now.plus(standardSeconds(3))), + TimestampedValue.of("a", now.plus(standardSeconds(1))), + TimestampedValue.of("b", now.plus(standardSeconds(2)))))); + + assertEquals("a", tracker.tryClaimNextPending().getValue()); + assertEquals("b", tracker.tryClaimNextPending().getValue()); + assertEquals("c", tracker.tryClaimNextPending().getValue()); + assertEquals("d", tracker.tryClaimNextPending().getValue()); + assertFalse(tracker.hasPending()); + assertEquals(BoundedWindow.TIMESTAMP_MAX_VALUE, tracker.getWatermark()); + + GrowthTracker residualTracker = newTracker(tracker.checkpoint()); + + // Verify residual: should be empty, since output was final. + residualTracker.checkDone(); + assertFalse(residualTracker.hasPending()); + assertFalse(residualTracker.shouldPollMore()); + // No more pending elements in residual restriction, but poll watermark still holds. + assertEquals(BoundedWindow.TIMESTAMP_MAX_VALUE, residualTracker.getWatermark()); + + // Verify current tracker: it was checkpointed, so should contain nothing else. + tracker.checkDone(); + assertFalse(tracker.hasPending()); + assertFalse(tracker.shouldPollMore()); + assertEquals(BoundedWindow.TIMESTAMP_MAX_VALUE, tracker.getWatermark()); + } +} From 8b60fcb6ddaa843e9d30f5abe7b29abc9f73f3ad Mon Sep 17 00:00:00 2001 From: Jason Kuster Date: Thu, 15 Jun 2017 16:14:16 -0700 Subject: [PATCH 218/346] Programmatically Create Beam Jenkins View. Signed-off-by: Jason Kuster --- .../jenkins/job_beam_ListView_Create.groovy | 42 +++++++++++++++++++ 1 file changed, 42 insertions(+) create mode 100644 .test-infra/jenkins/job_beam_ListView_Create.groovy diff --git a/.test-infra/jenkins/job_beam_ListView_Create.groovy b/.test-infra/jenkins/job_beam_ListView_Create.groovy new file mode 100644 index 0000000000000..9c304da46c2b0 --- /dev/null +++ b/.test-infra/jenkins/job_beam_ListView_Create.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. + */ + +// This creates a standard List View for Beam. It filters out executors not associated with Beam +// and pulls in all jobs which start with 'beam_'. +listView('Beam') { + description('Jenkins builds for the Beam project.') + filterBuildQueue() + filterExecutors() + jobs { + regex(/beam_.+/) + } + jobFilters { + all() + } + columns { + status() + weather() + name() + lastSuccess() + lastFailure() + lastDuration() + lastBuildConsole() + buildButton() + } +} + From 42638e248f588d538367817a2cda3328c43d48a6 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Isma=C3=ABl=20Mej=C3=ADa?= Date: Thu, 6 Jul 2017 17:53:36 +0200 Subject: [PATCH 219/346] [BEAM-2530] Upgrade maven plugins to the latest versions This is part of the ongoing work to support java 9 --- pom.xml | 8 +++++--- sdks/java/io/hadoop/jdk1.8-tests/pom.xml | 1 - .../src/test/resources/projects/basic/reference/pom.xml | 6 +++--- 3 files changed, 8 insertions(+), 7 deletions(-) diff --git a/pom.xml b/pom.xml index e0ec1362a6782..8535574ae3bba 100644 --- a/pom.xml +++ b/pom.xml @@ -157,10 +157,12 @@ 2.0 2.20 2.20 - 3.6.1 + 3.6.2 3.0.1 + 3.0.0-M1 1.6.0 3.0.2 + 3.0.0-M1 3.0.2 3.0.0 @@ -1389,7 +1391,7 @@ here, we leave things simple here. --> org.apache.maven.plugins maven-javadoc-plugin - 2.10.4 + ${maven-javadoc-plugin.version} ${beam.javadoc_opts} Apache Beam SDK for Java, version ${project.version} API @@ -1780,7 +1782,7 @@ org.apache.maven.plugins maven-enforcer-plugin - 1.4.1 + ${maven-enforcer-plugin.version} enforce diff --git a/sdks/java/io/hadoop/jdk1.8-tests/pom.xml b/sdks/java/io/hadoop/jdk1.8-tests/pom.xml index 12944f49c3581..8df255201ffb2 100644 --- a/sdks/java/io/hadoop/jdk1.8-tests/pom.xml +++ b/sdks/java/io/hadoop/jdk1.8-tests/pom.xml @@ -39,7 +39,6 @@ org.apache.maven.plugins maven-enforcer-plugin - 1.4.1 enforce 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 6056fb0083ea7..91da6eb2ac7d3 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 @@ -27,9 +27,9 @@ @project.version@ - 3.6.1 - 1.6.0 - 1.7.14 + @maven-compiler-plugin.version@ + @maven-exec-plugin.version@ + @slf4j.version@ From 14563c642bb9266e2db74813cf47902f56558271 Mon Sep 17 00:00:00 2001 From: Nathan Howell Date: Fri, 28 Jul 2017 14:23:27 -0700 Subject: [PATCH 220/346] [BEAM-2690] HCatalogIO needs provided scope dependencies on Hadoop, Hive --- sdks/java/io/hcatalog/pom.xml | 3 +++ 1 file changed, 3 insertions(+) diff --git a/sdks/java/io/hcatalog/pom.xml b/sdks/java/io/hcatalog/pom.xml index 52eae345dbbd4..34e60da3130b4 100644 --- a/sdks/java/io/hcatalog/pom.xml +++ b/sdks/java/io/hcatalog/pom.xml @@ -61,6 +61,7 @@ org.apache.hadoop hadoop-common + provided @@ -89,6 +90,7 @@ org.apache.hive hive-exec ${hive.version} + provided @@ -101,6 +103,7 @@ org.apache.hive.hcatalog hive-hcatalog-core ${hive.version} + provided org.apache.hive From 64cf18fcdb4237189a5212b6476bdadf73a2ac7f Mon Sep 17 00:00:00 2001 From: Thomas Groh Date: Wed, 26 Jul 2017 15:34:23 -0700 Subject: [PATCH 221/346] Remove References to CloudObject from the Java Harness Migrates to using the shared Runner API definitions. --- .../beam/fn/harness/BeamFnDataReadRunner.java | 27 +++++++---------- .../fn/harness/BeamFnDataWriteRunner.java | 22 +++++--------- .../fn/harness/BeamFnDataReadRunnerTest.java | 28 +++++++++--------- .../fn/harness/BeamFnDataWriteRunnerTest.java | 24 ++++++--------- .../beam/fn/harness/FnApiDoFnRunnerTest.java | 29 ------------------- 5 files changed, 41 insertions(+), 89 deletions(-) diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/BeamFnDataReadRunner.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/BeamFnDataReadRunner.java index e2c17b0b8ef9e..1e611db2b1696 100644 --- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/BeamFnDataReadRunner.java +++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/BeamFnDataReadRunner.java @@ -24,7 +24,6 @@ import com.google.auto.service.AutoService; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Multimap; -import com.google.protobuf.BytesValue; import java.io.IOException; import java.util.Collection; import java.util.Map; @@ -35,8 +34,8 @@ import org.apache.beam.fn.harness.fn.ThrowingConsumer; import org.apache.beam.fn.harness.fn.ThrowingRunnable; import org.apache.beam.fn.v1.BeamFnApi; -import org.apache.beam.runners.dataflow.util.CloudObject; -import org.apache.beam.runners.dataflow.util.CloudObjects; +import org.apache.beam.runners.core.construction.CoderTranslation; +import org.apache.beam.runners.core.construction.RehydratedComponents; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.common.runner.v1.RunnerApi; import org.apache.beam.sdk.options.PipelineOptions; @@ -91,8 +90,9 @@ public BeamFnDataReadRunner createRunnerForPTransform( .setPrimitiveTransformReference(pTransformId) .setName(getOnlyElement(pTransform.getOutputsMap().keySet())) .build(); - RunnerApi.Coder coderSpec = coders.get(pCollections.get( - getOnlyElement(pTransform.getOutputsMap().values())).getCoderId()); + RunnerApi.Coder coderSpec = + coders.get( + pCollections.get(getOnlyElement(pTransform.getOutputsMap().values())).getCoderId()); Collection>> consumers = (Collection) pCollectionIdsToConsumers.get( getOnlyElement(pTransform.getOutputsMap().values())); @@ -102,6 +102,7 @@ public BeamFnDataReadRunner createRunnerForPTransform( processBundleInstructionId, target, coderSpec, + coders, beamFnDataClient, consumers); addStartFunction.accept(runner::registerInputLocation); @@ -124,6 +125,7 @@ public BeamFnDataReadRunner createRunnerForPTransform( Supplier processBundleInstructionIdSupplier, BeamFnApi.Target inputTarget, RunnerApi.Coder coderSpec, + Map coders, BeamFnDataClient beamFnDataClientFactory, Collection>> consumers) throws IOException { @@ -137,17 +139,10 @@ public BeamFnDataReadRunner createRunnerForPTransform( @SuppressWarnings("unchecked") Coder> coder = (Coder>) - CloudObjects.coderFromCloudObject( - CloudObject.fromSpec( - OBJECT_MAPPER.readValue( - coderSpec - .getSpec() - .getSpec() - .getParameter() - .unpack(BytesValue.class) - .getValue() - .newInput(), - Map.class))); + CoderTranslation.fromProto( + coderSpec, + RehydratedComponents.forComponents( + RunnerApi.Components.newBuilder().putAllCoders(coders).build())); this.coder = coder; } diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/BeamFnDataWriteRunner.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/BeamFnDataWriteRunner.java index eec4dfde83c86..bbed75301bd80 100644 --- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/BeamFnDataWriteRunner.java +++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/BeamFnDataWriteRunner.java @@ -24,7 +24,6 @@ import com.google.auto.service.AutoService; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Multimap; -import com.google.protobuf.BytesValue; import java.io.IOException; import java.util.Map; import java.util.function.Consumer; @@ -34,8 +33,8 @@ import org.apache.beam.fn.harness.fn.ThrowingConsumer; import org.apache.beam.fn.harness.fn.ThrowingRunnable; import org.apache.beam.fn.v1.BeamFnApi; -import org.apache.beam.runners.dataflow.util.CloudObject; -import org.apache.beam.runners.dataflow.util.CloudObjects; +import org.apache.beam.runners.core.construction.CoderTranslation; +import org.apache.beam.runners.core.construction.RehydratedComponents; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.common.runner.v1.RunnerApi; import org.apache.beam.sdk.options.PipelineOptions; @@ -93,6 +92,7 @@ public BeamFnDataWriteRunner createRunnerForPTransform( processBundleInstructionId, target, coderSpec, + coders, beamFnDataClient); addStartFunction.accept(runner::registerForOutput); pCollectionIdsToConsumers.put( @@ -117,6 +117,7 @@ public BeamFnDataWriteRunner createRunnerForPTransform( Supplier processBundleInstructionIdSupplier, BeamFnApi.Target outputTarget, RunnerApi.Coder coderSpec, + Map coders, BeamFnDataClient beamFnDataClientFactory) throws IOException { this.apiServiceDescriptor = functionSpec.getParameter().unpack(BeamFnApi.RemoteGrpcPort.class) @@ -128,17 +129,10 @@ public BeamFnDataWriteRunner createRunnerForPTransform( @SuppressWarnings("unchecked") Coder> coder = (Coder>) - CloudObjects.coderFromCloudObject( - CloudObject.fromSpec( - OBJECT_MAPPER.readValue( - coderSpec - .getSpec() - .getSpec() - .getParameter() - .unpack(BytesValue.class) - .getValue() - .newInput(), - Map.class))); + CoderTranslation.fromProto( + coderSpec, + RehydratedComponents.forComponents( + RunnerApi.Components.newBuilder().putAllCoders(coders).build())); this.coder = coder; } diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/BeamFnDataReadRunnerTest.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/BeamFnDataReadRunnerTest.java index a7c66663bbd8a..d712f5fa4738a 100644 --- a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/BeamFnDataReadRunnerTest.java +++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/BeamFnDataReadRunnerTest.java @@ -30,7 +30,6 @@ import static org.mockito.Mockito.verifyZeroInteractions; import static org.mockito.Mockito.when; -import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Suppliers; import com.google.common.collect.HashMultimap; import com.google.common.collect.ImmutableList; @@ -39,8 +38,6 @@ import com.google.common.collect.Multimap; import com.google.common.util.concurrent.Uninterruptibles; import com.google.protobuf.Any; -import com.google.protobuf.ByteString; -import com.google.protobuf.BytesValue; import java.io.IOException; import java.util.ArrayList; import java.util.List; @@ -56,10 +53,11 @@ import org.apache.beam.fn.harness.test.TestExecutors; import org.apache.beam.fn.harness.test.TestExecutors.TestExecutorService; import org.apache.beam.fn.v1.BeamFnApi; -import org.apache.beam.runners.dataflow.util.CloudObjects; +import org.apache.beam.runners.core.construction.CoderTranslation; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.StringUtf8Coder; import org.apache.beam.sdk.common.runner.v1.RunnerApi; +import org.apache.beam.sdk.common.runner.v1.RunnerApi.MessageWithComponents; import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.transforms.windowing.GlobalWindow; import org.apache.beam.sdk.util.WindowedValue; @@ -79,7 +77,6 @@ @RunWith(JUnit4.class) public class BeamFnDataReadRunnerTest { - private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); private static final BeamFnApi.RemoteGrpcPort PORT_SPEC = BeamFnApi.RemoteGrpcPort.newBuilder() .setApiServiceDescriptor(BeamFnApi.ApiServiceDescriptor.getDefaultInstance()).build(); private static final RunnerApi.FunctionSpec FUNCTION_SPEC = RunnerApi.FunctionSpec.newBuilder() @@ -88,19 +85,19 @@ public class BeamFnDataReadRunnerTest { WindowedValue.getFullCoder(StringUtf8Coder.of(), GlobalWindow.Coder.INSTANCE); private static final String CODER_SPEC_ID = "string-coder-id"; private static final RunnerApi.Coder CODER_SPEC; + private static final RunnerApi.Components COMPONENTS; private static final String URN = "urn:org.apache.beam:source:runner:0.1"; static { try { - CODER_SPEC = RunnerApi.Coder.newBuilder().setSpec( - RunnerApi.SdkFunctionSpec.newBuilder().setSpec( - RunnerApi.FunctionSpec.newBuilder().setParameter( - Any.pack(BytesValue.newBuilder().setValue(ByteString.copyFrom( - OBJECT_MAPPER.writeValueAsBytes(CloudObjects.asCloudObject(CODER)))) - .build())) - .build()) - .build()) - .build(); + MessageWithComponents coderAndComponents = CoderTranslation.toProto(CODER); + CODER_SPEC = coderAndComponents.getCoder(); + COMPONENTS = + coderAndComponents + .getComponents() + .toBuilder() + .putCoders(CODER_SPEC_ID, CODER_SPEC) + .build(); } catch (IOException e) { throw new ExceptionInInitializerError(e); } @@ -150,7 +147,7 @@ public void testCreatingAndProcessingBeamFnDataReadRunner() throws Exception { Suppliers.ofInstance(bundleId)::get, ImmutableMap.of("outputPC", RunnerApi.PCollection.newBuilder().setCoderId(CODER_SPEC_ID).build()), - ImmutableMap.of(CODER_SPEC_ID, CODER_SPEC), + COMPONENTS.getCodersMap(), consumers, startFunctions::add, finishFunctions::add); @@ -200,6 +197,7 @@ public void testReuseForMultipleBundles() throws Exception { bundleId::get, INPUT_TARGET, CODER_SPEC, + COMPONENTS.getCodersMap(), mockBeamFnDataClient, ImmutableList.of(valuesA::add, valuesB::add)); diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/BeamFnDataWriteRunnerTest.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/BeamFnDataWriteRunnerTest.java index 28838b127be28..0caf19e318ef9 100644 --- a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/BeamFnDataWriteRunnerTest.java +++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/BeamFnDataWriteRunnerTest.java @@ -32,15 +32,12 @@ import static org.mockito.Mockito.verifyZeroInteractions; import static org.mockito.Mockito.when; -import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Suppliers; import com.google.common.collect.HashMultimap; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Iterables; import com.google.common.collect.Multimap; import com.google.protobuf.Any; -import com.google.protobuf.ByteString; -import com.google.protobuf.BytesValue; import java.io.IOException; import java.util.ArrayList; import java.util.List; @@ -53,10 +50,11 @@ import org.apache.beam.fn.harness.fn.ThrowingConsumer; import org.apache.beam.fn.harness.fn.ThrowingRunnable; import org.apache.beam.fn.v1.BeamFnApi; -import org.apache.beam.runners.dataflow.util.CloudObjects; +import org.apache.beam.runners.core.construction.CoderTranslation; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.StringUtf8Coder; import org.apache.beam.sdk.common.runner.v1.RunnerApi; +import org.apache.beam.sdk.common.runner.v1.RunnerApi.MessageWithComponents; import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.transforms.windowing.GlobalWindow; import org.apache.beam.sdk.util.WindowedValue; @@ -74,7 +72,6 @@ @RunWith(JUnit4.class) public class BeamFnDataWriteRunnerTest { - private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); private static final BeamFnApi.RemoteGrpcPort PORT_SPEC = BeamFnApi.RemoteGrpcPort.newBuilder() .setApiServiceDescriptor(BeamFnApi.ApiServiceDescriptor.getDefaultInstance()).build(); private static final RunnerApi.FunctionSpec FUNCTION_SPEC = RunnerApi.FunctionSpec.newBuilder() @@ -83,19 +80,15 @@ public class BeamFnDataWriteRunnerTest { private static final Coder> CODER = WindowedValue.getFullCoder(StringUtf8Coder.of(), GlobalWindow.Coder.INSTANCE); private static final RunnerApi.Coder CODER_SPEC; + private static final RunnerApi.Components COMPONENTS; private static final String URN = "urn:org.apache.beam:sink:runner:0.1"; static { try { - CODER_SPEC = RunnerApi.Coder.newBuilder().setSpec( - RunnerApi.SdkFunctionSpec.newBuilder().setSpec( - RunnerApi.FunctionSpec.newBuilder().setParameter( - Any.pack(BytesValue.newBuilder().setValue(ByteString.copyFrom( - OBJECT_MAPPER.writeValueAsBytes(CloudObjects.asCloudObject(CODER)))) - .build())) - .build()) - .build()) - .build(); + MessageWithComponents coderAndComponents = CoderTranslation.toProto(CODER); + CODER_SPEC = coderAndComponents.getCoder(); + COMPONENTS = + coderAndComponents.getComponents().toBuilder().putCoders(CODER_ID, CODER_SPEC).build(); } catch (IOException e) { throw new ExceptionInInitializerError(e); } @@ -140,7 +133,7 @@ public void testCreatingAndProcessingBeamFnDataWriteRunner() throws Exception { Suppliers.ofInstance(bundleId)::get, ImmutableMap.of("inputPC", RunnerApi.PCollection.newBuilder().setCoderId(CODER_ID).build()), - ImmutableMap.of(CODER_ID, CODER_SPEC), + COMPONENTS.getCodersMap(), consumers, startFunctions::add, finishFunctions::add); @@ -201,6 +194,7 @@ public void testReuseForMultipleBundles() throws Exception { bundleId::get, OUTPUT_TARGET, CODER_SPEC, + COMPONENTS.getCodersMap(), mockBeamFnDataClient); // Process for bundle id 0 diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/FnApiDoFnRunnerTest.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/FnApiDoFnRunnerTest.java index 98362a22a9345..e269bcc595671 100644 --- a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/FnApiDoFnRunnerTest.java +++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/FnApiDoFnRunnerTest.java @@ -25,7 +25,6 @@ import static org.junit.Assert.assertThat; import static org.junit.Assert.fail; -import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Suppliers; import com.google.common.collect.HashMultimap; import com.google.common.collect.ImmutableList; @@ -35,19 +34,14 @@ import com.google.protobuf.Any; import com.google.protobuf.ByteString; import com.google.protobuf.BytesValue; -import com.google.protobuf.Message; -import java.io.IOException; import java.util.ArrayList; import java.util.List; -import java.util.Map; import java.util.ServiceLoader; import org.apache.beam.fn.harness.PTransformRunnerFactory.Registrar; import org.apache.beam.fn.harness.fn.ThrowingConsumer; import org.apache.beam.fn.harness.fn.ThrowingRunnable; import org.apache.beam.runners.core.construction.ParDoTranslation; -import org.apache.beam.runners.dataflow.util.CloudObjects; import org.apache.beam.runners.dataflow.util.DoFnInfo; -import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.StringUtf8Coder; import org.apache.beam.sdk.common.runner.v1.RunnerApi; import org.apache.beam.sdk.options.PipelineOptionsFactory; @@ -66,28 +60,6 @@ /** Tests for {@link FnApiDoFnRunner}. */ @RunWith(JUnit4.class) public class FnApiDoFnRunnerTest { - - private static final ObjectMapper OBJECT_MAPPER = new ObjectMapper(); - private static final Coder> STRING_CODER = - WindowedValue.getFullCoder(StringUtf8Coder.of(), GlobalWindow.Coder.INSTANCE); - private static final String STRING_CODER_SPEC_ID = "999L"; - private static final RunnerApi.Coder STRING_CODER_SPEC; - - static { - try { - STRING_CODER_SPEC = RunnerApi.Coder.newBuilder() - .setSpec(RunnerApi.SdkFunctionSpec.newBuilder() - .setSpec(RunnerApi.FunctionSpec.newBuilder() - .setParameter(Any.pack(BytesValue.newBuilder().setValue(ByteString.copyFrom( - OBJECT_MAPPER.writeValueAsBytes(CloudObjects.asCloudObject(STRING_CODER)))) - .build()))) - .build()) - .build(); - } catch (IOException e) { - throw new ExceptionInInitializerError(e); - } - } - private static class TestDoFn extends DoFn { private static final TupleTag mainOutput = new TupleTag<>("mainOutput"); private static final TupleTag additionalOutput = new TupleTag<>("output"); @@ -117,7 +89,6 @@ public void finishBundle(FinishBundleContext context) { */ @Test public void testCreatingAndProcessingDoFn() throws Exception { - Map fnApiRegistry = ImmutableMap.of(STRING_CODER_SPEC_ID, STRING_CODER_SPEC); String pTransformId = "pTransformId"; String mainOutputId = "101"; String additionalOutputId = "102"; From 7d64b769a7099e767a81c693294b5a66791a441b Mon Sep 17 00:00:00 2001 From: Robert Bradshaw Date: Wed, 26 Jul 2017 16:30:56 -0700 Subject: [PATCH 222/346] Translate ParDo operations through the Runner API. --- sdks/python/apache_beam/pipeline.py | 11 +++++++ sdks/python/apache_beam/transforms/core.py | 38 +++++++++++++++++++++- sdks/python/apache_beam/utils/urns.py | 3 ++ 3 files changed, 51 insertions(+), 1 deletion(-) diff --git a/sdks/python/apache_beam/pipeline.py b/sdks/python/apache_beam/pipeline.py index fe36d85a7a3b0..8553f7ca7eab9 100644 --- a/sdks/python/apache_beam/pipeline.py +++ b/sdks/python/apache_beam/pipeline.py @@ -66,6 +66,7 @@ from apache_beam.options.pipeline_options import TypeOptions from apache_beam.options.pipeline_options_validator import PipelineOptionsValidator from apache_beam.utils.annotations import deprecated +from apache_beam.utils import urns __all__ = ['Pipeline'] @@ -474,6 +475,12 @@ def __reduce__(self): return str, ('Pickled pipeline stub.',) def _verify_runner_api_compatible(self): + if self._options.view_as(TypeOptions).runtime_type_check: + # This option is incompatible with the runner API as it requires + # the runner to inspect non-serialized hints on the transform + # itself. + return False + class Visitor(PipelineVisitor): # pylint: disable=used-before-assignment ok = True # Really a nonlocal. @@ -745,6 +752,10 @@ def from_runner_api(proto, context): result.outputs = { None if tag == 'None' else tag: context.pcollections.get_by_id(id) for tag, id in proto.outputs.items()} + # This annotation is expected by some runners. + if proto.spec.urn == urns.PARDO_TRANSFORM: + result.transform.output_tags = set(proto.outputs.keys()).difference( + {'None'}) if not result.parts: for tag, pc in result.outputs.items(): if pc not in result.inputs: diff --git a/sdks/python/apache_beam/transforms/core.py b/sdks/python/apache_beam/transforms/core.py index 25fe39ff37450..9168a898a181c 100644 --- a/sdks/python/apache_beam/transforms/core.py +++ b/sdks/python/apache_beam/transforms/core.py @@ -23,10 +23,13 @@ import inspect import types +from google.protobuf import wrappers_pb2 + from apache_beam import pvalue from apache_beam import typehints from apache_beam import coders from apache_beam.coders import typecoders +from apache_beam.internal import pickler from apache_beam.internal import util from apache_beam.portability.api import beam_runner_api_pb2 from apache_beam.transforms import ptransform @@ -50,6 +53,7 @@ from apache_beam.typehints.decorators import WithTypeHints from apache_beam.typehints.trivial_inference import element_type from apache_beam.typehints.typehints import is_consistent_with +from apache_beam.utils import proto_utils from apache_beam.utils import urns from apache_beam.options.pipeline_options import TypeOptions @@ -136,7 +140,7 @@ def set_element(self, windowed_value): self.windows = windowed_value.windows -class DoFn(WithTypeHints, HasDisplayData): +class DoFn(WithTypeHints, HasDisplayData, urns.RunnerApiFn): """A function object used by a transform with custom processing. The ParDo transform is such a transform. The ParDo.apply @@ -236,6 +240,8 @@ def is_process_bounded(self): return False # Method is a classmethod return True + urns.RunnerApiFn.register_pickle_urn(urns.PICKLED_DO_FN) + def _fn_takes_side_inputs(fn): try: @@ -686,6 +692,36 @@ def with_outputs(self, *tags, **main_kw): raise ValueError('Unexpected keyword arguments: %s' % main_kw.keys()) return _MultiParDo(self, tags, main_tag) + def _pardo_fn_data(self): + si_tags_and_types = [] + windowing = None + return self.fn, self.args, self.kwargs, si_tags_and_types, windowing + + def to_runner_api_parameter(self, context): + return ( + urns.PARDO_TRANSFORM, + beam_runner_api_pb2.ParDoPayload( + do_fn=beam_runner_api_pb2.SdkFunctionSpec( + spec=beam_runner_api_pb2.FunctionSpec( + urn=urns.PICKLED_DO_FN_INFO, + parameter=proto_utils.pack_Any( + wrappers_pb2.BytesValue( + value=pickler.dumps( + self._pardo_fn_data()))))))) + + @PTransform.register_urn( + urns.PARDO_TRANSFORM, beam_runner_api_pb2.ParDoPayload) + def from_runner_api_parameter(pardo_payload, context): + assert pardo_payload.do_fn.spec.urn == urns.PICKLED_DO_FN_INFO + fn, args, kwargs, si_tags_and_types, windowing = pickler.loads( + proto_utils.unpack_Any( + pardo_payload.do_fn.spec.parameter, wrappers_pb2.BytesValue).value) + if si_tags_and_types: + raise NotImplementedError('deferred side inputs') + elif windowing: + raise NotImplementedError('explicit windowing') + return ParDo(fn, *args, **kwargs) + class _MultiParDo(PTransform): diff --git a/sdks/python/apache_beam/utils/urns.py b/sdks/python/apache_beam/utils/urns.py index 7110802f3d476..dcdf0f3eee01e 100644 --- a/sdks/python/apache_beam/utils/urns.py +++ b/sdks/python/apache_beam/utils/urns.py @@ -32,10 +32,13 @@ SLIDING_WINDOWS_FN = "beam:windowfn:sliding_windows:v0.1" SESSION_WINDOWS_FN = "beam:windowfn:session_windows:v0.1" +PICKLED_DO_FN = "beam:dofn:pickled_python:v0.1" +PICKLED_DO_FN_INFO = "beam:dofn:pickled_python_info:v0.1" PICKLED_COMBINE_FN = "beam:combinefn:pickled_python:v0.1" PICKLED_CODER = "beam:coder:pickled_python:v0.1" PICKLED_TRANSFORM = "beam:ptransform:pickled_python:v0.1" +PARDO_TRANSFORM = "beam:ptransform:pardo:v0.1" COMBINE_PER_KEY_TRANSFORM = "beam:ptransform:combine_per_key:v0.1" COMBINE_GROUPED_VALUES_TRANSFORM = "beam:ptransform:combine_grouped_values:v0.1" FLATTEN_TRANSFORM = "beam:ptransform:flatten:v0.1" From 3c0c337401ca164093c06df5b9021106ba7e5eb9 Mon Sep 17 00:00:00 2001 From: Robert Bradshaw Date: Wed, 26 Jul 2017 18:01:49 -0700 Subject: [PATCH 223/346] Streaming fixes. --- .../runners/direct/direct_runner.py | 27 +++++++++++++++++-- sdks/python/apache_beam/transforms/core.py | 13 +++++++++ sdks/python/apache_beam/utils/urns.py | 1 + 3 files changed, 39 insertions(+), 2 deletions(-) diff --git a/sdks/python/apache_beam/runners/direct/direct_runner.py b/sdks/python/apache_beam/runners/direct/direct_runner.py index 1a94b3d2b458d..7a88d0e5cf01b 100644 --- a/sdks/python/apache_beam/runners/direct/direct_runner.py +++ b/sdks/python/apache_beam/runners/direct/direct_runner.py @@ -26,6 +26,8 @@ import collections import logging +from google.protobuf import wrappers_pb2 + import apache_beam as beam from apache_beam import typehints from apache_beam.metrics.execution import MetricsEnvironment @@ -35,6 +37,7 @@ from apache_beam.runners.runner import PipelineRunner from apache_beam.runners.runner import PipelineState from apache_beam.runners.runner import PValueCache +from apache_beam.transforms.ptransform import PTransform from apache_beam.transforms.core import _GroupAlsoByWindow from apache_beam.transforms.core import _GroupByKeyOnly from apache_beam.options.pipeline_options import DirectOptions @@ -54,14 +57,34 @@ @typehints.with_output_types(typehints.KV[K, typehints.Iterable[V]]) class _StreamingGroupByKeyOnly(_GroupByKeyOnly): """Streaming GroupByKeyOnly placeholder for overriding in DirectRunner.""" - pass + urn = "direct_runner:streaming_gbko:v0.1" + + # These are needed due to apply overloads. + def to_runner_api_parameter(self, unused_context): + return _StreamingGroupByKeyOnly.urn, None + + @PTransform.register_urn(urn, None) + def from_runner_api_parameter(unused_payload, unused_context): + return _StreamingGroupByKeyOnly() @typehints.with_input_types(typehints.KV[K, typehints.Iterable[V]]) @typehints.with_output_types(typehints.KV[K, typehints.Iterable[V]]) class _StreamingGroupAlsoByWindow(_GroupAlsoByWindow): """Streaming GroupAlsoByWindow placeholder for overriding in DirectRunner.""" - pass + urn = "direct_runner:streaming_gabw:v0.1" + + # These are needed due to apply overloads. + def to_runner_api_parameter(self, context): + return ( + _StreamingGroupAlsoByWindow.urn, + wrappers_pb2.BytesValue(value=context.windowing_strategies.get_id( + self.windowing))) + + @PTransform.register_urn(urn, wrappers_pb2.BytesValue) + def from_runner_api_parameter(payload, context): + return _StreamingGroupAlsoByWindow( + context.windowing_strategies.get_by_id(payload.value)) class DirectRunner(PipelineRunner): diff --git a/sdks/python/apache_beam/transforms/core.py b/sdks/python/apache_beam/transforms/core.py index 9168a898a181c..671fea4f215d0 100644 --- a/sdks/python/apache_beam/transforms/core.py +++ b/sdks/python/apache_beam/transforms/core.py @@ -698,6 +698,7 @@ def _pardo_fn_data(self): return self.fn, self.args, self.kwargs, si_tags_and_types, windowing def to_runner_api_parameter(self, context): + assert self.__class__ is ParDo return ( urns.PARDO_TRANSFORM, beam_runner_api_pb2.ParDoPayload( @@ -1210,6 +1211,18 @@ def expand(self, pcoll): self._check_pcollection(pcoll) return pvalue.PCollection(pcoll.pipeline) + def to_runner_api_parameter(self, context): + return ( + urns.GROUP_ALSO_BY_WINDOW_TRANSFORM, + wrappers_pb2.BytesValue(value=context.windowing_strategies.get_id( + self.windowing))) + + @PTransform.register_urn( + urns.GROUP_ALSO_BY_WINDOW_TRANSFORM, wrappers_pb2.BytesValue) + def from_runner_api_parameter(payload, context): + return _GroupAlsoByWindow( + context.windowing_strategies.get_by_id(payload.value)) + class _GroupAlsoByWindowDoFn(DoFn): # TODO(robertwb): Support combiner lifting. diff --git a/sdks/python/apache_beam/utils/urns.py b/sdks/python/apache_beam/utils/urns.py index dcdf0f3eee01e..e7ef80bbd4e4e 100644 --- a/sdks/python/apache_beam/utils/urns.py +++ b/sdks/python/apache_beam/utils/urns.py @@ -39,6 +39,7 @@ PICKLED_TRANSFORM = "beam:ptransform:pickled_python:v0.1" PARDO_TRANSFORM = "beam:ptransform:pardo:v0.1" +GROUP_ALSO_BY_WINDOW_TRANSFORM = "beam:ptransform:group_also_by_window:v0.1" COMBINE_PER_KEY_TRANSFORM = "beam:ptransform:combine_per_key:v0.1" COMBINE_GROUPED_VALUES_TRANSFORM = "beam:ptransform:combine_grouped_values:v0.1" FLATTEN_TRANSFORM = "beam:ptransform:flatten:v0.1" From b179eca90f2af262aed637a2a0c099680a0822d2 Mon Sep 17 00:00:00 2001 From: Robert Bradshaw Date: Wed, 26 Jul 2017 18:21:20 -0700 Subject: [PATCH 224/346] Translate GroupByKey[Only] through the Runner API. --- sdks/python/apache_beam/transforms/core.py | 14 ++++++++++++++ sdks/python/apache_beam/utils/urns.py | 2 ++ 2 files changed, 16 insertions(+) diff --git a/sdks/python/apache_beam/transforms/core.py b/sdks/python/apache_beam/transforms/core.py index 671fea4f215d0..cff6dbec86014 100644 --- a/sdks/python/apache_beam/transforms/core.py +++ b/sdks/python/apache_beam/transforms/core.py @@ -1184,6 +1184,13 @@ def expand(self, pcoll): | 'GroupByKey' >> _GroupByKeyOnly() | 'GroupByWindow' >> _GroupAlsoByWindow(pcoll.windowing)) + def to_runner_api_parameter(self, unused_context): + return urns.GROUP_BY_KEY_TRANSFORM, None + + @PTransform.register_urn(urns.GROUP_BY_KEY_TRANSFORM, None) + def from_runner_api_parameter(unused_payload, unused_context): + return GroupByKey() + @typehints.with_input_types(typehints.KV[K, V]) @typehints.with_output_types(typehints.KV[K, typehints.Iterable[V]]) @@ -1197,6 +1204,13 @@ def expand(self, pcoll): self._check_pcollection(pcoll) return pvalue.PCollection(pcoll.pipeline) + def to_runner_api_parameter(self, unused_context): + return urns.GROUP_BY_KEY_ONLY_TRANSFORM, None + + @PTransform.register_urn(urns.GROUP_BY_KEY_ONLY_TRANSFORM, None) + def from_runner_api_parameter(unused_payload, unused_context): + return _GroupByKeyOnly() + @typehints.with_input_types(typehints.KV[K, typehints.Iterable[V]]) @typehints.with_output_types(typehints.KV[K, typehints.Iterable[V]]) diff --git a/sdks/python/apache_beam/utils/urns.py b/sdks/python/apache_beam/utils/urns.py index e7ef80bbd4e4e..0013cb3b6e2c5 100644 --- a/sdks/python/apache_beam/utils/urns.py +++ b/sdks/python/apache_beam/utils/urns.py @@ -39,6 +39,8 @@ PICKLED_TRANSFORM = "beam:ptransform:pickled_python:v0.1" PARDO_TRANSFORM = "beam:ptransform:pardo:v0.1" +GROUP_BY_KEY_TRANSFORM = "beam:ptransform:group_by_key:v0.1" +GROUP_BY_KEY_ONLY_TRANSFORM = "beam:ptransform:group_by_key_only:v0.1" GROUP_ALSO_BY_WINDOW_TRANSFORM = "beam:ptransform:group_also_by_window:v0.1" COMBINE_PER_KEY_TRANSFORM = "beam:ptransform:combine_per_key:v0.1" COMBINE_GROUPED_VALUES_TRANSFORM = "beam:ptransform:combine_grouped_values:v0.1" From 2ee7422f1dbd00e3c40ac95e9eaddc745d46fe65 Mon Sep 17 00:00:00 2001 From: Robert Bradshaw Date: Wed, 26 Jul 2017 18:21:47 -0700 Subject: [PATCH 225/346] More informative references in the proto representation. --- sdks/python/apache_beam/pipeline.py | 3 ++- sdks/python/apache_beam/runners/pipeline_context.py | 8 ++++---- 2 files changed, 6 insertions(+), 5 deletions(-) diff --git a/sdks/python/apache_beam/pipeline.py b/sdks/python/apache_beam/pipeline.py index 8553f7ca7eab9..e7c23225d51cd 100644 --- a/sdks/python/apache_beam/pipeline.py +++ b/sdks/python/apache_beam/pipeline.py @@ -730,7 +730,8 @@ def transform_to_runner_api(transform, context): return beam_runner_api_pb2.PTransform( unique_name=self.full_label, spec=transform_to_runner_api(self.transform, context), - subtransforms=[context.transforms.get_id(part) for part in self.parts], + subtransforms=[context.transforms.get_id(part, label=part.full_label) + for part in self.parts], # TODO(BEAM-115): Side inputs. inputs={tag: context.pcollections.get_id(pc) for tag, pc in self.named_inputs().items()}, diff --git a/sdks/python/apache_beam/runners/pipeline_context.py b/sdks/python/apache_beam/runners/pipeline_context.py index a40069b4280e2..f4de42ac82af1 100644 --- a/sdks/python/apache_beam/runners/pipeline_context.py +++ b/sdks/python/apache_beam/runners/pipeline_context.py @@ -43,18 +43,18 @@ def __init__(self, context, obj_type, proto_map=None): self._id_to_proto = proto_map if proto_map else {} self._counter = 0 - def _unique_ref(self, obj=None): + def _unique_ref(self, obj=None, label=None): self._counter += 1 return "ref_%s_%s_%s" % ( - self._obj_type.__name__, type(obj).__name__, self._counter) + self._obj_type.__name__, label or type(obj).__name__, self._counter) def populate_map(self, proto_map): for id, proto in self._id_to_proto.items(): proto_map[id].CopyFrom(proto) - def get_id(self, obj): + def get_id(self, obj, label=None): if obj not in self._obj_to_id: - id = self._unique_ref(obj) + id = self._unique_ref(obj, label) self._id_to_obj[id] = obj self._obj_to_id[obj] = id self._id_to_proto[id] = obj.to_runner_api(self._pipeline_context) From e23964467a57e91b3d2ce20981a5c05f80748ce5 Mon Sep 17 00:00:00 2001 From: Sourabh Bajaj Date: Wed, 2 Aug 2017 07:44:16 -0700 Subject: [PATCH 226/346] Add the Beam Job API service definition --- .../src/main/proto/beam_job_api.proto | 143 ++++++++++++++++++ 1 file changed, 143 insertions(+) create mode 100644 sdks/common/runner-api/src/main/proto/beam_job_api.proto diff --git a/sdks/common/runner-api/src/main/proto/beam_job_api.proto b/sdks/common/runner-api/src/main/proto/beam_job_api.proto new file mode 100644 index 0000000000000..7be14cc90f486 --- /dev/null +++ b/sdks/common/runner-api/src/main/proto/beam_job_api.proto @@ -0,0 +1,143 @@ +/* + * 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. + */ + +/* + * Protocol Buffers describing the Job API, api for communicating with a runner + * for job submission over GRPC. + */ + +syntax = "proto3"; + +package org.apache.beam.runner_api.v1; + +option java_package = "org.apache.beam.sdk.common.runner.v1"; +option java_outer_classname = "JobApi"; + +import "beam_runner_api.proto"; +import "google/protobuf/struct.proto"; + + +// Job Service for running RunnerAPI pipelines +service JobService { + // Submit the job for execution + rpc run (SubmitJobRequest) returns (SubmitJobResponse) {} + + // Get the current state of the job + rpc getState (GetJobStateRequest) returns (GetJobStateResponse) {} + + // Cancel the job + rpc cancel (CancelJobRequest) returns (CancelJobResponse) {} + + // Subscribe to a stream of state changes of the job, will immediately return the current state of the job as the first response. + rpc getStateStream (GetJobStateRequest) returns (stream GetJobStateResponse) {} + + // Subscribe to a stream of state changes and messages from the job + rpc getMessageStream (JobMessagesRequest) returns (stream JobMessagesResponse) {} +} + + +// Submit is a synchronus request that returns a jobId back +// Throws error GRPC_STATUS_UNAVAILABLE if server is down +// Throws error ALREADY_EXISTS if the jobName is reused as runners are permitted to deduplicate based on the name of the job. +// Throws error UNKNOWN for all other issues +message SubmitJobRequest { + org.apache.beam.runner_api.v1.Pipeline pipeline = 1; // (required) + google.protobuf.Struct pipelineOptions = 2; // (required) + string jobName = 3; // (required) +} + +message SubmitJobResponse { + // JobId is used as an identifier for the job in all future calls. + string jobId = 1; // (required) +} + + +// Cancel is a synchronus request that returns a jobState back +// Throws error GRPC_STATUS_UNAVAILABLE if server is down +// Throws error NOT_FOUND if the jobId is not found +message CancelJobRequest { + string jobId = 1; // (required) + +} + +// Valid responses include any terminal state or CANCELLING +message CancelJobResponse { + JobState.JobStateType state = 1; // (required) +} + + +// GetState is a synchronus request that returns a jobState back +// Throws error GRPC_STATUS_UNAVAILABLE if server is down +// Throws error NOT_FOUND if the jobId is not found +message GetJobStateRequest { + string jobId = 1; // (required) + +} + +message GetJobStateResponse { + JobState.JobStateType state = 1; // (required) +} + + +// GetJobMessages is a streaming api for streaming job messages from the service +// One request will connect you to the job and you'll get a stream of job state +// and job messages back; one is used for logging and the other for detecting +// the job ended. +message JobMessagesRequest { + string jobId = 1; // (required) + +} + +message JobMessage { + string messageId = 1; + string time = 2; + MessageImportance importance = 3; + string messageText = 4; + + enum MessageImportance { + JOB_MESSAGE_DEBUG = 0; + JOB_MESSAGE_DETAILED = 1; + JOB_MESSAGE_BASIC = 2; + JOB_MESSAGE_WARNING = 3; + JOB_MESSAGE_ERROR = 4; + } +} + +message JobMessagesResponse { + oneof response { + JobMessage messageResponse = 1; + GetJobStateResponse stateResponse = 2; + } +} + +message JobState { + // Enumeration of all JobStates + enum JobStateType { + UNKNOWN = 0; + STOPPED = 1; + RUNNING = 2; + DONE = 3; + FAILED = 4; + CANCELLED = 5; + UPDATED = 6; + DRAINING = 7; + DRAINED = 8; + STARTING = 9; + CANCELLING = 10; + } +} From ff4b36c8ae1bd5e436ad63a32997273c8b4a97fe Mon Sep 17 00:00:00 2001 From: Eugene Kirpichov Date: Thu, 27 Jul 2017 13:05:23 -0700 Subject: [PATCH 227/346] [BEAM-2670] Fixes SparkRuntimeContext.getPipelineOptions() It used a global variable to store the deserialized options, so even if there were several instances of SparkRuntimeContext created with different PipelineOptions, they would all return the same value depending on which one was asked first. --- .../spark/translation/EvaluationContext.java | 2 +- .../translation/SparkRuntimeContext.java | 48 ++++++++----------- .../translation/SparkRuntimeContextTest.java | 2 +- 3 files changed, 22 insertions(+), 30 deletions(-) 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 0c6c4d1cb6607..23e430a671440 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 @@ -65,7 +65,7 @@ public EvaluationContext(JavaSparkContext jsc, Pipeline pipeline, PipelineOption this.jsc = jsc; this.pipeline = pipeline; this.options = options; - this.runtime = new SparkRuntimeContext(pipeline, options); + this.runtime = new SparkRuntimeContext(options); } public EvaluationContext( 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 f3fe99c9c074c..6361bb2cc0fad 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 @@ -21,11 +21,12 @@ import com.fasterxml.jackson.core.JsonProcessingException; import com.fasterxml.jackson.databind.Module; import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.base.Function; +import com.google.common.base.Supplier; +import com.google.common.base.Suppliers; import java.io.IOException; import java.io.Serializable; -import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.coders.CoderRegistry; -import org.apache.beam.sdk.io.FileSystems; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.util.common.ReflectHelpers; @@ -34,11 +35,16 @@ * data flow program is launched. */ public class SparkRuntimeContext implements Serializable { - private final String serializedPipelineOptions; + private final Supplier optionsSupplier; private transient CoderRegistry coderRegistry; - SparkRuntimeContext(Pipeline pipeline, PipelineOptions options) { - this.serializedPipelineOptions = serializePipelineOptions(options); + SparkRuntimeContext(PipelineOptions options) { + String serializedPipelineOptions = serializePipelineOptions(options); + this.optionsSupplier = + Suppliers.memoize( + Suppliers.compose( + new DeserializeOptions(), + Suppliers.ofInstance(serializedPipelineOptions))); } /** @@ -59,16 +65,8 @@ private String serializePipelineOptions(PipelineOptions pipelineOptions) { } } - private static PipelineOptions deserializePipelineOptions(String serializedPipelineOptions) { - try { - return createMapper().readValue(serializedPipelineOptions, PipelineOptions.class); - } catch (IOException e) { - throw new IllegalStateException("Failed to deserialize the pipeline options.", e); - } - } - public PipelineOptions getPipelineOptions() { - return PipelineOptionsHolder.getOrInit(serializedPipelineOptions); + return optionsSupplier.get(); } public CoderRegistry getCoderRegistry() { @@ -78,21 +76,15 @@ public CoderRegistry getCoderRegistry() { return coderRegistry; } - private static class PipelineOptionsHolder { - // on executors, this should deserialize once. - private static transient volatile PipelineOptions pipelineOptions = null; - - static PipelineOptions getOrInit(String serializedPipelineOptions) { - if (pipelineOptions == null) { - synchronized (PipelineOptionsHolder.class) { - if (pipelineOptions == null) { - pipelineOptions = deserializePipelineOptions(serializedPipelineOptions); - } - } - // Register standard FileSystems. - FileSystems.setDefaultPipelineOptions(pipelineOptions); + private static class DeserializeOptions + implements Function, Serializable { + @Override + public PipelineOptions apply(String options) { + try { + return createMapper().readValue(options, PipelineOptions.class); + } catch (IOException e) { + throw new IllegalStateException("Failed to deserialize the pipeline options.", e); } - return pipelineOptions; } } } diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/SparkRuntimeContextTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/SparkRuntimeContextTest.java index e8f578a54d8ef..456056af733d5 100644 --- a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/SparkRuntimeContextTest.java +++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/SparkRuntimeContextTest.java @@ -105,7 +105,7 @@ public void testSerializingPipelineOptionsWithCustomUserType() throws Exception .as(JacksonIncompatibleOptions.class); options.setRunner(CrashingRunner.class); Pipeline p = Pipeline.create(options); - SparkRuntimeContext context = new SparkRuntimeContext(p, options); + SparkRuntimeContext context = new SparkRuntimeContext(options); ByteArrayOutputStream baos = new ByteArrayOutputStream(); try (ObjectOutputStream outputStream = new ObjectOutputStream(baos)) { From 7db051aeae2b8e6b2dbfcc1da31410ec118299f6 Mon Sep 17 00:00:00 2001 From: Eugene Kirpichov Date: Fri, 28 Jul 2017 12:48:41 -0700 Subject: [PATCH 228/346] Introduces SerializablePipelineOptions in core-construction Removes analogous classes from spark/flink and their tests. The analogous class in Spark was SparkRuntimeContext, which also contained a CoderRegistry, but the CoderRegistry was used only in a class that was itself unused. I removed that class. This also allows removing a bunch of Jackson dependencies from Spark runner. --- runners/apex/pom.xml | 8 - .../operators/ApexGroupByKeyOperator.java | 6 +- .../operators/ApexParDoOperator.java | 6 +- .../ApexReadUnboundedInputOperator.java | 6 +- .../utils/SerializablePipelineOptions.java | 78 --------- .../utils/PipelineOptionsTest.java | 150 ---------------- runners/core-construction-java/pom.xml | 15 ++ .../SerializablePipelineOptions.java | 74 ++++++++ .../SerializablePipelineOptionsTest.java | 89 ++++++++++ runners/flink/pom.xml | 10 -- .../functions/FlinkDoFnFunction.java | 10 +- .../FlinkMergingNonShuffleReduceFunction.java | 8 +- .../functions/FlinkPartialReduceFunction.java | 8 +- .../functions/FlinkReduceFunction.java | 8 +- .../functions/FlinkStatefulDoFnFunction.java | 10 +- .../utils/SerializedPipelineOptions.java | 77 -------- .../wrappers/SourceInputFormat.java | 10 +- .../wrappers/streaming/DoFnOperator.java | 10 +- .../streaming/SplittableDoFnOperator.java | 2 +- .../streaming/io/BoundedSourceWrapper.java | 10 +- .../streaming/io/UnboundedSourceWrapper.java | 12 +- .../runners/flink/PipelineOptionsTest.java | 165 +----------------- runners/spark/pom.xml | 12 -- .../spark/aggregators/NamedAggregators.java | 93 ---------- .../beam/runners/spark/io/SourceDStream.java | 20 +-- .../beam/runners/spark/io/SourceRDD.java | 22 +-- .../spark/io/SparkUnboundedSource.java | 6 +- .../SparkGroupAlsoByWindowViaWindowSet.java | 10 +- .../spark/stateful/StateSpecFunctions.java | 8 +- .../spark/translation/EvaluationContext.java | 11 +- .../spark/translation/MultiDoFnFunction.java | 16 +- .../translation/SparkAbstractCombineFn.java | 9 +- .../translation/SparkGlobalCombineFn.java | 5 +- ...arkGroupAlsoByWindowViaOutputBufferFn.java | 9 +- .../translation/SparkKeyedCombineFn.java | 5 +- .../translation/SparkRuntimeContext.java | 90 ---------- .../translation/TransformTranslator.java | 27 ++- .../StreamingTransformTranslator.java | 20 +-- .../translation/SparkRuntimeContextTest.java | 122 ------------- .../beam/sdk/options/PipelineOptions.java | 7 +- .../beam/sdk/options/ValueProviders.java | 8 +- 41 files changed, 327 insertions(+), 945 deletions(-) delete mode 100644 runners/apex/src/main/java/org/apache/beam/runners/apex/translation/utils/SerializablePipelineOptions.java delete mode 100644 runners/apex/src/test/java/org/apache/beam/runners/apex/translation/utils/PipelineOptionsTest.java create mode 100644 runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/SerializablePipelineOptions.java create mode 100644 runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/SerializablePipelineOptionsTest.java delete mode 100644 runners/flink/src/main/java/org/apache/beam/runners/flink/translation/utils/SerializedPipelineOptions.java delete mode 100644 runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkRuntimeContext.java delete mode 100644 runners/spark/src/test/java/org/apache/beam/runners/spark/translation/SparkRuntimeContextTest.java diff --git a/runners/apex/pom.xml b/runners/apex/pom.xml index fd5aafb992be6..96aac8bbd404c 100644 --- a/runners/apex/pom.xml +++ b/runners/apex/pom.xml @@ -62,14 +62,6 @@ malhar-library ${apex.malhar.version} - - com.fasterxml.jackson.core - jackson-core - - - com.fasterxml.jackson.core - jackson-databind - org.apache.apex apex-engine 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 39f681fcb821c..5c0d72f01e587 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 @@ -33,7 +33,6 @@ import org.apache.beam.runners.apex.ApexPipelineOptions; import org.apache.beam.runners.apex.translation.utils.ApexStateInternals.ApexStateBackend; import org.apache.beam.runners.apex.translation.utils.ApexStreamTuple; -import org.apache.beam.runners.apex.translation.utils.SerializablePipelineOptions; import org.apache.beam.runners.core.NullSideInputReader; import org.apache.beam.runners.core.OutputWindowedValue; import org.apache.beam.runners.core.ReduceFnRunner; @@ -41,6 +40,7 @@ import org.apache.beam.runners.core.SystemReduceFn; import org.apache.beam.runners.core.TimerInternals; import org.apache.beam.runners.core.TimerInternals.TimerData; +import org.apache.beam.runners.core.construction.SerializablePipelineOptions; import org.apache.beam.runners.core.construction.TriggerTranslation; import org.apache.beam.runners.core.triggers.ExecutableTriggerStateMachine; import org.apache.beam.runners.core.triggers.TriggerStateMachines; @@ -149,7 +149,9 @@ public void endWindow() { @Override public void setup(OperatorContext context) { - this.traceTuples = ApexStreamTuple.Logging.isDebugEnabled(serializedOptions.get(), this); + this.traceTuples = + ApexStreamTuple.Logging.isDebugEnabled( + serializedOptions.get().as(ApexPipelineOptions.class), this); } @Override 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 c3cbab2c54987..4dc807d5c57de 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 @@ -40,7 +40,6 @@ import org.apache.beam.runners.apex.translation.utils.ApexStateInternals.ApexStateBackend; import org.apache.beam.runners.apex.translation.utils.ApexStreamTuple; 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.StateInternalsProxy; import org.apache.beam.runners.apex.translation.utils.ValueAndCoderKryoSerializable; import org.apache.beam.runners.core.DoFnRunner; @@ -64,6 +63,7 @@ import org.apache.beam.runners.core.TimerInternals; import org.apache.beam.runners.core.TimerInternals.TimerData; import org.apache.beam.runners.core.TimerInternalsFactory; +import org.apache.beam.runners.core.construction.SerializablePipelineOptions; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.KvCoder; import org.apache.beam.sdk.coders.ListCoder; @@ -386,7 +386,9 @@ private void outputWatermark(ApexStreamTuple.WatermarkTuple mark) { @Override public void setup(OperatorContext context) { - this.traceTuples = ApexStreamTuple.Logging.isDebugEnabled(pipelineOptions.get(), this); + this.traceTuples = + ApexStreamTuple.Logging.isDebugEnabled( + pipelineOptions.get().as(ApexPipelineOptions.class), this); SideInputReader sideInputReader = NullSideInputReader.of(sideInputs); if (!sideInputs.isEmpty()) { sideInputHandler = new SideInputHandler(sideInputs, sideInputStateInternals); diff --git a/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/operators/ApexReadUnboundedInputOperator.java b/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/operators/ApexReadUnboundedInputOperator.java index 1549560f56fe6..21fb9d2dd7de5 100644 --- a/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/operators/ApexReadUnboundedInputOperator.java +++ b/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/operators/ApexReadUnboundedInputOperator.java @@ -30,8 +30,8 @@ import org.apache.beam.runners.apex.ApexPipelineOptions; import org.apache.beam.runners.apex.translation.utils.ApexStreamTuple; import org.apache.beam.runners.apex.translation.utils.ApexStreamTuple.DataTuple; -import org.apache.beam.runners.apex.translation.utils.SerializablePipelineOptions; import org.apache.beam.runners.apex.translation.utils.ValuesSource; +import org.apache.beam.runners.core.construction.SerializablePipelineOptions; import org.apache.beam.sdk.io.UnboundedSource; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.GlobalWindow; @@ -119,7 +119,9 @@ public void endWindow() { @Override public void setup(OperatorContext context) { - this.traceTuples = ApexStreamTuple.Logging.isDebugEnabled(pipelineOptions.get(), this); + this.traceTuples = + ApexStreamTuple.Logging.isDebugEnabled( + pipelineOptions.get().as(ApexPipelineOptions.class), this); try { reader = source.createReader(this.pipelineOptions.get(), null); available = reader.start(); diff --git a/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/utils/SerializablePipelineOptions.java b/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/utils/SerializablePipelineOptions.java deleted file mode 100644 index 46b04fc712fbb..0000000000000 --- a/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/utils/SerializablePipelineOptions.java +++ /dev/null @@ -1,78 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.beam.runners.apex.translation.utils; - -import com.fasterxml.jackson.databind.Module; -import com.fasterxml.jackson.databind.ObjectMapper; -import java.io.Externalizable; -import java.io.IOException; -import java.io.ObjectInput; -import java.io.ObjectOutput; -import java.util.concurrent.atomic.AtomicBoolean; -import org.apache.beam.runners.apex.ApexPipelineOptions; -import org.apache.beam.sdk.io.FileSystems; -import org.apache.beam.sdk.options.PipelineOptions; -import org.apache.beam.sdk.util.common.ReflectHelpers; - -/** - * A wrapper to enable serialization of {@link PipelineOptions}. - */ -public class SerializablePipelineOptions implements Externalizable { - - /* Used to ensure we initialize file systems exactly once, because it's a slow operation. */ - private static final AtomicBoolean FILE_SYSTEMS_INTIIALIZED = new AtomicBoolean(false); - - private transient ApexPipelineOptions pipelineOptions; - - public SerializablePipelineOptions(ApexPipelineOptions pipelineOptions) { - this.pipelineOptions = pipelineOptions; - } - - public SerializablePipelineOptions() { - } - - public ApexPipelineOptions get() { - return this.pipelineOptions; - } - - @Override - public void writeExternal(ObjectOutput out) throws IOException { - out.writeUTF(createMapper().writeValueAsString(pipelineOptions)); - } - - @Override - public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException { - String s = in.readUTF(); - this.pipelineOptions = createMapper().readValue(s, PipelineOptions.class) - .as(ApexPipelineOptions.class); - - if (FILE_SYSTEMS_INTIIALIZED.compareAndSet(false, true)) { - FileSystems.setDefaultPipelineOptions(pipelineOptions); - } - } - - /** - * Use an {@link ObjectMapper} configured with any {@link Module}s in the class path allowing - * for user specified configuration injection into the ObjectMapper. This supports user custom - * types on {@link PipelineOptions}. - */ - private static ObjectMapper createMapper() { - return new ObjectMapper().registerModules( - ObjectMapper.findModules(ReflectHelpers.findClassLoader())); - } -} diff --git a/runners/apex/src/test/java/org/apache/beam/runners/apex/translation/utils/PipelineOptionsTest.java b/runners/apex/src/test/java/org/apache/beam/runners/apex/translation/utils/PipelineOptionsTest.java deleted file mode 100644 index 118ff996bfa8c..0000000000000 --- a/runners/apex/src/test/java/org/apache/beam/runners/apex/translation/utils/PipelineOptionsTest.java +++ /dev/null @@ -1,150 +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.apex.translation.utils; - -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNotNull; - -import com.datatorrent.common.util.FSStorageAgent; -import com.esotericsoftware.kryo.serializers.FieldSerializer.Bind; -import com.esotericsoftware.kryo.serializers.JavaSerializer; -import com.fasterxml.jackson.core.JsonGenerator; -import com.fasterxml.jackson.core.JsonParser; -import com.fasterxml.jackson.core.JsonProcessingException; -import com.fasterxml.jackson.databind.DeserializationContext; -import com.fasterxml.jackson.databind.JsonDeserializer; -import com.fasterxml.jackson.databind.JsonSerializer; -import com.fasterxml.jackson.databind.Module; -import com.fasterxml.jackson.databind.SerializerProvider; -import com.fasterxml.jackson.databind.annotation.JsonDeserialize; -import com.fasterxml.jackson.databind.annotation.JsonSerialize; -import com.fasterxml.jackson.databind.module.SimpleModule; -import com.google.auto.service.AutoService; -import java.io.ByteArrayInputStream; -import java.io.ByteArrayOutputStream; -import java.io.IOException; -import org.apache.beam.runners.apex.ApexPipelineOptions; -import org.apache.beam.sdk.options.Default; -import org.apache.beam.sdk.options.Description; -import org.apache.beam.sdk.options.PipelineOptionsFactory; -import org.junit.Test; - -/** - * Tests the serialization of PipelineOptions. - */ -public class PipelineOptionsTest { - - /** - * Interface for testing. - */ - public interface MyOptions extends ApexPipelineOptions { - @Description("Bla bla bla") - @Default.String("Hello") - String getTestOption(); - void setTestOption(String value); - } - - private static class OptionsWrapper { - private OptionsWrapper() { - this(null); // required for Kryo - } - private OptionsWrapper(ApexPipelineOptions options) { - this.options = new SerializablePipelineOptions(options); - } - @Bind(JavaSerializer.class) - private final SerializablePipelineOptions options; - } - - @Test - public void testSerialization() { - OptionsWrapper wrapper = new OptionsWrapper( - PipelineOptionsFactory.fromArgs("--testOption=nothing").as(MyOptions.class)); - ByteArrayOutputStream bos = new ByteArrayOutputStream(); - FSStorageAgent.store(bos, wrapper); - - ByteArrayInputStream bis = new ByteArrayInputStream(bos.toByteArray()); - OptionsWrapper wrapperCopy = (OptionsWrapper) FSStorageAgent.retrieve(bis); - assertNotNull(wrapperCopy.options); - assertEquals("nothing", wrapperCopy.options.get().as(MyOptions.class).getTestOption()); - } - - @Test - public void testSerializationWithUserCustomType() { - OptionsWrapper wrapper = new OptionsWrapper( - PipelineOptionsFactory.fromArgs("--jacksonIncompatible=\"testValue\"") - .as(JacksonIncompatibleOptions.class)); - ByteArrayOutputStream bos = new ByteArrayOutputStream(); - FSStorageAgent.store(bos, wrapper); - - ByteArrayInputStream bis = new ByteArrayInputStream(bos.toByteArray()); - OptionsWrapper wrapperCopy = (OptionsWrapper) FSStorageAgent.retrieve(bis); - assertNotNull(wrapperCopy.options); - assertEquals("testValue", - wrapperCopy.options.get().as(JacksonIncompatibleOptions.class) - .getJacksonIncompatible().value); - } - - /** PipelineOptions used to test auto registration of Jackson modules. */ - public interface JacksonIncompatibleOptions extends ApexPipelineOptions { - JacksonIncompatible getJacksonIncompatible(); - void setJacksonIncompatible(JacksonIncompatible value); - } - - /** A Jackson {@link Module} to test auto-registration of modules. */ - @AutoService(Module.class) - public static class RegisteredTestModule extends SimpleModule { - public RegisteredTestModule() { - super("RegisteredTestModule"); - setMixInAnnotation(JacksonIncompatible.class, JacksonIncompatibleMixin.class); - } - } - - /** A class which Jackson does not know how to serialize/deserialize. */ - public static class JacksonIncompatible { - private final String value; - public JacksonIncompatible(String value) { - this.value = value; - } - } - - /** A Jackson mixin used to add annotations to other classes. */ - @JsonDeserialize(using = JacksonIncompatibleDeserializer.class) - @JsonSerialize(using = JacksonIncompatibleSerializer.class) - public static final class JacksonIncompatibleMixin {} - - /** A Jackson deserializer for {@link JacksonIncompatible}. */ - public static class JacksonIncompatibleDeserializer extends - JsonDeserializer { - - @Override - public JacksonIncompatible deserialize(JsonParser jsonParser, - DeserializationContext deserializationContext) throws IOException, JsonProcessingException { - return new JacksonIncompatible(jsonParser.readValueAs(String.class)); - } - } - - /** A Jackson serializer for {@link JacksonIncompatible}. */ - public static class JacksonIncompatibleSerializer extends JsonSerializer { - - @Override - public void serialize(JacksonIncompatible jacksonIncompatible, JsonGenerator jsonGenerator, - SerializerProvider serializerProvider) throws IOException, JsonProcessingException { - jsonGenerator.writeString(jacksonIncompatible.value); - } - } -} diff --git a/runners/core-construction-java/pom.xml b/runners/core-construction-java/pom.xml index b85b5f5ff5413..1a529140ec91a 100644 --- a/runners/core-construction-java/pom.xml +++ b/runners/core-construction-java/pom.xml @@ -64,6 +64,21 @@ beam-sdks-java-core + + com.fasterxml.jackson.core + jackson-annotations + + + + com.fasterxml.jackson.core + jackson-databind + + + + com.fasterxml.jackson.core + jackson-core + + com.google.protobuf protobuf-java diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/SerializablePipelineOptions.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/SerializablePipelineOptions.java new file mode 100644 index 0000000000000..e697fb237f181 --- /dev/null +++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/SerializablePipelineOptions.java @@ -0,0 +1,74 @@ +/* + * 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.construction; + +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.ObjectMapper; +import java.io.IOException; +import java.io.ObjectInputStream; +import java.io.Serializable; +import org.apache.beam.sdk.io.FileSystems; +import org.apache.beam.sdk.options.PipelineOptions; +import org.apache.beam.sdk.util.common.ReflectHelpers; + +/** + * Holds a {@link PipelineOptions} in JSON serialized form and calls {@link + * FileSystems#setDefaultPipelineOptions(PipelineOptions)} on construction or on deserialization. + */ +public class SerializablePipelineOptions implements Serializable { + private static final ObjectMapper MAPPER = + new ObjectMapper() + .registerModules(ObjectMapper.findModules(ReflectHelpers.findClassLoader())); + + private final String serializedPipelineOptions; + private transient PipelineOptions options; + + public SerializablePipelineOptions(PipelineOptions options) { + this.serializedPipelineOptions = serializeToJson(options); + this.options = options; + FileSystems.setDefaultPipelineOptions(options); + } + + public PipelineOptions get() { + return options; + } + + private void readObject(ObjectInputStream is) throws IOException, ClassNotFoundException { + is.defaultReadObject(); + this.options = deserializeFromJson(serializedPipelineOptions); + // TODO https://issues.apache.org/jira/browse/BEAM-2712: remove this call. + FileSystems.setDefaultPipelineOptions(options); + } + + private static String serializeToJson(PipelineOptions options) { + try { + return MAPPER.writeValueAsString(options); + } catch (JsonProcessingException e) { + throw new IllegalArgumentException("Failed to serialize PipelineOptions", e); + } + } + + private static PipelineOptions deserializeFromJson(String options) { + try { + return MAPPER.readValue(options, PipelineOptions.class); + } catch (IOException e) { + throw new IllegalArgumentException("Failed to deserialize PipelineOptions", e); + } + } +} diff --git a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/SerializablePipelineOptionsTest.java b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/SerializablePipelineOptionsTest.java new file mode 100644 index 0000000000000..cd470b22648e6 --- /dev/null +++ b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/SerializablePipelineOptionsTest.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.core.construction; + +import static org.junit.Assert.assertEquals; + +import com.fasterxml.jackson.annotation.JsonIgnore; +import org.apache.beam.sdk.options.Default; +import org.apache.beam.sdk.options.PipelineOptions; +import org.apache.beam.sdk.options.PipelineOptionsFactory; +import org.apache.beam.sdk.util.SerializableUtils; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +/** Tests for {@link SerializablePipelineOptions}. */ +@RunWith(JUnit4.class) +public class SerializablePipelineOptionsTest { + /** Options for testing. */ + public interface MyOptions extends PipelineOptions { + String getFoo(); + + void setFoo(String foo); + + @JsonIgnore + @Default.String("not overridden") + String getIgnoredField(); + + void setIgnoredField(String value); + } + + @Test + public void testSerializationAndDeserialization() throws Exception { + PipelineOptions options = + PipelineOptionsFactory.fromArgs("--foo=testValue", "--ignoredField=overridden") + .as(MyOptions.class); + + SerializablePipelineOptions serializableOptions = new SerializablePipelineOptions(options); + assertEquals("testValue", serializableOptions.get().as(MyOptions.class).getFoo()); + assertEquals("overridden", serializableOptions.get().as(MyOptions.class).getIgnoredField()); + + SerializablePipelineOptions copy = SerializableUtils.clone(serializableOptions); + assertEquals("testValue", copy.get().as(MyOptions.class).getFoo()); + assertEquals("not overridden", copy.get().as(MyOptions.class).getIgnoredField()); + } + + @Test + public void testIndependence() throws Exception { + SerializablePipelineOptions first = + new SerializablePipelineOptions( + PipelineOptionsFactory.fromArgs("--foo=first").as(MyOptions.class)); + SerializablePipelineOptions firstCopy = SerializableUtils.clone(first); + SerializablePipelineOptions second = + new SerializablePipelineOptions( + PipelineOptionsFactory.fromArgs("--foo=second").as(MyOptions.class)); + SerializablePipelineOptions secondCopy = SerializableUtils.clone(second); + + assertEquals("first", first.get().as(MyOptions.class).getFoo()); + assertEquals("first", firstCopy.get().as(MyOptions.class).getFoo()); + assertEquals("second", second.get().as(MyOptions.class).getFoo()); + assertEquals("second", secondCopy.get().as(MyOptions.class).getFoo()); + + first.get().as(MyOptions.class).setFoo("new first"); + firstCopy.get().as(MyOptions.class).setFoo("new firstCopy"); + second.get().as(MyOptions.class).setFoo("new second"); + secondCopy.get().as(MyOptions.class).setFoo("new secondCopy"); + + assertEquals("new first", first.get().as(MyOptions.class).getFoo()); + assertEquals("new firstCopy", firstCopy.get().as(MyOptions.class).getFoo()); + assertEquals("new second", second.get().as(MyOptions.class).getFoo()); + assertEquals("new secondCopy", secondCopy.get().as(MyOptions.class).getFoo()); + } +} diff --git a/runners/flink/pom.xml b/runners/flink/pom.xml index c063a2de425e2..06746fdff2f45 100644 --- a/runners/flink/pom.xml +++ b/runners/flink/pom.xml @@ -255,16 +255,6 @@ jackson-annotations - - com.fasterxml.jackson.core - jackson-core - - - - com.fasterxml.jackson.core - jackson-databind - - com.google.guava guava diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkDoFnFunction.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkDoFnFunction.java index d8ed622ffb890..30481688f6a09 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkDoFnFunction.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkDoFnFunction.java @@ -22,9 +22,9 @@ import java.util.Map; import org.apache.beam.runners.core.DoFnRunner; import org.apache.beam.runners.core.DoFnRunners; +import org.apache.beam.runners.core.construction.SerializablePipelineOptions; import org.apache.beam.runners.flink.FlinkPipelineOptions; import org.apache.beam.runners.flink.metrics.DoFnRunnerWithMetricsUpdate; -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.join.RawUnionValue; @@ -50,7 +50,7 @@ public class FlinkDoFnFunction extends RichMapPartitionFunction, WindowedValue> { - private final SerializedPipelineOptions serializedOptions; + private final SerializablePipelineOptions serializedOptions; private final DoFn doFn; private final String stepName; @@ -75,7 +75,7 @@ public FlinkDoFnFunction( this.doFn = doFn; this.stepName = stepName; this.sideInputs = sideInputs; - this.serializedOptions = new SerializedPipelineOptions(options); + this.serializedOptions = new SerializablePipelineOptions(options); this.windowingStrategy = windowingStrategy; this.outputMap = outputMap; this.mainOutputTag = mainOutputTag; @@ -101,7 +101,7 @@ public void mapPartition( List> additionalOutputTags = Lists.newArrayList(outputMap.keySet()); DoFnRunner doFnRunner = DoFnRunners.simpleRunner( - serializedOptions.getPipelineOptions(), doFn, + serializedOptions.get(), doFn, new FlinkSideInputReader(sideInputs, runtimeContext), outputManager, mainOutputTag, @@ -109,7 +109,7 @@ public void mapPartition( new FlinkNoOpStepContext(), windowingStrategy); - if ((serializedOptions.getPipelineOptions().as(FlinkPipelineOptions.class)) + if ((serializedOptions.get().as(FlinkPipelineOptions.class)) .getEnableMetrics()) { doFnRunner = new DoFnRunnerWithMetricsUpdate<>(stepName, doFnRunner, getRuntimeContext()); } diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkMergingNonShuffleReduceFunction.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkMergingNonShuffleReduceFunction.java index 13be91312497d..c73dade29ad5b 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkMergingNonShuffleReduceFunction.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkMergingNonShuffleReduceFunction.java @@ -18,7 +18,7 @@ package org.apache.beam.runners.flink.translation.functions; import java.util.Map; -import org.apache.beam.runners.flink.translation.utils.SerializedPipelineOptions; +import org.apache.beam.runners.core.construction.SerializablePipelineOptions; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.CombineFnBase; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; @@ -47,7 +47,7 @@ public class FlinkMergingNonShuffleReduceFunction< private final Map, WindowingStrategy> sideInputs; - private final SerializedPipelineOptions serializedOptions; + private final SerializablePipelineOptions serializedOptions; public FlinkMergingNonShuffleReduceFunction( CombineFnBase.GlobalCombineFn combineFn, @@ -60,7 +60,7 @@ public FlinkMergingNonShuffleReduceFunction( this.windowingStrategy = windowingStrategy; this.sideInputs = sideInputs; - this.serializedOptions = new SerializedPipelineOptions(pipelineOptions); + this.serializedOptions = new SerializablePipelineOptions(pipelineOptions); } @@ -69,7 +69,7 @@ public void reduce( Iterable>> elements, Collector>> out) throws Exception { - PipelineOptions options = serializedOptions.getPipelineOptions(); + PipelineOptions options = serializedOptions.get(); FlinkSideInputReader sideInputReader = new FlinkSideInputReader(sideInputs, getRuntimeContext()); diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkPartialReduceFunction.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkPartialReduceFunction.java index db12a49e76f2e..49e821c0efcdd 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkPartialReduceFunction.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkPartialReduceFunction.java @@ -18,7 +18,7 @@ package org.apache.beam.runners.flink.translation.functions; import java.util.Map; -import org.apache.beam.runners.flink.translation.utils.SerializedPipelineOptions; +import org.apache.beam.runners.core.construction.SerializablePipelineOptions; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.CombineFnBase; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; @@ -46,7 +46,7 @@ public class FlinkPartialReduceFunction windowingStrategy; - protected final SerializedPipelineOptions serializedOptions; + protected final SerializablePipelineOptions serializedOptions; protected final Map, WindowingStrategy> sideInputs; @@ -59,7 +59,7 @@ public FlinkPartialReduceFunction( this.combineFn = combineFn; this.windowingStrategy = windowingStrategy; this.sideInputs = sideInputs; - this.serializedOptions = new SerializedPipelineOptions(pipelineOptions); + this.serializedOptions = new SerializablePipelineOptions(pipelineOptions); } @@ -68,7 +68,7 @@ public void combine( Iterable>> elements, Collector>> out) throws Exception { - PipelineOptions options = serializedOptions.getPipelineOptions(); + PipelineOptions options = serializedOptions.get(); FlinkSideInputReader sideInputReader = new FlinkSideInputReader(sideInputs, getRuntimeContext()); diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkReduceFunction.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkReduceFunction.java index 53d71d85c376c..6645b3ac6a358 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkReduceFunction.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkReduceFunction.java @@ -18,7 +18,7 @@ package org.apache.beam.runners.flink.translation.functions; import java.util.Map; -import org.apache.beam.runners.flink.translation.utils.SerializedPipelineOptions; +import org.apache.beam.runners.core.construction.SerializablePipelineOptions; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.CombineFnBase; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; @@ -48,7 +48,7 @@ public class FlinkReduceFunction protected final Map, WindowingStrategy> sideInputs; - protected final SerializedPipelineOptions serializedOptions; + protected final SerializablePipelineOptions serializedOptions; public FlinkReduceFunction( CombineFnBase.GlobalCombineFn combineFn, @@ -61,7 +61,7 @@ public FlinkReduceFunction( this.windowingStrategy = windowingStrategy; this.sideInputs = sideInputs; - this.serializedOptions = new SerializedPipelineOptions(pipelineOptions); + this.serializedOptions = new SerializablePipelineOptions(pipelineOptions); } @@ -70,7 +70,7 @@ public void reduce( Iterable>> elements, Collector>> out) throws Exception { - PipelineOptions options = serializedOptions.getPipelineOptions(); + PipelineOptions options = serializedOptions.get(); FlinkSideInputReader sideInputReader = new FlinkSideInputReader(sideInputs, getRuntimeContext()); diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkStatefulDoFnFunction.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkStatefulDoFnFunction.java index 11d4fee48957a..412269ca270c3 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkStatefulDoFnFunction.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkStatefulDoFnFunction.java @@ -31,9 +31,9 @@ import org.apache.beam.runners.core.StateNamespace; import org.apache.beam.runners.core.StateNamespaces; import org.apache.beam.runners.core.TimerInternals; +import org.apache.beam.runners.core.construction.SerializablePipelineOptions; import org.apache.beam.runners.flink.FlinkPipelineOptions; import org.apache.beam.runners.flink.metrics.DoFnRunnerWithMetricsUpdate; -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.ParDo; @@ -61,7 +61,7 @@ public class FlinkStatefulDoFnFunction private String stepName; private final WindowingStrategy windowingStrategy; private final Map, WindowingStrategy> sideInputs; - private final SerializedPipelineOptions serializedOptions; + private final SerializablePipelineOptions serializedOptions; private final Map, Integer> outputMap; private final TupleTag mainOutputTag; private transient DoFnInvoker doFnInvoker; @@ -79,7 +79,7 @@ public FlinkStatefulDoFnFunction( this.stepName = stepName; this.windowingStrategy = windowingStrategy; this.sideInputs = sideInputs; - this.serializedOptions = new SerializedPipelineOptions(pipelineOptions); + this.serializedOptions = new SerializablePipelineOptions(pipelineOptions); this.outputMap = outputMap; this.mainOutputTag = mainOutputTag; } @@ -118,7 +118,7 @@ public void reduce( List> additionalOutputTags = Lists.newArrayList(outputMap.keySet()); DoFnRunner, OutputT> doFnRunner = DoFnRunners.simpleRunner( - serializedOptions.getPipelineOptions(), dofn, + serializedOptions.get(), dofn, new FlinkSideInputReader(sideInputs, runtimeContext), outputManager, mainOutputTag, @@ -135,7 +135,7 @@ public TimerInternals timerInternals() { }, windowingStrategy); - if ((serializedOptions.getPipelineOptions().as(FlinkPipelineOptions.class)) + if ((serializedOptions.get().as(FlinkPipelineOptions.class)) .getEnableMetrics()) { doFnRunner = new DoFnRunnerWithMetricsUpdate<>(stepName, doFnRunner, getRuntimeContext()); } diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/utils/SerializedPipelineOptions.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/utils/SerializedPipelineOptions.java deleted file mode 100644 index 40b6dd6155621..0000000000000 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/utils/SerializedPipelineOptions.java +++ /dev/null @@ -1,77 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.beam.runners.flink.translation.utils; - -import static com.google.common.base.Preconditions.checkNotNull; - -import com.fasterxml.jackson.databind.Module; -import com.fasterxml.jackson.databind.ObjectMapper; -import java.io.ByteArrayOutputStream; -import java.io.IOException; -import java.io.Serializable; -import org.apache.beam.sdk.io.FileSystems; -import org.apache.beam.sdk.options.PipelineOptions; -import org.apache.beam.sdk.util.common.ReflectHelpers; - -/** - * Encapsulates the PipelineOptions in serialized form to ship them to the cluster. - */ -public class SerializedPipelineOptions implements Serializable { - - private final byte[] serializedOptions; - - /** Lazily initialized copy of deserialized options. */ - private transient PipelineOptions pipelineOptions; - - public SerializedPipelineOptions(PipelineOptions options) { - checkNotNull(options, "PipelineOptions must not be null."); - - try (ByteArrayOutputStream baos = new ByteArrayOutputStream()) { - createMapper().writeValue(baos, options); - this.serializedOptions = baos.toByteArray(); - } catch (Exception e) { - throw new RuntimeException("Couldn't serialize PipelineOptions.", e); - } - - } - - public PipelineOptions getPipelineOptions() { - if (pipelineOptions == null) { - try { - pipelineOptions = createMapper().readValue(serializedOptions, PipelineOptions.class); - - FileSystems.setDefaultPipelineOptions(pipelineOptions); - } catch (IOException e) { - throw new RuntimeException("Couldn't deserialize the PipelineOptions.", e); - } - } - - return pipelineOptions; - } - - /** - * Use an {@link ObjectMapper} configured with any {@link Module}s in the class path allowing - * for user specified configuration injection into the ObjectMapper. This supports user custom - * types on {@link PipelineOptions}. - */ - private static ObjectMapper createMapper() { - return new ObjectMapper().registerModules( - ObjectMapper.findModules(ReflectHelpers.findClassLoader())); - } -} diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/SourceInputFormat.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/SourceInputFormat.java index 27e691231faeb..3f9d60101e075 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/SourceInputFormat.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/SourceInputFormat.java @@ -19,9 +19,9 @@ import java.io.IOException; import java.util.List; +import org.apache.beam.runners.core.construction.SerializablePipelineOptions; import org.apache.beam.runners.flink.metrics.FlinkMetricContainer; import org.apache.beam.runners.flink.metrics.ReaderInvocationUtil; -import org.apache.beam.runners.flink.translation.utils.SerializedPipelineOptions; import org.apache.beam.sdk.io.BoundedSource; import org.apache.beam.sdk.io.Source; import org.apache.beam.sdk.options.PipelineOptions; @@ -50,7 +50,7 @@ public class SourceInputFormat private final BoundedSource initialSource; private transient PipelineOptions options; - private final SerializedPipelineOptions serializedOptions; + private final SerializablePipelineOptions serializedOptions; private transient BoundedSource.BoundedReader reader; private boolean inputAvailable = false; @@ -61,12 +61,12 @@ public SourceInputFormat( String stepName, BoundedSource initialSource, PipelineOptions options) { this.stepName = stepName; this.initialSource = initialSource; - this.serializedOptions = new SerializedPipelineOptions(options); + this.serializedOptions = new SerializablePipelineOptions(options); } @Override public void configure(Configuration configuration) { - options = serializedOptions.getPipelineOptions(); + options = serializedOptions.get(); } @Override @@ -76,7 +76,7 @@ public void open(SourceInputSplit sourceInputSplit) throws IOException { readerInvoker = new ReaderInvocationUtil<>( stepName, - serializedOptions.getPipelineOptions(), + serializedOptions.get(), metricContainer); reader = ((BoundedSource) sourceInputSplit.getSource()).createReader(options); diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DoFnOperator.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DoFnOperator.java index 7995ea8c1f79d..62de42313b357 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DoFnOperator.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DoFnOperator.java @@ -47,10 +47,10 @@ import org.apache.beam.runners.core.StatefulDoFnRunner; import org.apache.beam.runners.core.TimerInternals; import org.apache.beam.runners.core.TimerInternals.TimerData; +import org.apache.beam.runners.core.construction.SerializablePipelineOptions; import org.apache.beam.runners.flink.FlinkPipelineOptions; import org.apache.beam.runners.flink.metrics.DoFnRunnerWithMetricsUpdate; import org.apache.beam.runners.flink.translation.types.CoderTypeSerializer; -import org.apache.beam.runners.flink.translation.utils.SerializedPipelineOptions; import org.apache.beam.runners.flink.translation.wrappers.streaming.state.FlinkBroadcastStateInternals; import org.apache.beam.runners.flink.translation.wrappers.streaming.state.FlinkKeyGroupStateInternals; import org.apache.beam.runners.flink.translation.wrappers.streaming.state.FlinkSplitStateInternals; @@ -106,7 +106,7 @@ public class DoFnOperator protected DoFn doFn; - protected final SerializedPipelineOptions serializedOptions; + protected final SerializablePipelineOptions serializedOptions; protected final TupleTag mainOutputTag; protected final List> additionalOutputTags; @@ -174,7 +174,7 @@ public DoFnOperator( this.additionalOutputTags = additionalOutputTags; this.sideInputTagMapping = sideInputTagMapping; this.sideInputs = sideInputs; - this.serializedOptions = new SerializedPipelineOptions(options); + this.serializedOptions = new SerializablePipelineOptions(options); this.windowingStrategy = windowingStrategy; this.outputManagerFactory = outputManagerFactory; @@ -256,7 +256,7 @@ public void open() throws Exception { org.apache.beam.runners.core.StepContext stepContext = createStepContext(); doFnRunner = DoFnRunners.simpleRunner( - serializedOptions.getPipelineOptions(), + serializedOptions.get(), doFn, sideInputReader, outputManager, @@ -301,7 +301,7 @@ public void open() throws Exception { stateCleaner); } - if ((serializedOptions.getPipelineOptions().as(FlinkPipelineOptions.class)) + if ((serializedOptions.get().as(FlinkPipelineOptions.class)) .getEnableMetrics()) { doFnRunner = new DoFnRunnerWithMetricsUpdate<>(stepName, doFnRunner, getRuntimeContext()); } diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/SplittableDoFnOperator.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/SplittableDoFnOperator.java index 2f095d481694e..be758a6f47643 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/SplittableDoFnOperator.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/SplittableDoFnOperator.java @@ -115,7 +115,7 @@ public TimerInternals timerInternalsForKey(String key) { ((ProcessFn) doFn).setProcessElementInvoker( new OutputAndTimeBoundedSplittableProcessElementInvoker<>( doFn, - serializedOptions.getPipelineOptions(), + serializedOptions.get(), new OutputWindowedValue() { @Override public void outputWindowedValue( diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/BoundedSourceWrapper.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/BoundedSourceWrapper.java index 6d756880f6185..5ddc46f880d82 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/BoundedSourceWrapper.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/BoundedSourceWrapper.java @@ -20,9 +20,9 @@ import com.google.common.annotations.VisibleForTesting; import java.util.ArrayList; import java.util.List; +import org.apache.beam.runners.core.construction.SerializablePipelineOptions; import org.apache.beam.runners.flink.metrics.FlinkMetricContainer; import org.apache.beam.runners.flink.metrics.ReaderInvocationUtil; -import org.apache.beam.runners.flink.translation.utils.SerializedPipelineOptions; import org.apache.beam.sdk.io.BoundedSource; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.windowing.GlobalWindow; @@ -48,7 +48,7 @@ public class BoundedSourceWrapper /** * Keep the options so that we can initialize the readers. */ - private final SerializedPipelineOptions serializedOptions; + private final SerializablePipelineOptions serializedOptions; /** * The split sources. We split them in the constructor to ensure that all parallel @@ -74,7 +74,7 @@ public BoundedSourceWrapper( BoundedSource source, int parallelism) throws Exception { this.stepName = stepName; - this.serializedOptions = new SerializedPipelineOptions(pipelineOptions); + this.serializedOptions = new SerializablePipelineOptions(pipelineOptions); long desiredBundleSize = source.getEstimatedSizeBytes(pipelineOptions) / parallelism; @@ -109,13 +109,13 @@ public void run(SourceContext> ctx) throws Exception { ReaderInvocationUtil> readerInvoker = new ReaderInvocationUtil<>( stepName, - serializedOptions.getPipelineOptions(), + serializedOptions.get(), metricContainer); readers = new ArrayList<>(); // initialize readers from scratch for (BoundedSource source : localSources) { - readers.add(source.createReader(serializedOptions.getPipelineOptions())); + readers.add(source.createReader(serializedOptions.get())); } if (readers.size() == 1) { diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/UnboundedSourceWrapper.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/UnboundedSourceWrapper.java index e75072a736d2a..817dd74d78fa2 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/UnboundedSourceWrapper.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/UnboundedSourceWrapper.java @@ -22,10 +22,10 @@ import java.util.Iterator; import java.util.LinkedHashMap; import java.util.List; +import org.apache.beam.runners.core.construction.SerializablePipelineOptions; import org.apache.beam.runners.flink.metrics.FlinkMetricContainer; import org.apache.beam.runners.flink.metrics.ReaderInvocationUtil; import org.apache.beam.runners.flink.translation.types.CoderTypeInformation; -import org.apache.beam.runners.flink.translation.utils.SerializedPipelineOptions; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.KvCoder; import org.apache.beam.sdk.coders.SerializableCoder; @@ -72,7 +72,7 @@ public class UnboundedSourceWrapper< /** * Keep the options so that we can initialize the localReaders. */ - private final SerializedPipelineOptions serializedOptions; + private final SerializablePipelineOptions serializedOptions; /** * For snapshot and restore. @@ -141,7 +141,7 @@ public UnboundedSourceWrapper( UnboundedSource source, int parallelism) throws Exception { this.stepName = stepName; - this.serializedOptions = new SerializedPipelineOptions(pipelineOptions); + this.serializedOptions = new SerializablePipelineOptions(pipelineOptions); if (source.requiresDeduping()) { LOG.warn("Source {} requires deduping but Flink runner doesn't support this yet.", source); @@ -189,7 +189,7 @@ public void open(Configuration parameters) throws Exception { stateForCheckpoint.get()) { localSplitSources.add(restored.getKey()); localReaders.add(restored.getKey().createReader( - serializedOptions.getPipelineOptions(), restored.getValue())); + serializedOptions.get(), restored.getValue())); } } else { // initialize localReaders and localSources from scratch @@ -198,7 +198,7 @@ public void open(Configuration parameters) throws Exception { UnboundedSource source = splitSources.get(i); UnboundedSource.UnboundedReader reader = - source.createReader(serializedOptions.getPipelineOptions(), null); + source.createReader(serializedOptions.get(), null); localSplitSources.add(source); localReaders.add(reader); } @@ -221,7 +221,7 @@ public void run(SourceContext>> ctx) th ReaderInvocationUtil> readerInvoker = new ReaderInvocationUtil<>( stepName, - serializedOptions.getPipelineOptions(), + serializedOptions.get(), metricContainer); if (localReaders.size() == 0) { diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/PipelineOptionsTest.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/PipelineOptionsTest.java index d0281eccbf666..eb0602658ae52 100644 --- a/runners/flink/src/test/java/org/apache/beam/runners/flink/PipelineOptionsTest.java +++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/PipelineOptionsTest.java @@ -17,32 +17,8 @@ */ package org.apache.beam.runners.flink; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertFalse; -import static org.junit.Assert.assertNotNull; -import static org.junit.Assert.assertNull; -import static org.junit.Assert.assertTrue; - -import com.fasterxml.jackson.core.JsonGenerator; -import com.fasterxml.jackson.core.JsonParser; -import com.fasterxml.jackson.core.JsonProcessingException; -import com.fasterxml.jackson.databind.DeserializationContext; -import com.fasterxml.jackson.databind.JsonDeserializer; -import com.fasterxml.jackson.databind.JsonSerializer; -import com.fasterxml.jackson.databind.Module; -import com.fasterxml.jackson.databind.SerializerProvider; -import com.fasterxml.jackson.databind.annotation.JsonDeserialize; -import com.fasterxml.jackson.databind.annotation.JsonSerialize; -import com.fasterxml.jackson.databind.module.SimpleModule; -import com.google.auto.service.AutoService; -import java.io.ByteArrayInputStream; -import java.io.ByteArrayOutputStream; -import java.io.IOException; -import java.io.ObjectInputStream; -import java.io.ObjectOutputStream; import java.util.Collections; import java.util.HashMap; -import org.apache.beam.runners.flink.translation.utils.SerializedPipelineOptions; import org.apache.beam.runners.flink.translation.wrappers.streaming.DoFnOperator; import org.apache.beam.sdk.coders.StringUtf8Coder; import org.apache.beam.sdk.options.Default; @@ -60,12 +36,10 @@ import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.typeinfo.TypeHint; import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.runtime.state.memory.MemoryStateBackend; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness; import org.joda.time.Instant; import org.junit.Assert; -import org.junit.BeforeClass; import org.junit.Test; /** @@ -73,9 +47,7 @@ */ public class PipelineOptionsTest { - /** - * Pipeline options. - */ + /** Pipeline options. */ public interface MyOptions extends FlinkPipelineOptions { @Description("Bla bla bla") @Default.String("Hello") @@ -83,60 +55,12 @@ public interface MyOptions extends FlinkPipelineOptions { void setTestOption(String value); } - private static MyOptions options; - private static SerializedPipelineOptions serializedOptions; - - private static final String[] args = new String[]{"--testOption=nothing"}; - - @BeforeClass - public static void beforeTest() { - options = PipelineOptionsFactory.fromArgs(args).as(MyOptions.class); - serializedOptions = new SerializedPipelineOptions(options); - } - - @Test - public void testDeserialization() { - MyOptions deserializedOptions = serializedOptions.getPipelineOptions().as(MyOptions.class); - assertEquals("nothing", deserializedOptions.getTestOption()); - } - - @Test - public void testIgnoredFieldSerialization() { - FlinkPipelineOptions options = PipelineOptionsFactory.as(FlinkPipelineOptions.class); - options.setStateBackend(new MemoryStateBackend()); - - FlinkPipelineOptions deserialized = - new SerializedPipelineOptions(options).getPipelineOptions().as(FlinkPipelineOptions.class); - - assertNull(deserialized.getStateBackend()); - } - - @Test - public void testEnableMetrics() { - FlinkPipelineOptions options = PipelineOptionsFactory.as(FlinkPipelineOptions.class); - options.setEnableMetrics(false); - assertFalse(options.getEnableMetrics()); - } - - @Test - public void testCaching() { - PipelineOptions deserializedOptions = - serializedOptions.getPipelineOptions().as(PipelineOptions.class); - - assertNotNull(deserializedOptions); - assertTrue(deserializedOptions == serializedOptions.getPipelineOptions()); - assertTrue(deserializedOptions == serializedOptions.getPipelineOptions()); - assertTrue(deserializedOptions == serializedOptions.getPipelineOptions()); - } - - @Test(expected = Exception.class) - public void testNonNull() { - new SerializedPipelineOptions(null); - } + private static MyOptions options = + PipelineOptionsFactory.fromArgs("--testOption=nothing").as(MyOptions.class); @Test(expected = Exception.class) public void parDoBaseClassPipelineOptionsNullTest() { - DoFnOperator doFnOperator = new DoFnOperator<>( + new DoFnOperator<>( new TestDoFn(), "stepName", WindowedValue.getValueOnlyCoder(StringUtf8Coder.of()), @@ -196,18 +120,7 @@ public void parDoBaseClassPipelineOptionsSerializationTest() throws Exception { } - @Test - public void testExternalizedCheckpointsConfigs() { - String[] args = new String[] { "--externalizedCheckpointsEnabled=true", - "--retainExternalizedCheckpointsOnCancellation=false" }; - final FlinkPipelineOptions options = PipelineOptionsFactory.fromArgs(args) - .as(FlinkPipelineOptions.class); - assertEquals(options.isExternalizedCheckpointsEnabled(), true); - assertEquals(options.getRetainExternalizedCheckpointsOnCancellation(), false); - } - private static class TestDoFn extends DoFn { - @ProcessElement public void processElement(ProcessContext c) throws Exception { Assert.assertNotNull(c.getPipelineOptions()); @@ -216,74 +129,4 @@ public void processElement(ProcessContext c) throws Exception { c.getPipelineOptions().as(MyOptions.class).getTestOption()); } } - - /** PipelineOptions used to test auto registration of Jackson modules. */ - public interface JacksonIncompatibleOptions extends PipelineOptions { - JacksonIncompatible getJacksonIncompatible(); - void setJacksonIncompatible(JacksonIncompatible value); - } - - /** A Jackson {@link Module} to test auto-registration of modules. */ - @AutoService(Module.class) - public static class RegisteredTestModule extends SimpleModule { - public RegisteredTestModule() { - super("RegisteredTestModule"); - setMixInAnnotation(JacksonIncompatible.class, JacksonIncompatibleMixin.class); - } - } - - /** A class which Jackson does not know how to serialize/deserialize. */ - public static class JacksonIncompatible { - private final String value; - public JacksonIncompatible(String value) { - this.value = value; - } - } - - /** A Jackson mixin used to add annotations to other classes. */ - @JsonDeserialize(using = JacksonIncompatibleDeserializer.class) - @JsonSerialize(using = JacksonIncompatibleSerializer.class) - public static final class JacksonIncompatibleMixin {} - - /** A Jackson deserializer for {@link JacksonIncompatible}. */ - public static class JacksonIncompatibleDeserializer extends - JsonDeserializer { - - @Override - public JacksonIncompatible deserialize(JsonParser jsonParser, - DeserializationContext deserializationContext) throws IOException, JsonProcessingException { - return new JacksonIncompatible(jsonParser.readValueAs(String.class)); - } - } - - /** A Jackson serializer for {@link JacksonIncompatible}. */ - public static class JacksonIncompatibleSerializer extends JsonSerializer { - - @Override - public void serialize(JacksonIncompatible jacksonIncompatible, JsonGenerator jsonGenerator, - SerializerProvider serializerProvider) throws IOException, JsonProcessingException { - jsonGenerator.writeString(jacksonIncompatible.value); - } - } - - @Test - public void testSerializingPipelineOptionsWithCustomUserType() throws Exception { - String expectedValue = "testValue"; - PipelineOptions options = PipelineOptionsFactory - .fromArgs("--jacksonIncompatible=\"" + expectedValue + "\"") - .as(JacksonIncompatibleOptions.class); - SerializedPipelineOptions context = new SerializedPipelineOptions(options); - - ByteArrayOutputStream baos = new ByteArrayOutputStream(); - try (ObjectOutputStream outputStream = new ObjectOutputStream(baos)) { - outputStream.writeObject(context); - } - try (ObjectInputStream inputStream = - new ObjectInputStream(new ByteArrayInputStream(baos.toByteArray()))) { - SerializedPipelineOptions copy = (SerializedPipelineOptions) inputStream.readObject(); - assertEquals(expectedValue, - copy.getPipelineOptions().as(JacksonIncompatibleOptions.class) - .getJacksonIncompatible().value); - } - } } diff --git a/runners/spark/pom.xml b/runners/spark/pom.xml index e823060119f08..b2e7fe48ab63e 100644 --- a/runners/spark/pom.xml +++ b/runners/spark/pom.xml @@ -35,7 +35,6 @@ UTF-8 UTF-8 0.9.0.1 - 2.4.4 3.1.2 @@ -182,20 +181,9 @@ auto-service true - - com.fasterxml.jackson.core - jackson-core - ${jackson.version} - com.fasterxml.jackson.core jackson-annotations - ${jackson.version} - - - com.fasterxml.jackson.core - jackson-databind - ${jackson.version} org.apache.avro diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/aggregators/NamedAggregators.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/aggregators/NamedAggregators.java index 27f2ec8897c13..a9f2c4456c0cb 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/aggregators/NamedAggregators.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/aggregators/NamedAggregators.java @@ -19,18 +19,11 @@ package org.apache.beam.runners.spark.aggregators; import com.google.common.base.Function; -import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Maps; -import java.io.IOException; -import java.io.ObjectInputStream; -import java.io.ObjectOutputStream; import java.io.Serializable; import java.util.Map; import java.util.TreeMap; -import org.apache.beam.runners.spark.translation.SparkRuntimeContext; -import org.apache.beam.sdk.coders.CannotProvideCoderException; -import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.transforms.Combine; /** @@ -51,17 +44,6 @@ public class NamedAggregators implements Serializable { public NamedAggregators() { } - /** - * Constructs a new named aggregators instance that contains a mapping from the specified - * `named` to the associated initial state. - * - * @param name Name of aggregator. - * @param state Associated State. - */ - public NamedAggregators(String name, State state) { - this.mNamedAggregators.put(name, state); - } - /** * @param name Name of aggregator to retrieve. * @param typeClass Type class to cast the value to. @@ -152,79 +134,4 @@ public interface State extends Serializable { Combine.CombineFn getCombineFn(); } - /** - * @param Input data type - * @param Intermediate data type (useful for averages) - * @param Output data type - */ - public static class CombineFunctionState - implements State { - - private Combine.CombineFn combineFn; - private Coder inCoder; - private SparkRuntimeContext ctxt; - private transient InterT state; - - public CombineFunctionState( - Combine.CombineFn combineFn, - Coder inCoder, - SparkRuntimeContext ctxt) { - this.combineFn = combineFn; - this.inCoder = inCoder; - this.ctxt = ctxt; - this.state = combineFn.createAccumulator(); - } - - @Override - public void update(InputT element) { - combineFn.addInput(state, element); - } - - @Override - public State merge(State other) { - this.state = combineFn.mergeAccumulators(ImmutableList.of(current(), other.current())); - return this; - } - - @Override - public InterT current() { - return state; - } - - @Override - public OutputT render() { - return combineFn.extractOutput(state); - } - - @Override - public Combine.CombineFn getCombineFn() { - return combineFn; - } - - private void writeObject(ObjectOutputStream oos) throws IOException { - oos.writeObject(ctxt); - oos.writeObject(combineFn); - oos.writeObject(inCoder); - try { - combineFn.getAccumulatorCoder(ctxt.getCoderRegistry(), inCoder) - .encode(state, oos); - } catch (CannotProvideCoderException e) { - throw new IllegalStateException("Could not determine coder for accumulator", e); - } - } - - @SuppressWarnings("unchecked") - private void readObject(ObjectInputStream ois) throws IOException, ClassNotFoundException { - ctxt = (SparkRuntimeContext) ois.readObject(); - combineFn = (Combine.CombineFn) ois.readObject(); - inCoder = (Coder) ois.readObject(); - try { - state = combineFn.getAccumulatorCoder(ctxt.getCoderRegistry(), inCoder) - .decode(ois); - } catch (CannotProvideCoderException e) { - throw new IllegalStateException("Could not determine coder for accumulator", e); - } - } - } - } 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 20aca5f10fdca..b7000b42bf8f3 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 @@ -20,8 +20,8 @@ import static com.google.common.base.Preconditions.checkArgument; +import org.apache.beam.runners.core.construction.SerializablePipelineOptions; import org.apache.beam.runners.spark.SparkPipelineOptions; -import org.apache.beam.runners.spark.translation.SparkRuntimeContext; import org.apache.beam.sdk.io.Source; import org.apache.beam.sdk.io.UnboundedSource; import org.apache.spark.api.java.JavaSparkContext$; @@ -58,7 +58,7 @@ class SourceDStream private static final Logger LOG = LoggerFactory.getLogger(SourceDStream.class); private final UnboundedSource unboundedSource; - private final SparkRuntimeContext runtimeContext; + private final SerializablePipelineOptions options; private final Duration boundReadDuration; // Reader cache interval to expire readers if they haven't been accessed in the last microbatch. // The reason we expire readers is that upon executor death/addition source split ownership can be @@ -81,20 +81,20 @@ class SourceDStream SourceDStream( StreamingContext ssc, UnboundedSource unboundedSource, - SparkRuntimeContext runtimeContext, + SerializablePipelineOptions options, Long boundMaxRecords) { super(ssc, JavaSparkContext$.MODULE$., CheckpointMarkT>>fakeClassTag()); this.unboundedSource = unboundedSource; - this.runtimeContext = runtimeContext; + this.options = options; - SparkPipelineOptions options = runtimeContext.getPipelineOptions().as( + SparkPipelineOptions sparkOptions = options.get().as( SparkPipelineOptions.class); // Reader cache expiration interval. 50% of batch interval is added to accommodate latency. - this.readerCacheInterval = 1.5 * options.getBatchIntervalMillis(); + this.readerCacheInterval = 1.5 * sparkOptions.getBatchIntervalMillis(); - this.boundReadDuration = boundReadDuration(options.getReadTimePercentage(), - options.getMinReadTimeMillis()); + this.boundReadDuration = boundReadDuration(sparkOptions.getReadTimePercentage(), + sparkOptions.getMinReadTimeMillis()); // set initial parallelism once. this.initialParallelism = ssc().sparkContext().defaultParallelism(); checkArgument(this.initialParallelism > 0, "Number of partitions must be greater than zero."); @@ -104,7 +104,7 @@ class SourceDStream try { this.numPartitions = createMicrobatchSource() - .split(options) + .split(sparkOptions) .size(); } catch (Exception e) { throw new RuntimeException(e); @@ -116,7 +116,7 @@ public scala.Option, CheckpointMarkT>>> compute(Time validT RDD, CheckpointMarkT>> rdd = new SourceRDD.Unbounded<>( ssc().sparkContext(), - runtimeContext, + options, createMicrobatchSource(), numPartitions); return scala.Option.apply(rdd); diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/io/SourceRDD.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/io/SourceRDD.java index 01cc1762413f0..a225e0f3c510e 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/io/SourceRDD.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/io/SourceRDD.java @@ -28,9 +28,9 @@ import java.util.Iterator; import java.util.List; import java.util.NoSuchElementException; +import org.apache.beam.runners.core.construction.SerializablePipelineOptions; import org.apache.beam.runners.core.metrics.MetricsContainerStepMap; import org.apache.beam.runners.spark.metrics.MetricsAccumulator; -import org.apache.beam.runners.spark.translation.SparkRuntimeContext; import org.apache.beam.sdk.io.BoundedSource; import org.apache.beam.sdk.io.Source; import org.apache.beam.sdk.io.UnboundedSource; @@ -66,7 +66,7 @@ public static class Bounded extends RDD> { private static final Logger LOG = LoggerFactory.getLogger(SourceRDD.Bounded.class); private final BoundedSource source; - private final SparkRuntimeContext runtimeContext; + private final SerializablePipelineOptions options; private final int numPartitions; private final String stepName; private final Accumulator metricsAccum; @@ -79,11 +79,11 @@ public static class Bounded extends RDD> { public Bounded( SparkContext sc, BoundedSource source, - SparkRuntimeContext runtimeContext, + SerializablePipelineOptions options, String stepName) { super(sc, NIL, JavaSparkContext$.MODULE$.>fakeClassTag()); this.source = source; - this.runtimeContext = runtimeContext; + this.options = options; // the input parallelism is determined by Spark's scheduler backend. // when running on YARN/SparkDeploy it's the result of max(totalCores, 2). // when running on Mesos it's 8. @@ -103,14 +103,14 @@ public Partition[] getPartitions() { long desiredSizeBytes = DEFAULT_BUNDLE_SIZE; try { desiredSizeBytes = source.getEstimatedSizeBytes( - runtimeContext.getPipelineOptions()) / numPartitions; + options.get()) / numPartitions; } catch (Exception e) { LOG.warn("Failed to get estimated bundle size for source {}, using default bundle " + "size of {} bytes.", source, DEFAULT_BUNDLE_SIZE); } try { List> partitionedSources = source.split(desiredSizeBytes, - runtimeContext.getPipelineOptions()); + options.get()); Partition[] partitions = new SourcePartition[partitionedSources.size()]; for (int i = 0; i < partitionedSources.size(); i++) { partitions[i] = new SourcePartition<>(id(), i, partitionedSources.get(i)); @@ -125,7 +125,7 @@ public Partition[] getPartitions() { private BoundedSource.BoundedReader createReader(SourcePartition partition) { try { return ((BoundedSource) partition.source).createReader( - runtimeContext.getPipelineOptions()); + options.get()); } catch (IOException e) { throw new RuntimeException("Failed to create reader from a BoundedSource.", e); } @@ -293,7 +293,7 @@ public static class Unbounded extends RDD, CheckpointMarkT>> { private final MicrobatchSource microbatchSource; - private final SparkRuntimeContext runtimeContext; + private final SerializablePipelineOptions options; private final Partitioner partitioner; // to satisfy Scala API. @@ -302,12 +302,12 @@ public static class Unbounded>emptyList()).toList(); public Unbounded(SparkContext sc, - SparkRuntimeContext runtimeContext, + SerializablePipelineOptions options, MicrobatchSource microbatchSource, int initialNumPartitions) { super(sc, NIL, JavaSparkContext$.MODULE$., CheckpointMarkT>>fakeClassTag()); - this.runtimeContext = runtimeContext; + this.options = options; this.microbatchSource = microbatchSource; this.partitioner = new HashPartitioner(initialNumPartitions); } @@ -316,7 +316,7 @@ public Unbounded(SparkContext sc, public Partition[] getPartitions() { try { final List> partitionedSources = - microbatchSource.split(runtimeContext.getPipelineOptions()); + microbatchSource.split(options.get()); final Partition[] partitions = new CheckpointableSourcePartition[partitionedSources.size()]; for (int i = 0; i < partitionedSources.size(); i++) { partitions[i] = 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 7106c73866a14..b31aa9f6e59ba 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 @@ -22,12 +22,12 @@ import java.io.IOException; import java.io.Serializable; import java.util.Collections; +import org.apache.beam.runners.core.construction.SerializablePipelineOptions; import org.apache.beam.runners.core.metrics.MetricsContainerStepMap; import org.apache.beam.runners.spark.SparkPipelineOptions; import org.apache.beam.runners.spark.coders.CoderHelpers; import org.apache.beam.runners.spark.metrics.MetricsAccumulator; import org.apache.beam.runners.spark.stateful.StateSpecFunctions; -import org.apache.beam.runners.spark.translation.SparkRuntimeContext; import org.apache.beam.runners.spark.translation.streaming.UnboundedDataset; import org.apache.beam.runners.spark.util.GlobalWatermarkHolder; import org.apache.beam.runners.spark.util.GlobalWatermarkHolder.SparkWatermarks; @@ -80,11 +80,11 @@ public class SparkUnboundedSource { public static UnboundedDataset read( JavaStreamingContext jssc, - SparkRuntimeContext rc, + SerializablePipelineOptions rc, UnboundedSource source, String stepName) { - SparkPipelineOptions options = rc.getPipelineOptions().as(SparkPipelineOptions.class); + SparkPipelineOptions options = rc.get().as(SparkPipelineOptions.class); Long maxRecordsPerBatch = options.getMaxRecordsPerBatch(); SourceDStream sourceDStream = new SourceDStream<>(jssc.ssc(), source, rc, maxRecordsPerBatch); diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/stateful/SparkGroupAlsoByWindowViaWindowSet.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/stateful/SparkGroupAlsoByWindowViaWindowSet.java index 1385e071978f7..1263618aa2bfd 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/stateful/SparkGroupAlsoByWindowViaWindowSet.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/stateful/SparkGroupAlsoByWindowViaWindowSet.java @@ -31,6 +31,7 @@ import org.apache.beam.runners.core.SystemReduceFn; import org.apache.beam.runners.core.TimerInternals; import org.apache.beam.runners.core.UnsupportedSideInputReader; +import org.apache.beam.runners.core.construction.SerializablePipelineOptions; import org.apache.beam.runners.core.construction.TriggerTranslation; import org.apache.beam.runners.core.metrics.CounterCell; import org.apache.beam.runners.core.metrics.MetricsContainerImpl; @@ -38,7 +39,6 @@ import org.apache.beam.runners.core.triggers.TriggerStateMachines; import org.apache.beam.runners.spark.SparkPipelineOptions; import org.apache.beam.runners.spark.coders.CoderHelpers; -import org.apache.beam.runners.spark.translation.SparkRuntimeContext; import org.apache.beam.runners.spark.translation.TranslationUtils; import org.apache.beam.runners.spark.translation.WindowingHelpers; import org.apache.beam.runners.spark.util.ByteArray; @@ -108,11 +108,11 @@ JavaDStream>>> groupAlsoByWindow( final Coder keyCoder, final Coder> wvCoder, final WindowingStrategy windowingStrategy, - final SparkRuntimeContext runtimeContext, + final SerializablePipelineOptions options, final List sourceIds) { final long batchDurationMillis = - runtimeContext.getPipelineOptions().as(SparkPipelineOptions.class).getBatchIntervalMillis(); + options.get().as(SparkPipelineOptions.class).getBatchIntervalMillis(); final IterableCoder> itrWvCoder = IterableCoder.of(wvCoder); final Coder iCoder = ((FullWindowedValueCoder) wvCoder).getValueCoder(); final Coder wCoder = @@ -123,7 +123,7 @@ JavaDStream>>> groupAlsoByWindow( TimerInternals.TimerDataCoder.of(windowingStrategy.getWindowFn().windowCoder()); long checkpointDurationMillis = - runtimeContext.getPipelineOptions().as(SparkPipelineOptions.class) + options.get().as(SparkPipelineOptions.class) .getCheckpointDurationMillis(); // we have to switch to Scala API to avoid Optional in the Java API, see: SPARK-4819. @@ -268,7 +268,7 @@ public JavaPairRDD call( outputHolder, new UnsupportedSideInputReader("GroupAlsoByWindow"), reduceFn, - runtimeContext.getPipelineOptions()); + options.get()); outputHolder.clear(); // clear before potential use. if (!seq.isEmpty()) { 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 549bd30a11aa6..1b54478703664 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 @@ -27,12 +27,12 @@ import java.util.ArrayList; import java.util.List; import java.util.concurrent.TimeUnit; +import org.apache.beam.runners.core.construction.SerializablePipelineOptions; import org.apache.beam.runners.core.metrics.MetricsContainerStepMap; import org.apache.beam.runners.spark.coders.CoderHelpers; import org.apache.beam.runners.spark.io.EmptyCheckpointMark; import org.apache.beam.runners.spark.io.MicrobatchSource; import org.apache.beam.runners.spark.io.SparkUnboundedSource.Metadata; -import org.apache.beam.runners.spark.translation.SparkRuntimeContext; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.io.Source; import org.apache.beam.sdk.io.UnboundedSource; @@ -91,7 +91,7 @@ private abstract static class SerializableFunction3 * *

              See also SPARK-4819.

              * - * @param runtimeContext A serializable {@link SparkRuntimeContext}. + * @param options A serializable {@link SerializablePipelineOptions}. * @param The type of the input stream elements. * @param The type of the {@link UnboundedSource.CheckpointMark}. * @return The appropriate {@link org.apache.spark.streaming.StateSpec} function. @@ -99,7 +99,7 @@ private abstract static class SerializableFunction3 public static scala.Function3, scala.Option, State>, Tuple2, Metadata>> mapSourceFunction( - final SparkRuntimeContext runtimeContext, final String stepName) { + final SerializablePipelineOptions options, final String stepName) { return new SerializableFunction3, Option, State>, Tuple2, Metadata>>() { @@ -151,7 +151,7 @@ public Tuple2, Metadata> apply( try { microbatchReader = (MicrobatchSource.Reader) - microbatchSource.getOrCreateReader(runtimeContext.getPipelineOptions(), + microbatchSource.getOrCreateReader(options.get(), checkpointMark); } catch (IOException e) { throw new RuntimeException(e); 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 23e430a671440..463e50706a993 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,6 +26,7 @@ import java.util.LinkedHashSet; import java.util.Map; import java.util.Set; +import org.apache.beam.runners.core.construction.SerializablePipelineOptions; import org.apache.beam.runners.core.construction.TransformInputs; import org.apache.beam.runners.spark.SparkPipelineOptions; import org.apache.beam.runners.spark.coders.CoderHelpers; @@ -50,7 +51,6 @@ public class EvaluationContext { private final JavaSparkContext jsc; private JavaStreamingContext jssc; - private final SparkRuntimeContext runtime; private final Pipeline pipeline; private final Map datasets = new LinkedHashMap<>(); private final Map pcollections = new LinkedHashMap<>(); @@ -60,12 +60,13 @@ public class EvaluationContext { private final SparkPCollectionView pviews = new SparkPCollectionView(); private final Map cacheCandidates = new HashMap<>(); private final PipelineOptions options; + private final SerializablePipelineOptions serializableOptions; public EvaluationContext(JavaSparkContext jsc, Pipeline pipeline, PipelineOptions options) { this.jsc = jsc; this.pipeline = pipeline; this.options = options; - this.runtime = new SparkRuntimeContext(options); + this.serializableOptions = new SerializablePipelineOptions(options); } public EvaluationContext( @@ -90,8 +91,8 @@ public PipelineOptions getOptions() { return options; } - public SparkRuntimeContext getRuntimeContext() { - return runtime; + public SerializablePipelineOptions getSerializableOptions() { + return serializableOptions; } public void setCurrentTransform(AppliedPTransform transform) { @@ -254,7 +255,7 @@ Iterable> getWindowedValues(PCollection pcollection) { } private String storageLevel() { - return runtime.getPipelineOptions().as(SparkPipelineOptions.class).getStorageLevel(); + return serializableOptions.get().as(SparkPipelineOptions.class).getStorageLevel(); } } diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/MultiDoFnFunction.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/MultiDoFnFunction.java index 23d5b32d4e7c7..72995833982ff 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/MultiDoFnFunction.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/MultiDoFnFunction.java @@ -34,8 +34,8 @@ import org.apache.beam.runners.core.StateInternals; import org.apache.beam.runners.core.StepContext; import org.apache.beam.runners.core.TimerInternals; +import org.apache.beam.runners.core.construction.SerializablePipelineOptions; import org.apache.beam.runners.core.metrics.MetricsContainerStepMap; -import org.apache.beam.runners.spark.aggregators.NamedAggregators; import org.apache.beam.runners.spark.util.SideInputBroadcast; import org.apache.beam.runners.spark.util.SparkSideInputReader; import org.apache.beam.sdk.transforms.DoFn; @@ -59,11 +59,10 @@ public class MultiDoFnFunction implements PairFlatMapFunction>, TupleTag, WindowedValue> { - private final Accumulator aggAccum; private final Accumulator metricsAccum; private final String stepName; private final DoFn doFn; - private final SparkRuntimeContext runtimeContext; + private final SerializablePipelineOptions options; private final TupleTag mainOutputTag; private final List> additionalOutputTags; private final Map, KV, SideInputBroadcast>> sideInputs; @@ -71,10 +70,9 @@ public class MultiDoFnFunction private final boolean stateful; /** - * @param aggAccum The Spark {@link Accumulator} that backs the Beam Aggregators. * @param metricsAccum The Spark {@link Accumulator} that backs the Beam metrics. * @param doFn The {@link DoFn} to be wrapped. - * @param runtimeContext The {@link SparkRuntimeContext}. + * @param options The {@link SerializablePipelineOptions}. * @param mainOutputTag The main output {@link TupleTag}. * @param additionalOutputTags Additional {@link TupleTag output tags}. * @param sideInputs Side inputs used in this {@link DoFn}. @@ -82,21 +80,19 @@ public class MultiDoFnFunction * @param stateful Stateful {@link DoFn}. */ public MultiDoFnFunction( - Accumulator aggAccum, Accumulator metricsAccum, String stepName, DoFn doFn, - SparkRuntimeContext runtimeContext, + SerializablePipelineOptions options, TupleTag mainOutputTag, List> additionalOutputTags, Map, KV, SideInputBroadcast>> sideInputs, WindowingStrategy windowingStrategy, boolean stateful) { - this.aggAccum = aggAccum; this.metricsAccum = metricsAccum; this.stepName = stepName; this.doFn = doFn; - this.runtimeContext = runtimeContext; + this.options = options; this.mainOutputTag = mainOutputTag; this.additionalOutputTags = additionalOutputTags; this.sideInputs = sideInputs; @@ -140,7 +136,7 @@ public TimerInternals timerInternals() { final DoFnRunner doFnRunner = DoFnRunners.simpleRunner( - runtimeContext.getPipelineOptions(), + options.get(), doFn, new SparkSideInputReader(sideInputs), outputManager, diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkAbstractCombineFn.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkAbstractCombineFn.java index 315f7fb18ddd8..d8d71ff4d4ec2 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkAbstractCombineFn.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkAbstractCombineFn.java @@ -30,6 +30,7 @@ import java.util.List; import java.util.Map; import org.apache.beam.runners.core.SideInputReader; +import org.apache.beam.runners.core.construction.SerializablePipelineOptions; import org.apache.beam.runners.spark.util.SideInputBroadcast; import org.apache.beam.runners.spark.util.SparkSideInputReader; import org.apache.beam.sdk.options.PipelineOptions; @@ -48,16 +49,16 @@ * {@link org.apache.beam.sdk.transforms.Combine.CombineFn}. */ public class SparkAbstractCombineFn implements Serializable { - protected final SparkRuntimeContext runtimeContext; + protected final SerializablePipelineOptions options; protected final Map, KV, SideInputBroadcast>> sideInputs; protected final WindowingStrategy windowingStrategy; public SparkAbstractCombineFn( - SparkRuntimeContext runtimeContext, + SerializablePipelineOptions options, Map, KV, SideInputBroadcast>> sideInputs, WindowingStrategy windowingStrategy) { - this.runtimeContext = runtimeContext; + this.options = options; this.sideInputs = sideInputs; this.windowingStrategy = (WindowingStrategy) windowingStrategy; } @@ -71,7 +72,7 @@ public SparkAbstractCombineFn( private transient SparkCombineContext combineContext; protected SparkCombineContext ctxtForInput(WindowedValue input) { if (combineContext == null) { - combineContext = new SparkCombineContext(runtimeContext.getPipelineOptions(), + combineContext = new SparkCombineContext(options.get(), new SparkSideInputReader(sideInputs)); } return combineContext.forInput(input); diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkGlobalCombineFn.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkGlobalCombineFn.java index d0e90387eb403..81416a3f706b6 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkGlobalCombineFn.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkGlobalCombineFn.java @@ -25,6 +25,7 @@ import java.util.List; import java.util.Map; import javax.annotation.Nullable; +import org.apache.beam.runners.core.construction.SerializablePipelineOptions; import org.apache.beam.runners.spark.util.SideInputBroadcast; import org.apache.beam.sdk.transforms.CombineWithContext; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; @@ -49,10 +50,10 @@ public class SparkGlobalCombineFn extends SparkAbstract public SparkGlobalCombineFn( CombineWithContext.CombineFnWithContext combineFn, - SparkRuntimeContext runtimeContext, + SerializablePipelineOptions options, Map, KV, SideInputBroadcast>> sideInputs, WindowingStrategy windowingStrategy) { - super(runtimeContext, sideInputs, windowingStrategy); + super(options, sideInputs, windowingStrategy); this.combineFn = combineFn; } diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkGroupAlsoByWindowViaOutputBufferFn.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkGroupAlsoByWindowViaOutputBufferFn.java index d2a34244e6e58..fcf438c96fd59 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkGroupAlsoByWindowViaOutputBufferFn.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkGroupAlsoByWindowViaOutputBufferFn.java @@ -30,6 +30,7 @@ import org.apache.beam.runners.core.SystemReduceFn; import org.apache.beam.runners.core.TimerInternals; import org.apache.beam.runners.core.UnsupportedSideInputReader; +import org.apache.beam.runners.core.construction.SerializablePipelineOptions; import org.apache.beam.runners.core.construction.TriggerTranslation; import org.apache.beam.runners.core.triggers.ExecutableTriggerStateMachine; import org.apache.beam.runners.core.triggers.TriggerStateMachines; @@ -55,18 +56,18 @@ public class SparkGroupAlsoByWindowViaOutputBufferFn windowingStrategy; private final StateInternalsFactory stateInternalsFactory; private final SystemReduceFn, Iterable, W> reduceFn; - private final SparkRuntimeContext runtimeContext; + private final SerializablePipelineOptions options; public SparkGroupAlsoByWindowViaOutputBufferFn( WindowingStrategy windowingStrategy, StateInternalsFactory stateInternalsFactory, SystemReduceFn, Iterable, W> reduceFn, - SparkRuntimeContext runtimeContext, + SerializablePipelineOptions options, Accumulator accumulator) { this.windowingStrategy = windowingStrategy; this.stateInternalsFactory = stateInternalsFactory; this.reduceFn = reduceFn; - this.runtimeContext = runtimeContext; + this.options = options; } @Override @@ -98,7 +99,7 @@ public Iterable>>> call( outputter, new UnsupportedSideInputReader("GroupAlsoByWindow"), reduceFn, - runtimeContext.getPipelineOptions()); + options.get()); // Process the grouped values. reduceFnRunner.processElements(values); diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkKeyedCombineFn.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkKeyedCombineFn.java index 7ac8e7d4904e6..55392e9c1e4d3 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkKeyedCombineFn.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkKeyedCombineFn.java @@ -25,6 +25,7 @@ import java.util.List; import java.util.Map; import javax.annotation.Nullable; +import org.apache.beam.runners.core.construction.SerializablePipelineOptions; import org.apache.beam.runners.spark.util.SideInputBroadcast; import org.apache.beam.sdk.transforms.CombineWithContext; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; @@ -49,10 +50,10 @@ public class SparkKeyedCombineFn extends SparkAbstra public SparkKeyedCombineFn( CombineWithContext.CombineFnWithContext combineFn, - SparkRuntimeContext runtimeContext, + SerializablePipelineOptions options, Map, KV, SideInputBroadcast>> sideInputs, WindowingStrategy windowingStrategy) { - super(runtimeContext, sideInputs, windowingStrategy); + super(options, sideInputs, windowingStrategy); this.combineFn = combineFn; } 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 deleted file mode 100644 index 6361bb2cc0fad..0000000000000 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkRuntimeContext.java +++ /dev/null @@ -1,90 +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 com.fasterxml.jackson.core.JsonProcessingException; -import com.fasterxml.jackson.databind.Module; -import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.common.base.Function; -import com.google.common.base.Supplier; -import com.google.common.base.Suppliers; -import java.io.IOException; -import java.io.Serializable; -import org.apache.beam.sdk.coders.CoderRegistry; -import org.apache.beam.sdk.options.PipelineOptions; -import org.apache.beam.sdk.util.common.ReflectHelpers; - -/** - * The SparkRuntimeContext allows us to define useful features on the client side before our - * data flow program is launched. - */ -public class SparkRuntimeContext implements Serializable { - private final Supplier optionsSupplier; - private transient CoderRegistry coderRegistry; - - SparkRuntimeContext(PipelineOptions options) { - String serializedPipelineOptions = serializePipelineOptions(options); - this.optionsSupplier = - Suppliers.memoize( - Suppliers.compose( - new DeserializeOptions(), - Suppliers.ofInstance(serializedPipelineOptions))); - } - - /** - * Use an {@link ObjectMapper} configured with any {@link Module}s in the class path allowing - * for user specified configuration injection into the ObjectMapper. This supports user custom - * types on {@link PipelineOptions}. - */ - private static ObjectMapper createMapper() { - return new ObjectMapper().registerModules( - ObjectMapper.findModules(ReflectHelpers.findClassLoader())); - } - - private String serializePipelineOptions(PipelineOptions pipelineOptions) { - try { - return createMapper().writeValueAsString(pipelineOptions); - } catch (JsonProcessingException e) { - throw new IllegalStateException("Failed to serialize the pipeline options.", e); - } - } - - public PipelineOptions getPipelineOptions() { - return optionsSupplier.get(); - } - - public CoderRegistry getCoderRegistry() { - if (coderRegistry == null) { - coderRegistry = CoderRegistry.createDefault(); - } - return coderRegistry; - } - - private static class DeserializeOptions - implements Function, Serializable { - @Override - public PipelineOptions apply(String options) { - try { - return createMapper().readValue(options, PipelineOptions.class); - } catch (IOException e) { - throw new IllegalStateException("Failed to deserialize the pipeline options.", e); - } - } - } -} 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 ac5e0cd8e9f74..e060e1d7897b3 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 @@ -146,7 +146,7 @@ public void evaluate(GroupByKey transform, EvaluationContext context) { windowingStrategy, new TranslationUtils.InMemoryStateInternalsFactory(), SystemReduceFn.buffering(coder.getValueCoder()), - context.getRuntimeContext(), + context.getSerializableOptions(), accum)); context.putDataset(transform, new BoundedDataset<>(groupedAlsoByWindow)); @@ -171,7 +171,7 @@ public void evaluate( (CombineWithContext.CombineFnWithContext) CombineFnUtil.toFnWithContext(transform.getFn()); final SparkKeyedCombineFn sparkCombineFn = - new SparkKeyedCombineFn<>(combineFn, context.getRuntimeContext(), + new SparkKeyedCombineFn<>(combineFn, context.getSerializableOptions(), TranslationUtils.getSideInputs(transform.getSideInputs(), context), context.getInput(transform).getWindowingStrategy()); @@ -222,18 +222,18 @@ public void evaluate( final WindowedValue.FullWindowedValueCoder wvoCoder = WindowedValue.FullWindowedValueCoder.of(oCoder, windowingStrategy.getWindowFn().windowCoder()); - final SparkRuntimeContext runtimeContext = context.getRuntimeContext(); final boolean hasDefault = transform.isInsertDefault(); final SparkGlobalCombineFn sparkCombineFn = new SparkGlobalCombineFn<>( combineFn, - runtimeContext, + context.getSerializableOptions(), TranslationUtils.getSideInputs(transform.getSideInputs(), context), windowingStrategy); final Coder aCoder; try { - aCoder = combineFn.getAccumulatorCoder(runtimeContext.getCoderRegistry(), iCoder); + aCoder = combineFn.getAccumulatorCoder( + context.getPipeline().getCoderRegistry(), iCoder); } catch (CannotProvideCoderException e) { throw new IllegalStateException("Could not determine coder for accumulator", e); } @@ -295,16 +295,16 @@ public void evaluate( (CombineWithContext.CombineFnWithContext) CombineFnUtil.toFnWithContext(transform.getFn()); final WindowingStrategy windowingStrategy = input.getWindowingStrategy(); - final SparkRuntimeContext runtimeContext = context.getRuntimeContext(); final Map, KV, SideInputBroadcast>> sideInputs = TranslationUtils.getSideInputs(transform.getSideInputs(), context); final SparkKeyedCombineFn sparkCombineFn = - new SparkKeyedCombineFn<>(combineFn, runtimeContext, sideInputs, windowingStrategy); + new SparkKeyedCombineFn<>( + combineFn, context.getSerializableOptions(), sideInputs, windowingStrategy); final Coder vaCoder; try { vaCoder = combineFn.getAccumulatorCoder( - runtimeContext.getCoderRegistry(), inputCoder.getValueCoder()); + context.getPipeline().getCoderRegistry(), inputCoder.getValueCoder()); } catch (CannotProvideCoderException e) { throw new IllegalStateException("Could not determine coder for accumulator", e); } @@ -360,7 +360,6 @@ public void evaluate( ((BoundedDataset) context.borrowDataset(transform)).getRDD(); WindowingStrategy windowingStrategy = context.getInput(transform).getWindowingStrategy(); - Accumulator aggAccum = AggregatorsAccumulator.getInstance(); Accumulator metricsAccum = MetricsAccumulator.getInstance(); JavaPairRDD, WindowedValue> all; @@ -370,11 +369,10 @@ public void evaluate( || signature.timerDeclarations().size() > 0; MultiDoFnFunction multiDoFnFunction = new MultiDoFnFunction<>( - aggAccum, metricsAccum, stepName, doFn, - context.getRuntimeContext(), + context.getSerializableOptions(), transform.getMainOutputTag(), transform.getAdditionalOutputTags().getAll(), TranslationUtils.getSideInputs(transform.getSideInputs(), context), @@ -452,10 +450,11 @@ private static TransformEvaluator> readBounded() { public void evaluate(Read.Bounded transform, EvaluationContext context) { String stepName = context.getCurrentTransform().getFullName(); final JavaSparkContext jsc = context.getSparkContext(); - final SparkRuntimeContext runtimeContext = context.getRuntimeContext(); // create an RDD from a BoundedSource. - JavaRDD> input = new SourceRDD.Bounded<>( - jsc.sc(), transform.getSource(), runtimeContext, stepName).toJavaRDD(); + JavaRDD> input = + new SourceRDD.Bounded<>( + jsc.sc(), transform.getSource(), context.getSerializableOptions(), stepName) + .toJavaRDD(); // cache to avoid re-evaluation of the source by Spark's lazy DAG evaluation. context.putDataset(transform, new BoundedDataset<>(input.cache())); } 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 cd5bb3ee5df61..38d6119b76da9 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 @@ -32,9 +32,8 @@ import java.util.Queue; import java.util.concurrent.LinkedBlockingQueue; import javax.annotation.Nonnull; +import org.apache.beam.runners.core.construction.SerializablePipelineOptions; import org.apache.beam.runners.core.metrics.MetricsContainerStepMap; -import org.apache.beam.runners.spark.aggregators.AggregatorsAccumulator; -import org.apache.beam.runners.spark.aggregators.NamedAggregators; import org.apache.beam.runners.spark.coders.CoderHelpers; import org.apache.beam.runners.spark.io.ConsoleIO; import org.apache.beam.runners.spark.io.CreateStream; @@ -50,7 +49,6 @@ import org.apache.beam.runners.spark.translation.SparkKeyedCombineFn; import org.apache.beam.runners.spark.translation.SparkPCollectionView; import org.apache.beam.runners.spark.translation.SparkPipelineTranslator; -import org.apache.beam.runners.spark.translation.SparkRuntimeContext; import org.apache.beam.runners.spark.translation.TransformEvaluator; import org.apache.beam.runners.spark.translation.TranslationUtils; import org.apache.beam.runners.spark.translation.WindowingHelpers; @@ -125,7 +123,7 @@ public void evaluate(Read.Unbounded transform, EvaluationContext context) { transform, SparkUnboundedSource.read( context.getStreamingContext(), - context.getRuntimeContext(), + context.getSerializableOptions(), transform.getSource(), stepName)); } @@ -273,7 +271,6 @@ public void evaluate(GroupByKey transform, EvaluationContext context) { JavaDStream>> dStream = inputDataset.getDStream(); @SuppressWarnings("unchecked") final KvCoder coder = (KvCoder) context.getInput(transform).getCoder(); - final SparkRuntimeContext runtimeContext = context.getRuntimeContext(); @SuppressWarnings("unchecked") final WindowingStrategy windowingStrategy = (WindowingStrategy) context.getInput(transform).getWindowingStrategy(); @@ -303,7 +300,7 @@ public JavaRDD>>>> call( coder.getKeyCoder(), wvCoder, windowingStrategy, - runtimeContext, + context.getSerializableOptions(), streamSources); context.putDataset(transform, new UnboundedDataset<>(outStream, streamSources)); @@ -336,7 +333,7 @@ public void evaluate(final Combine.GroupedValues transform, ((UnboundedDataset>>) context.borrowDataset(transform)); JavaDStream>>> dStream = unboundedDataset.getDStream(); - final SparkRuntimeContext runtimeContext = context.getRuntimeContext(); + final SerializablePipelineOptions options = context.getSerializableOptions(); final SparkPCollectionView pviews = context.getPViews(); JavaDStream>> outStream = dStream.transform( @@ -347,7 +344,7 @@ public void evaluate(final Combine.GroupedValues transform, call(JavaRDD>>> rdd) throws Exception { SparkKeyedCombineFn combineFnWithContext = - new SparkKeyedCombineFn<>(fn, runtimeContext, + new SparkKeyedCombineFn<>(fn, options, TranslationUtils.getSideInputs(transform.getSideInputs(), new JavaSparkContext(rdd.context()), pviews), windowingStrategy); @@ -374,7 +371,7 @@ public void evaluate( final DoFn doFn = transform.getFn(); rejectSplittable(doFn); rejectStateAndTimers(doFn); - final SparkRuntimeContext runtimeContext = context.getRuntimeContext(); + final SerializablePipelineOptions options = context.getSerializableOptions(); final SparkPCollectionView pviews = context.getPViews(); final WindowingStrategy windowingStrategy = context.getInput(transform).getWindowingStrategy(); @@ -393,8 +390,6 @@ public void evaluate( @Override public JavaPairRDD, WindowedValue> call( JavaRDD> rdd) throws Exception { - final Accumulator aggAccum = - AggregatorsAccumulator.getInstance(); final Accumulator metricsAccum = MetricsAccumulator.getInstance(); final Map, KV, SideInputBroadcast>> @@ -405,11 +400,10 @@ public JavaPairRDD, WindowedValue> call( pviews); return rdd.mapPartitionsToPair( new MultiDoFnFunction<>( - aggAccum, metricsAccum, stepName, doFn, - runtimeContext, + options, transform.getMainOutputTag(), transform.getAdditionalOutputTags().getAll(), sideInputs, diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/SparkRuntimeContextTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/SparkRuntimeContextTest.java deleted file mode 100644 index 456056af733d5..0000000000000 --- a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/SparkRuntimeContextTest.java +++ /dev/null @@ -1,122 +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.junit.Assert.assertEquals; - -import com.fasterxml.jackson.core.JsonGenerator; -import com.fasterxml.jackson.core.JsonParser; -import com.fasterxml.jackson.core.JsonProcessingException; -import com.fasterxml.jackson.databind.DeserializationContext; -import com.fasterxml.jackson.databind.JsonDeserializer; -import com.fasterxml.jackson.databind.JsonSerializer; -import com.fasterxml.jackson.databind.Module; -import com.fasterxml.jackson.databind.SerializerProvider; -import com.fasterxml.jackson.databind.annotation.JsonDeserialize; -import com.fasterxml.jackson.databind.annotation.JsonSerialize; -import com.fasterxml.jackson.databind.module.SimpleModule; -import com.google.auto.service.AutoService; -import java.io.ByteArrayInputStream; -import java.io.ByteArrayOutputStream; -import java.io.IOException; -import java.io.ObjectInputStream; -import java.io.ObjectOutputStream; -import org.apache.beam.sdk.Pipeline; -import org.apache.beam.sdk.options.PipelineOptions; -import org.apache.beam.sdk.options.PipelineOptionsFactory; -import org.apache.beam.sdk.testing.CrashingRunner; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.JUnit4; - -/** - * Tests for {@link SparkRuntimeContext}. - */ -@RunWith(JUnit4.class) -public class SparkRuntimeContextTest { - /** PipelineOptions used to test auto registration of Jackson modules. */ - public interface JacksonIncompatibleOptions extends PipelineOptions { - JacksonIncompatible getJacksonIncompatible(); - void setJacksonIncompatible(JacksonIncompatible value); - } - - /** A Jackson {@link Module} to test auto-registration of modules. */ - @AutoService(Module.class) - public static class RegisteredTestModule extends SimpleModule { - public RegisteredTestModule() { - super("RegisteredTestModule"); - setMixInAnnotation(JacksonIncompatible.class, JacksonIncompatibleMixin.class); - } - } - - /** A class which Jackson does not know how to serialize/deserialize. */ - public static class JacksonIncompatible { - private final String value; - public JacksonIncompatible(String value) { - this.value = value; - } - } - - /** A Jackson mixin used to add annotations to other classes. */ - @JsonDeserialize(using = JacksonIncompatibleDeserializer.class) - @JsonSerialize(using = JacksonIncompatibleSerializer.class) - public static final class JacksonIncompatibleMixin {} - - /** A Jackson deserializer for {@link JacksonIncompatible}. */ - public static class JacksonIncompatibleDeserializer extends - JsonDeserializer { - - @Override - public JacksonIncompatible deserialize(JsonParser jsonParser, - DeserializationContext deserializationContext) throws IOException, JsonProcessingException { - return new JacksonIncompatible(jsonParser.readValueAs(String.class)); - } - } - - /** A Jackson serializer for {@link JacksonIncompatible}. */ - public static class JacksonIncompatibleSerializer extends JsonSerializer { - - @Override - public void serialize(JacksonIncompatible jacksonIncompatible, JsonGenerator jsonGenerator, - SerializerProvider serializerProvider) throws IOException, JsonProcessingException { - jsonGenerator.writeString(jacksonIncompatible.value); - } - } - - @Test - public void testSerializingPipelineOptionsWithCustomUserType() throws Exception { - PipelineOptions options = PipelineOptionsFactory.fromArgs("--jacksonIncompatible=\"testValue\"") - .as(JacksonIncompatibleOptions.class); - options.setRunner(CrashingRunner.class); - Pipeline p = Pipeline.create(options); - SparkRuntimeContext context = new SparkRuntimeContext(options); - - ByteArrayOutputStream baos = new ByteArrayOutputStream(); - try (ObjectOutputStream outputStream = new ObjectOutputStream(baos)) { - outputStream.writeObject(context); - } - try (ObjectInputStream inputStream = - new ObjectInputStream(new ByteArrayInputStream(baos.toByteArray()))) { - SparkRuntimeContext copy = (SparkRuntimeContext) inputStream.readObject(); - assertEquals("testValue", - copy.getPipelineOptions().as(JacksonIncompatibleOptions.class) - .getJacksonIncompatible().value); - } - } -} 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 9a4d25ae37975..5cc0b3fc2074a 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 @@ -176,7 +176,12 @@ * *

              Serialization Of PipelineOptions

              * - * {@link PipelineRunner}s require support for options to be serialized. Each property + * {@link PipelineOptions} is intentionally not marked {@link java.io.Serializable}, in order + * to discourage pipeline authors from capturing {@link PipelineOptions} at pipeline construction + * time, because a pipeline may be saved as a template and run with a different set of options + * than the ones it was constructed with. See {@link Pipeline#run(PipelineOptions)}. + * + *

              However, {@link PipelineRunner}s require support for options to be serialized. Each property * within {@link PipelineOptions} must be able to be serialized using Jackson's * {@link ObjectMapper} or the getter method for the property annotated with * {@link JsonIgnore @JsonIgnore}. diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/options/ValueProviders.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/options/ValueProviders.java index bc479a2a533c3..2fffffa871e4e 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/options/ValueProviders.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/options/ValueProviders.java @@ -19,11 +19,9 @@ import static com.google.common.base.Preconditions.checkNotNull; -import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.databind.node.ObjectNode; import java.io.IOException; import java.util.Map; -import org.apache.beam.sdk.util.common.ReflectHelpers; /** * Utilities for working with the {@link ValueProvider} interface. @@ -37,11 +35,9 @@ private ValueProviders() {} */ public static String updateSerializedOptions( String serializedOptions, Map runtimeValues) { - ObjectMapper mapper = new ObjectMapper().registerModules( - ObjectMapper.findModules(ReflectHelpers.findClassLoader())); ObjectNode root, options; try { - root = mapper.readValue(serializedOptions, ObjectNode.class); + root = PipelineOptionsFactory.MAPPER.readValue(serializedOptions, ObjectNode.class); options = (ObjectNode) root.get("options"); checkNotNull(options, "Unable to locate 'options' in %s", serializedOptions); } catch (IOException e) { @@ -53,7 +49,7 @@ public static String updateSerializedOptions( options.put(entry.getKey(), entry.getValue()); } try { - return mapper.writeValueAsString(root); + return PipelineOptionsFactory.MAPPER.writeValueAsString(root); } catch (IOException e) { throw new RuntimeException("Unable to parse re-serialize options", e); } From a919f8e7769cb4e10e380553ee9ce7feb6ab369f Mon Sep 17 00:00:00 2001 From: bchambers Date: Tue, 1 Aug 2017 14:11:24 -0700 Subject: [PATCH 229/346] [BEAM-2708] Configure BZIP2 to read all "streams" Without this, CompressionMode.BZIP2 only supports "standard" bz2 files containing a single stream. With this change, BZIP2 also supports bz2 files containing multiple streams, such as those produced by pbzip2. --- .../apache/beam/sdk/io/CompressedSource.java | 2 +- .../beam/sdk/io/CompressedSourceTest.java | 44 ++++++++++++++++++- 2 files changed, 44 insertions(+), 2 deletions(-) diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/CompressedSource.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/CompressedSource.java index 4baac367f6869..ad81b61bc54d7 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/CompressedSource.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/CompressedSource.java @@ -146,7 +146,7 @@ public boolean matches(String fileName) { public ReadableByteChannel createDecompressingChannel(ReadableByteChannel channel) throws IOException { return Channels.newChannel( - new BZip2CompressorInputStream(Channels.newInputStream(channel))); + new BZip2CompressorInputStream(Channels.newInputStream(channel), true)); } }, 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 3fff31993306a..fa28e4b2d9f5c 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 @@ -252,6 +252,39 @@ public void testReadConcatenatedGzip() throws IOException { p.run(); } + /** + * Test a bzip2 file containing multiple streams is correctly decompressed. + * + *

              A bzip2 file may contain multiple streams and should decompress as the concatenation of + * those streams. + */ + @Test + @Category(NeedsRunner.class) + public void testReadMultiStreamBzip2() throws IOException { + CompressionMode mode = CompressionMode.BZIP2; + byte[] input1 = generateInput(5, 587973); + byte[] input2 = generateInput(5, 387374); + + ByteArrayOutputStream stream1 = new ByteArrayOutputStream(); + try (OutputStream os = getOutputStreamForMode(mode, stream1)) { + os.write(input1); + } + + ByteArrayOutputStream stream2 = new ByteArrayOutputStream(); + try (OutputStream os = getOutputStreamForMode(mode, stream2)) { + os.write(input2); + } + + File tmpFile = tmpFolder.newFile(); + try (OutputStream os = new FileOutputStream(tmpFile)) { + os.write(stream1.toByteArray()); + os.write(stream2.toByteArray()); + } + + byte[] output = Bytes.concat(input1, input2); + verifyReadContents(output, tmpFile, mode); + } + /** * Test reading empty input with bzip2. */ @@ -470,7 +503,16 @@ public void populateDisplayData(DisplayData.Builder builder) { */ private byte[] generateInput(int size) { // Arbitrary but fixed seed - Random random = new Random(285930); + return generateInput(size, 285930); + } + + + /** + * Generate byte array of given size. + */ + private byte[] generateInput(int size, int seed) { + // Arbitrary but fixed seed + Random random = new Random(seed); byte[] buff = new byte[size]; random.nextBytes(buff); return buff; From fc0f9c7b00d60213b893097571549c074fabd4b9 Mon Sep 17 00:00:00 2001 From: Mairbek Khadikov Date: Mon, 17 Jul 2017 12:37:47 -0700 Subject: [PATCH 230/346] [BEAM-1542] Specifies a User Agent in Spanner Client --- .../sdk/io/gcp/spanner/AbstractSpannerFn.java | 15 ++++++++++++++- .../beam/sdk/io/gcp/spanner/SpannerConfig.java | 11 ----------- 2 files changed, 14 insertions(+), 12 deletions(-) diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/AbstractSpannerFn.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/AbstractSpannerFn.java index 00008f1ebdcc1..50efdea41b094 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/AbstractSpannerFn.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/AbstractSpannerFn.java @@ -22,12 +22,16 @@ import com.google.cloud.spanner.Spanner; import com.google.cloud.spanner.SpannerOptions; import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.util.ReleaseInfo; /** * Abstract {@link DoFn} that manages {@link Spanner} lifecycle. Use {@link * AbstractSpannerFn#databaseClient} to access the Cloud Spanner database client. */ abstract class AbstractSpannerFn extends DoFn { + // A common user agent token that indicates that this request was originated from Apache Beam. + private static final String USER_AGENT_PREFIX = "Apache_Beam_Java"; + private transient Spanner spanner; private transient DatabaseClient databaseClient; @@ -36,7 +40,16 @@ abstract class AbstractSpannerFn extends DoFn @Setup public void setup() throws Exception { SpannerConfig spannerConfig = getSpannerConfig(); - SpannerOptions options = spannerConfig.buildSpannerOptions(); + SpannerOptions.Builder builder = SpannerOptions.newBuilder(); + if (spannerConfig.getProjectId() != null) { + builder.setProjectId(spannerConfig.getProjectId().get()); + } + if (spannerConfig.getServiceFactory() != null) { + builder.setServiceFactory(spannerConfig.getServiceFactory()); + } + ReleaseInfo releaseInfo = ReleaseInfo.getReleaseInfo(); + builder.setUserAgentPrefix(USER_AGENT_PREFIX + "/" + releaseInfo.getVersion()); + SpannerOptions options = builder.build(); spanner = options.getService(); databaseClient = spanner.getDatabaseClient(DatabaseId .of(options.getProjectId(), spannerConfig.getInstanceId().get(), diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerConfig.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerConfig.java index 02716fbaf4804..034c38acd23ba 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerConfig.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerConfig.java @@ -49,17 +49,6 @@ public abstract class SpannerConfig implements Serializable { abstract Builder toBuilder(); - SpannerOptions buildSpannerOptions() { - SpannerOptions.Builder builder = SpannerOptions.newBuilder(); - if (getProjectId() != null) { - builder.setProjectId(getProjectId().get()); - } - if (getServiceFactory() != null) { - builder.setServiceFactory(getServiceFactory()); - } - return builder.build(); - } - public static SpannerConfig create() { return builder().build(); } From 9cbc0d3b7d9c23acafd101f1e9093914fb792f39 Mon Sep 17 00:00:00 2001 From: Robert Bradshaw Date: Tue, 1 Aug 2017 16:18:15 -0700 Subject: [PATCH 231/346] Fix typo in dataflow_runner. --- sdks/python/apache_beam/runners/dataflow/dataflow_runner.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sdks/python/apache_beam/runners/dataflow/dataflow_runner.py b/sdks/python/apache_beam/runners/dataflow/dataflow_runner.py index d653e91d0923a..87785a275b578 100644 --- a/sdks/python/apache_beam/runners/dataflow/dataflow_runner.py +++ b/sdks/python/apache_beam/runners/dataflow/dataflow_runner.py @@ -592,7 +592,7 @@ def run_CombineValues(self, transform_node): # Note that the accumulator must not have a WindowedValue encoding, while # the output of this step does in fact have a WindowedValue encoding. accumulator_encoding = self._get_cloud_encoding( - transform_node.fn.get_accumulator_coder()) + transform_node.transform.fn.get_accumulator_coder()) output_encoding = self._get_encoded_output_coder(transform_node) step.encoding = output_encoding From 01f1d94bdaa5db08e4fb7b4e22d1b29a1ebd1e26 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Isma=C3=ABl=20Mej=C3=ADa?= Date: Fri, 28 Jul 2017 16:19:08 +0200 Subject: [PATCH 232/346] Remove unneeded surefire configuration for hadoop-file-system --- examples/java/pom.xml | 14 -------------- examples/java8/pom.xml | 11 ----------- sdks/java/io/google-cloud-platform/pom.xml | 12 ------------ sdks/java/io/hadoop-file-system/pom.xml | 14 -------------- sdks/java/io/kafka/pom.xml | 12 ------------ sdks/java/io/kinesis/pom.xml | 10 ---------- 6 files changed, 73 deletions(-) diff --git a/examples/java/pom.xml b/examples/java/pom.xml index 12fe06f3cc2c8..ade4cac7a4902 100644 --- a/examples/java/pom.xml +++ b/examples/java/pom.xml @@ -365,20 +365,7 @@ - - - org.apache.maven.plugins - maven-surefire-plugin - - - - - - - - - org.jacoco @@ -518,7 +505,6 @@ - org.jacoco diff --git a/sdks/java/io/google-cloud-platform/pom.xml b/sdks/java/io/google-cloud-platform/pom.xml index adb7e32fb8d2d..7842bcd58e163 100644 --- a/sdks/java/io/google-cloud-platform/pom.xml +++ b/sdks/java/io/google-cloud-platform/pom.xml @@ -46,18 +46,6 @@ - - - - org.apache.maven.plugins - maven-surefire-plugin - - - false - - - - diff --git a/sdks/java/io/hadoop-file-system/pom.xml b/sdks/java/io/hadoop-file-system/pom.xml index a9c2e57b15dbf..3cc7e001df528 100644 --- a/sdks/java/io/hadoop-file-system/pom.xml +++ b/sdks/java/io/hadoop-file-system/pom.xml @@ -30,20 +30,6 @@ Apache Beam :: SDKs :: Java :: IO :: Hadoop File System Library to read and write Hadoop/HDFS file formats from Beam. - - - - org.apache.maven.plugins - maven-surefire-plugin - - - false - - - - - - org.apache.beam diff --git a/sdks/java/io/kafka/pom.xml b/sdks/java/io/kafka/pom.xml index 1256c46d17bfd..39025105dce52 100644 --- a/sdks/java/io/kafka/pom.xml +++ b/sdks/java/io/kafka/pom.xml @@ -46,18 +46,6 @@ - - - - org.apache.maven.plugins - maven-surefire-plugin - - - false - - - - diff --git a/sdks/java/io/kinesis/pom.xml b/sdks/java/io/kinesis/pom.xml index 46d5e2604303c..872c5905857b0 100644 --- a/sdks/java/io/kinesis/pom.xml +++ b/sdks/java/io/kinesis/pom.xml @@ -31,16 +31,6 @@ - - org.apache.maven.plugins - maven-surefire-plugin - - - false - - - - org.apache.maven.plugins From b5017bfc85fdf5bb7cec4d2fe52963e9db32ed18 Mon Sep 17 00:00:00 2001 From: Flavio Fiszman Date: Fri, 30 Jun 2017 10:14:34 -0700 Subject: [PATCH 233/346] Add local filesystem as default and strip away prefix from local files in pipeline options. --- .../org/apache/beam/sdk/io/FileSystems.java | 17 +++---- .../apache/beam/sdk/io/LocalFileSystem.java | 36 +++++++++++++- .../beam/sdk/io/LocalFileSystemTest.java | 49 +++++++++++++++++++ 3 files changed, 92 insertions(+), 10 deletions(-) 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 index 2ed29e3a67183..bd4668f970d32 100644 --- 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 @@ -69,13 +69,13 @@ @Experimental(Kind.FILESYSTEM) public class FileSystems { - public static final String DEFAULT_SCHEME = "default"; + public static final String DEFAULT_SCHEME = "file"; private static final Pattern FILE_SCHEME_PATTERN = Pattern.compile("(?[a-zA-Z][-a-zA-Z0-9+.]*):.*"); private static final AtomicReference> SCHEME_TO_FILESYSTEM = new AtomicReference>( - ImmutableMap.of("file", new LocalFileSystem())); + ImmutableMap.of(DEFAULT_SCHEME, new LocalFileSystem())); /********************************** METHODS FOR CLIENT **********************************/ @@ -99,6 +99,9 @@ public class FileSystems { * component of {@link ResourceId}. This allows SDK libraries to construct file system agnostic * spec. {@link FileSystem FileSystems} can support additional patterns for user-provided specs. * + *

              In case the spec schemes don't match any known {@link FileSystem} implementations, + * FileSystems will attempt to use {@link LocalFileSystem} to resolve a path. + * * @return {@code List} in the same order of the input specs. * * @throws IllegalArgumentException if specs are invalid -- empty or have different schemes. @@ -176,7 +179,7 @@ public static List matchResources(List resourceIds) thr .transform(new Function() { @Override public String apply(@Nonnull ResourceId resourceId) { - return resourceId.toString(); + return resourceId.toString(); }}) .toList()); } @@ -423,7 +426,7 @@ private static String parseScheme(String spec) { Matcher matcher = FILE_SCHEME_PATTERN.matcher(spec); if (!matcher.matches()) { - return "file"; + return DEFAULT_SCHEME; } else { return matcher.group("scheme").toLowerCase(); } @@ -440,11 +443,7 @@ static FileSystem getFileSystemInternal(String scheme) { if (rval != null) { return rval; } - rval = schemeToFileSystem.get(DEFAULT_SCHEME); - if (rval != null) { - return rval; - } - throw new IllegalStateException("Unable to find registrar for " + scheme); + return schemeToFileSystem.get(DEFAULT_SCHEME); } /********************************** METHODS FOR REGISTRATION **********************************/ 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 index b732bee612036..5fe894d869021 100644 --- 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 @@ -38,6 +38,7 @@ import java.nio.file.PathMatcher; import java.nio.file.Paths; import java.nio.file.StandardCopyOption; +import java.util.Arrays; import java.util.Collection; import java.util.Collections; import java.util.List; @@ -46,11 +47,32 @@ import org.apache.beam.sdk.io.fs.MatchResult; import org.apache.beam.sdk.io.fs.MatchResult.Metadata; import org.apache.beam.sdk.io.fs.MatchResult.Status; +import org.apache.commons.lang3.SystemUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; /** * {@link FileSystem} implementation for local files. + * + * {@link #match} should interpret {@code spec} and resolve paths correctly according to OS being + * used. In order to do that specs should be defined in one of the below formats: + * + *

              Linux/Mac: + *

                + *
              • pom.xml
              • + *
              • /Users/beam/Documents/pom.xml
              • + *
              • file:/Users/beam/Documents/pom.xml
              • + *
              • file:///Users/beam/Documents/pom.xml
              • + *
              + * + *

              Windows OS: + *

                + *
              • pom.xml
              • + *
              • C:/Users/beam/Documents/pom.xml
              • + *
              • C:\\Users\\beam\\Documents\\pom.xml
              • + *
              • file:/C:/Users/beam/Documents/pom.xml
              • + *
              • file:///C:/Users/beam/Documents/pom.xml
              • + *
              */ class LocalFileSystem extends FileSystem { @@ -176,8 +198,20 @@ protected String getScheme() { } private MatchResult matchOne(String spec) throws IOException { - File file = Paths.get(spec).toFile(); + if (spec.toLowerCase().startsWith("file:")) { + spec = spec.substring("file:".length()); + } + if (SystemUtils.IS_OS_WINDOWS) { + List prefixes = Arrays.asList("///", "/"); + for (String prefix : prefixes) { + if (spec.toLowerCase().startsWith(prefix)) { + spec = spec.substring(prefix.length()); + } + } + } + + File file = Paths.get(spec).toFile(); if (file.exists()) { return MatchResult.create(Status.OK, ImmutableList.of(toMetadata(file))); } diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/LocalFileSystemTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/LocalFileSystemTest.java index 048908fbb1717..aaaeb83e1dcb1 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/LocalFileSystemTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/LocalFileSystemTest.java @@ -45,7 +45,9 @@ import org.apache.beam.sdk.io.fs.MatchResult; import org.apache.beam.sdk.io.fs.MatchResult.Metadata; import org.apache.beam.sdk.io.fs.ResolveOptions.StandardResolveOptions; +import org.apache.beam.sdk.testing.RestoreSystemProperties; import org.apache.beam.sdk.util.MimeTypes; +import org.apache.commons.lang3.SystemUtils; import org.hamcrest.Matchers; import org.junit.Rule; import org.junit.Test; @@ -61,6 +63,7 @@ public class LocalFileSystemTest { @Rule public ExpectedException thrown = ExpectedException.none(); @Rule public TemporaryFolder temporaryFolder = new TemporaryFolder(); + @Rule public RestoreSystemProperties restoreSystemProperties = new RestoreSystemProperties(); private LocalFileSystem localFileSystem = new LocalFileSystem(); @Test @@ -241,6 +244,52 @@ public void testMatchMultipleWithFileExtension() throws Exception { containsInAnyOrder(expected.toArray(new String[expected.size()]))); } + @Test + public void testMatchInDirectory() throws Exception { + List expected = ImmutableList.of(temporaryFolder.newFile("a").toString()); + temporaryFolder.newFile("aa"); + temporaryFolder.newFile("ab"); + + String expectedFile = expected.get(0); + int slashIndex = expectedFile.lastIndexOf('/'); + if (SystemUtils.IS_OS_WINDOWS) { + slashIndex = expectedFile.lastIndexOf('\\'); + } + String directory = expectedFile.substring(0, slashIndex); + String relative = expectedFile.substring(slashIndex + 1); + System.setProperty("user.dir", directory); + List results = localFileSystem.match(ImmutableList.of(relative)); + assertThat( + toFilenames(results), + containsInAnyOrder(expected.toArray(new String[expected.size()]))); + } + + @Test + public void testMatchWithFileSlashPrefix() throws Exception { + List expected = ImmutableList.of(temporaryFolder.newFile("a").toString()); + temporaryFolder.newFile("aa"); + temporaryFolder.newFile("ab"); + + String file = "file:/" + temporaryFolder.getRoot().toPath().resolve("a").toString(); + List results = localFileSystem.match(ImmutableList.of(file)); + assertThat( + toFilenames(results), + containsInAnyOrder(expected.toArray(new String[expected.size()]))); + } + + @Test + public void testMatchWithFileThreeSlashesPrefix() throws Exception { + List expected = ImmutableList.of(temporaryFolder.newFile("a").toString()); + temporaryFolder.newFile("aa"); + temporaryFolder.newFile("ab"); + + String file = "file:///" + temporaryFolder.getRoot().toPath().resolve("a").toString(); + List results = localFileSystem.match(ImmutableList.of(file)); + assertThat( + toFilenames(results), + containsInAnyOrder(expected.toArray(new String[expected.size()]))); + } + @Test public void testMatchMultipleWithoutSubdirectoryExpansion() throws Exception { File unmatchedSubDir = temporaryFolder.newFolder("aaa"); From 1e582e6e5c68ab394a5d4f0555715277101d43b6 Mon Sep 17 00:00:00 2001 From: Pablo Date: Fri, 28 Jul 2017 09:46:48 -0700 Subject: [PATCH 234/346] Improving naming of steps that consume side inputs --- sdks/python/apache_beam/runners/dataflow/dataflow_runner.py | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/sdks/python/apache_beam/runners/dataflow/dataflow_runner.py b/sdks/python/apache_beam/runners/dataflow/dataflow_runner.py index 87785a275b578..0df18825536ee 100644 --- a/sdks/python/apache_beam/runners/dataflow/dataflow_runner.py +++ b/sdks/python/apache_beam/runners/dataflow/dataflow_runner.py @@ -517,10 +517,12 @@ def run_ParDo(self, transform_node): si_labels[side_pval] = si_label # Now create the step for the ParDo transform being handled. + transform_name = transform_node.full_label.rsplit('/', 1)[-1] step = self._add_step( TransformNames.DO, transform_node.full_label + ( - '/Do' if transform_node.side_inputs else ''), + '/{}'.format(transform_name) + if transform_node.side_inputs else ''), transform_node, transform_node.transform.output_tags) fn_data = self._pardo_fn_data(transform_node, lookup_label) From 5d46243992948ab6d4c9436e353989b49186354b Mon Sep 17 00:00:00 2001 From: "chamikara@google.com" Date: Wed, 2 Aug 2017 22:49:33 -0700 Subject: [PATCH 235/346] Adds support for reading concatenated bzip2 files. Adds tests for concatenated gzip and bzip2 files. Removes test 'test_model_textio_gzip_concatenated' in 'snippets_test.py' since it's actually hitting 'DummyReadTransform' and not testing this feature. --- .../examples/snippets/snippets_test.py | 16 --- sdks/python/apache_beam/io/filesystem.py | 31 +++-- sdks/python/apache_beam/io/textio_test.py | 115 ++++++++++++++++++ 3 files changed, 129 insertions(+), 33 deletions(-) diff --git a/sdks/python/apache_beam/examples/snippets/snippets_test.py b/sdks/python/apache_beam/examples/snippets/snippets_test.py index 31f71b3bbb0da..9183d0dfea190 100644 --- a/sdks/python/apache_beam/examples/snippets/snippets_test.py +++ b/sdks/python/apache_beam/examples/snippets/snippets_test.py @@ -589,22 +589,6 @@ def test_model_textio_compressed(self): snippets.model_textio_compressed( {'read': gzip_file_name}, ['aa', 'bb', 'cc']) - def test_model_textio_gzip_concatenated(self): - temp_path_1 = self.create_temp_file('a\nb\nc\n') - temp_path_2 = self.create_temp_file('p\nq\nr\n') - temp_path_3 = self.create_temp_file('x\ny\nz') - gzip_file_name = temp_path_1 + '.gz' - with open(temp_path_1) as src, gzip.open(gzip_file_name, 'wb') as dst: - dst.writelines(src) - with open(temp_path_2) as src, gzip.open(gzip_file_name, 'ab') as dst: - dst.writelines(src) - with open(temp_path_3) as src, gzip.open(gzip_file_name, 'ab') as dst: - dst.writelines(src) - # Add the temporary gzip file to be cleaned up as well. - self.temp_files.append(gzip_file_name) - snippets.model_textio_compressed( - {'read': gzip_file_name}, ['a', 'b', 'c', 'p', 'q', 'r', 'x', 'y', 'z']) - @unittest.skipIf(datastore_pb2 is None, 'GCP dependencies are not installed') def test_model_datastoreio(self): # We cannot test datastoreio functionality in unit tests therefore we limit diff --git a/sdks/python/apache_beam/io/filesystem.py b/sdks/python/apache_beam/io/filesystem.py index 1f65d0a3a9edc..ef3040c9e727f 100644 --- a/sdks/python/apache_beam/io/filesystem.py +++ b/sdks/python/apache_beam/io/filesystem.py @@ -187,29 +187,26 @@ def _fetch_to_internal_buffer(self, num_bytes): del buf # Free up some possibly large and no-longer-needed memory. self._read_buffer.write(decompressed) else: - # EOF reached. - # Verify completeness and no corruption and flush (if needed by - # the underlying algorithm). - if self._compression_type == CompressionTypes.BZIP2: - # Having unused_data past end of stream would imply file corruption. - assert not self._decompressor.unused_data, 'Possible file corruption.' - try: - # EOF implies that the underlying BZIP2 stream must also have - # reached EOF. We expect this to raise an EOFError and we catch it - # below. Any other kind of error though would be problematic. - self._decompressor.decompress('dummy') - assert False, 'Possible file corruption.' - except EOFError: - pass # All is as expected! - elif self._compression_type == CompressionTypes.GZIP: - # If Gzip file check if there is unused data generated by gzip concat + # EOF of current stream reached. + # + # Any uncompressed data at the end of the stream of a gzip or bzip2 + # file that is not corrupted points to a concatenated compressed + # file. We read concatenated files by recursively creating decompressor + # objects for the unused compressed data. + if (self._compression_type == CompressionTypes.BZIP2 or + self._compression_type == CompressionTypes.GZIP): if self._decompressor.unused_data != '': buf = self._decompressor.unused_data - self._decompressor = zlib.decompressobj(self._gzip_mask) + self._decompressor = ( + bz2.BZ2Decompressor() + if self._compression_type == CompressionTypes.BZIP2 + else zlib.decompressobj(self._gzip_mask)) decompressed = self._decompressor.decompress(buf) self._read_buffer.write(decompressed) continue else: + # Gzip and bzip2 formats do not require flushing remaining data in the + # decompressor into the read buffer when fully decompressing files. self._read_buffer.write(self._decompressor.flush()) # Record that we have hit the end of file, so we won't unnecessarily diff --git a/sdks/python/apache_beam/io/textio_test.py b/sdks/python/apache_beam/io/textio_test.py index 9a4ec47e583f7..8bd7116ce295a 100644 --- a/sdks/python/apache_beam/io/textio_test.py +++ b/sdks/python/apache_beam/io/textio_test.py @@ -401,6 +401,64 @@ def test_read_bzip2(self): assert_that(pcoll, equal_to(lines)) pipeline.run() + def test_read_corrupted_bzip2_fails(self): + _, lines = write_data(15) + file_name = self._create_temp_file() + with bz2.BZ2File(file_name, 'wb') as f: + f.write('\n'.join(lines)) + + with open(file_name, 'wb') as f: + f.write('corrupt') + + pipeline = TestPipeline() + pcoll = pipeline | 'Read' >> ReadFromText( + file_name, + compression_type=CompressionTypes.BZIP2) + assert_that(pcoll, equal_to(lines)) + with self.assertRaises(Exception): + pipeline.run() + + def test_read_bzip2_concat(self): + bzip2_file_name1 = self._create_temp_file() + lines = ['a', 'b', 'c'] + with bz2.BZ2File(bzip2_file_name1, 'wb') as dst: + data = '\n'.join(lines) + '\n' + dst.write(data) + + bzip2_file_name2 = self._create_temp_file() + lines = ['p', 'q', 'r'] + with bz2.BZ2File(bzip2_file_name2, 'wb') as dst: + data = '\n'.join(lines) + '\n' + dst.write(data) + + bzip2_file_name3 = self._create_temp_file() + lines = ['x', 'y', 'z'] + with bz2.BZ2File(bzip2_file_name3, 'wb') as dst: + data = '\n'.join(lines) + '\n' + dst.write(data) + + final_bzip2_file = self._create_temp_file() + with open(bzip2_file_name1, 'rb') as src, open( + final_bzip2_file, 'wb') as dst: + dst.writelines(src.readlines()) + + with open(bzip2_file_name2, 'rb') as src, open( + final_bzip2_file, 'ab') as dst: + dst.writelines(src.readlines()) + + with open(bzip2_file_name3, 'rb') as src, open( + final_bzip2_file, 'ab') as dst: + dst.writelines(src.readlines()) + + pipeline = TestPipeline() + lines = pipeline | 'ReadFromText' >> beam.io.ReadFromText( + final_bzip2_file, + compression_type=beam.io.filesystem.CompressionTypes.BZIP2) + + expected = ['a', 'b', 'c', 'p', 'q', 'r', 'x', 'y', 'z'] + assert_that(lines, equal_to(expected)) + pipeline.run() + def test_read_gzip(self): _, lines = write_data(15) file_name = self._create_temp_file() @@ -415,6 +473,63 @@ def test_read_gzip(self): assert_that(pcoll, equal_to(lines)) pipeline.run() + def test_read_corrupted_gzip_fails(self): + _, lines = write_data(15) + file_name = self._create_temp_file() + with gzip.GzipFile(file_name, 'wb') as f: + f.write('\n'.join(lines)) + + with open(file_name, 'wb') as f: + f.write('corrupt') + + pipeline = TestPipeline() + pcoll = pipeline | 'Read' >> ReadFromText( + file_name, + 0, CompressionTypes.GZIP, + True, coders.StrUtf8Coder()) + assert_that(pcoll, equal_to(lines)) + + with self.assertRaises(Exception): + pipeline.run() + + def test_read_gzip_concat(self): + gzip_file_name1 = self._create_temp_file() + lines = ['a', 'b', 'c'] + with gzip.open(gzip_file_name1, 'wb') as dst: + data = '\n'.join(lines) + '\n' + dst.write(data) + + gzip_file_name2 = self._create_temp_file() + lines = ['p', 'q', 'r'] + with gzip.open(gzip_file_name2, 'wb') as dst: + data = '\n'.join(lines) + '\n' + dst.write(data) + + gzip_file_name3 = self._create_temp_file() + lines = ['x', 'y', 'z'] + with gzip.open(gzip_file_name3, 'wb') as dst: + data = '\n'.join(lines) + '\n' + dst.write(data) + + final_gzip_file = self._create_temp_file() + with open(gzip_file_name1, 'rb') as src, open(final_gzip_file, 'wb') as dst: + dst.writelines(src.readlines()) + + with open(gzip_file_name2, 'rb') as src, open(final_gzip_file, 'ab') as dst: + dst.writelines(src.readlines()) + + with open(gzip_file_name3, 'rb') as src, open(final_gzip_file, 'ab') as dst: + dst.writelines(src.readlines()) + + pipeline = TestPipeline() + lines = pipeline | 'ReadFromText' >> beam.io.ReadFromText( + final_gzip_file, + compression_type=beam.io.filesystem.CompressionTypes.GZIP) + + expected = ['a', 'b', 'c', 'p', 'q', 'r', 'x', 'y', 'z'] + assert_that(lines, equal_to(expected)) + pipeline.run() + def test_read_gzip_large(self): _, lines = write_data(10000) file_name = self._create_temp_file() From ac7f9739b01626abc559748ae983f6eb988430af Mon Sep 17 00:00:00 2001 From: Luke Cwik Date: Tue, 25 Jul 2017 09:02:41 -0700 Subject: [PATCH 236/346] [BEAM-1347] Add utility to be able to model inbound reading as a single input stream --- .../beam/fn/harness/stream/DataStreams.java | 158 ++++++++++++++++++ .../fn/harness/stream/DataStreamsTest.java | 92 ++++++++++ 2 files changed, 250 insertions(+) create mode 100644 sdks/java/harness/src/main/java/org/apache/beam/fn/harness/stream/DataStreams.java create mode 100644 sdks/java/harness/src/test/java/org/apache/beam/fn/harness/stream/DataStreamsTest.java diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/stream/DataStreams.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/stream/DataStreams.java new file mode 100644 index 0000000000000..d23d784df5dc6 --- /dev/null +++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/stream/DataStreams.java @@ -0,0 +1,158 @@ +/* + * 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.fn.harness.stream; + +import com.google.common.io.ByteStreams; +import com.google.protobuf.ByteString; +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.util.Iterator; +import java.util.NoSuchElementException; +import java.util.concurrent.BlockingQueue; +import org.apache.beam.fn.harness.fn.CloseableThrowingConsumer; + +/** + * {@link #inbound(Iterator)} treats multiple {@link ByteString}s as a single input stream and + * {@link #outbound(CloseableThrowingConsumer)} treats a single {@link OutputStream} as mulitple + * {@link ByteString}s. + */ +public class DataStreams { + /** + * Converts multiple {@link ByteString}s into a single {@link InputStream}. + * + *

              The iterator is accessed lazily. The supplied {@link Iterator} should block until + * either it knows that no more values will be provided or it has the next {@link ByteString}. + */ + public static InputStream inbound(Iterator bytes) { + return new Inbound(bytes); + } + + /** + * Converts a single {@link OutputStream} into multiple {@link ByteString}s. + */ + public static OutputStream outbound(CloseableThrowingConsumer consumer) { + // TODO: Migrate logic from BeamFnDataBufferingOutboundObserver + throw new UnsupportedOperationException(); + } + + /** + * An input stream which concatenates multiple {@link ByteString}s. Lazily accesses the + * first {@link Iterator} on first access of this input stream. + * + *

              Closing this input stream has no effect. + */ + private static class Inbound extends InputStream { + private static final InputStream EMPTY_STREAM = new InputStream() { + @Override + public int read() throws IOException { + return -1; + } + }; + + private final Iterator bytes; + private InputStream currentStream; + + public Inbound(Iterator bytes) { + this.currentStream = EMPTY_STREAM; + this.bytes = bytes; + } + + @Override + public int read() throws IOException { + int rval = -1; + // Move on to the next stream if we have read nothing + while ((rval = currentStream.read()) == -1 && bytes.hasNext()) { + currentStream = bytes.next().newInput(); + } + return rval; + } + + @Override + public int read(byte[] b, int off, int len) throws IOException { + int remainingLen = len; + while ((remainingLen -= ByteStreams.read( + currentStream, b, off + len - remainingLen, remainingLen)) > 0) { + if (bytes.hasNext()) { + currentStream = bytes.next().newInput(); + } else { + int bytesRead = len - remainingLen; + return bytesRead > 0 ? bytesRead : -1; + } + } + return len - remainingLen; + } + } + + /** + * Allows for one or more writing threads to append values to this iterator while one reading + * thread reads values. {@link #hasNext()} and {@link #next()} will block until a value is + * available or this has been closed. + * + *

              External synchronization must be provided if multiple readers would like to access the + * {@link Iterator#hasNext()} and {@link Iterator#next()} methods. + * + *

              The order or values which are appended to this iterator is nondeterministic when multiple + * threads call {@link #accept(Object)}. + */ + public static class BlockingQueueIterator implements + CloseableThrowingConsumer, Iterator { + private static final Object POISION_PILL = new Object(); + private final BlockingQueue queue; + + /** Only accessed by {@link Iterator#hasNext()} and {@link Iterator#next()} methods. */ + private T currentElement; + + public BlockingQueueIterator(BlockingQueue queue) { + this.queue = queue; + } + + @Override + public void close() throws Exception { + queue.put((T) POISION_PILL); + } + + @Override + public void accept(T t) throws Exception { + queue.put(t); + } + + @Override + public boolean hasNext() { + if (currentElement == null) { + try { + currentElement = queue.take(); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new IllegalStateException(e); + } + } + return currentElement != POISION_PILL; + } + + @Override + public T next() { + if (!hasNext()) { + throw new NoSuchElementException(); + } + T rval = currentElement; + currentElement = null; + return rval; + } + } +} diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/stream/DataStreamsTest.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/stream/DataStreamsTest.java new file mode 100644 index 0000000000000..d1415700d967b --- /dev/null +++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/stream/DataStreamsTest.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.fn.harness.stream; + +import static org.junit.Assert.assertEquals; + +import com.google.common.collect.Iterators; +import com.google.protobuf.ByteString; +import java.io.IOException; +import java.util.Arrays; +import java.util.List; +import java.util.concurrent.ArrayBlockingQueue; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.SynchronousQueue; +import org.apache.beam.fn.harness.stream.DataStreams.BlockingQueueIterator; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +/** Tests for {@link DataStreams}. */ +@RunWith(JUnit4.class) +public class DataStreamsTest { + private static final ByteString BYTES_A = ByteString.copyFromUtf8("TestData"); + private static final ByteString BYTES_B = ByteString.copyFromUtf8("SomeOtherTestData"); + + @Test + public void testEmptyRead() throws Exception { + assertEquals(ByteString.EMPTY, read()); + assertEquals(ByteString.EMPTY, read(ByteString.EMPTY)); + assertEquals(ByteString.EMPTY, read(ByteString.EMPTY, ByteString.EMPTY)); + } + + @Test + public void testRead() throws Exception { + assertEquals(BYTES_A.concat(BYTES_B), read(BYTES_A, BYTES_B)); + assertEquals(BYTES_A.concat(BYTES_B), read(BYTES_A, ByteString.EMPTY, BYTES_B)); + assertEquals(BYTES_A.concat(BYTES_B), read(BYTES_A, BYTES_B, ByteString.EMPTY)); + } + + @Test(timeout = 10_000) + public void testBlockingQueueIteratorWithoutBlocking() throws Exception { + BlockingQueueIterator iterator = + new BlockingQueueIterator<>(new ArrayBlockingQueue<>(3)); + + iterator.accept("A"); + iterator.accept("B"); + iterator.close(); + + assertEquals(Arrays.asList("A", "B"), + Arrays.asList(Iterators.toArray(iterator, String.class))); + } + + @Test(timeout = 10_000) + public void testBlockingQueueIteratorWithBlocking() throws Exception { + // The synchronous queue only allows for one element to transfer at a time and blocks + // the sending/receiving parties until both parties are there. + final BlockingQueueIterator iterator = + new BlockingQueueIterator<>(new SynchronousQueue<>()); + final CompletableFuture> valuesFuture = new CompletableFuture<>(); + Thread appender = new Thread() { + @Override + public void run() { + valuesFuture.complete(Arrays.asList(Iterators.toArray(iterator, String.class))); + } + }; + appender.start(); + iterator.accept("A"); + iterator.accept("B"); + iterator.close(); + assertEquals(Arrays.asList("A", "B"), valuesFuture.get()); + appender.join(); + } + + private static ByteString read(ByteString... bytes) throws IOException { + return ByteString.readFrom(DataStreams.inbound(Arrays.asList(bytes).iterator())); + } +} From e017a0ec8a16b63828d0955f405b23bc9771bc9e Mon Sep 17 00:00:00 2001 From: Eugene Kirpichov Date: Wed, 26 Jul 2017 16:05:27 -0700 Subject: [PATCH 237/346] Makes all Source classes override getOutputCoder instead of getDefaultOutputCoder --- .../apex/translation/utils/ValuesSource.java | 2 +- .../runners/apex/examples/UnboundedTextSource.java | 2 +- .../apex/translation/GroupByKeyTranslatorTest.java | 2 +- .../apex/translation/utils/CollectionSource.java | 2 +- .../UnboundedReadFromBoundedSource.java | 8 ++++---- .../core/construction/ReadTranslationTest.java | 4 ++-- .../UnboundedReadFromBoundedSourceTest.java | 2 +- .../direct/BoundedReadEvaluatorFactoryTest.java | 2 +- .../beam/runners/direct/DirectRunnerTest.java | 4 ++-- .../direct/UnboundedReadEvaluatorFactoryTest.java | 2 +- .../streaming/io/UnboundedSocketSource.java | 2 +- .../flink/streaming/TestCountingSource.java | 2 +- .../beam/runners/dataflow/DataflowRunner.java | 6 +++--- .../beam/runners/spark/io/MicrobatchSource.java | 4 ++-- .../runners/spark/io/SparkUnboundedSource.java | 2 +- .../runners/spark/stateful/StateSpecFunctions.java | 2 +- .../java/org/apache/beam/sdk/io/AvroSource.java | 2 +- .../sdk/io/BoundedReadFromUnboundedSource.java | 6 +++--- .../org/apache/beam/sdk/io/CompressedSource.java | 6 +++--- .../org/apache/beam/sdk/io/CountingSource.java | 4 ++-- .../src/main/java/org/apache/beam/sdk/io/Read.java | 4 ++-- .../main/java/org/apache/beam/sdk/io/Source.java | 14 ++++++++++---- .../java/org/apache/beam/sdk/io/TFRecordIO.java | 2 +- .../java/org/apache/beam/sdk/io/TextSource.java | 2 +- .../apache/beam/sdk/testing/SourceTestUtils.java | 12 ++++++------ .../org/apache/beam/sdk/transforms/Create.java | 2 +- .../apache/beam/sdk/io/CompressedSourceTest.java | 2 +- .../apache/beam/sdk/io/FileBasedSourceTest.java | 2 +- .../apache/beam/sdk/io/OffsetBasedSourceTest.java | 2 +- .../test/java/org/apache/beam/sdk/io/ReadTest.java | 4 ++-- .../sdk/runners/dataflow/TestCountingSource.java | 2 +- .../org/apache/beam/sdk/transforms/CreateTest.java | 4 ++-- .../java/org/apache/beam/sdk/io/amqp/AmqpIO.java | 2 +- .../apache/beam/sdk/io/cassandra/CassandraIO.java | 2 +- .../beam/sdk/io/elasticsearch/ElasticsearchIO.java | 2 +- .../sdk/io/gcp/bigquery/BigQuerySourceBase.java | 4 ++-- .../beam/sdk/io/gcp/bigtable/BigtableIO.java | 2 +- .../sdk/io/gcp/pubsub/PubsubUnboundedSource.java | 2 +- .../io/hadoop/inputformat/HadoopInputFormatIO.java | 2 +- .../java/org/apache/beam/sdk/io/hbase/HBaseIO.java | 2 +- .../apache/beam/sdk/io/hcatalog/HCatalogIO.java | 2 +- .../java/org/apache/beam/sdk/io/jms/JmsIO.java | 2 +- .../java/org/apache/beam/sdk/io/kafka/KafkaIO.java | 2 +- .../apache/beam/sdk/io/kinesis/KinesisSource.java | 2 +- .../beam/sdk/io/mongodb/MongoDbGridFSIO.java | 2 +- .../org/apache/beam/sdk/io/mongodb/MongoDbIO.java | 2 +- .../java/org/apache/beam/sdk/io/mqtt/MqttIO.java | 2 +- .../java/org/apache/beam/sdk/io/xml/XmlSource.java | 2 +- 48 files changed, 79 insertions(+), 73 deletions(-) diff --git a/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/utils/ValuesSource.java b/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/utils/ValuesSource.java index 41f027faa96df..4a00ff1c59c51 100644 --- a/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/utils/ValuesSource.java +++ b/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/utils/ValuesSource.java @@ -81,7 +81,7 @@ public void validate() { } @Override - public Coder getDefaultOutputCoder() { + public Coder getOutputCoder() { return iterableCoder.getElemCoder(); } diff --git a/runners/apex/src/test/java/org/apache/beam/runners/apex/examples/UnboundedTextSource.java b/runners/apex/src/test/java/org/apache/beam/runners/apex/examples/UnboundedTextSource.java index c590a2e9783c2..8f3e6bc680951 100644 --- a/runners/apex/src/test/java/org/apache/beam/runners/apex/examples/UnboundedTextSource.java +++ b/runners/apex/src/test/java/org/apache/beam/runners/apex/examples/UnboundedTextSource.java @@ -59,7 +59,7 @@ public void validate() { } @Override - public Coder getDefaultOutputCoder() { + public Coder getOutputCoder() { return StringUtf8Coder.of(); } 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 9c61b47be0204..58f33aec62f17 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 @@ -153,7 +153,7 @@ public void validate() { } @Override - public Coder getDefaultOutputCoder() { + public Coder getOutputCoder() { return StringUtf8Coder.of(); } diff --git a/runners/apex/src/test/java/org/apache/beam/runners/apex/translation/utils/CollectionSource.java b/runners/apex/src/test/java/org/apache/beam/runners/apex/translation/utils/CollectionSource.java index 288aadedb46ca..01a2a85e8d386 100644 --- a/runners/apex/src/test/java/org/apache/beam/runners/apex/translation/utils/CollectionSource.java +++ b/runners/apex/src/test/java/org/apache/beam/runners/apex/translation/utils/CollectionSource.java @@ -67,7 +67,7 @@ public void validate() { } @Override - public Coder getDefaultOutputCoder() { + public Coder getOutputCoder() { return coder; } diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/UnboundedReadFromBoundedSource.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/UnboundedReadFromBoundedSource.java index 24eb384792686..f35f4c3de08c2 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/UnboundedReadFromBoundedSource.java +++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/UnboundedReadFromBoundedSource.java @@ -92,7 +92,7 @@ public PCollection expand(PBegin input) { @Override protected Coder getDefaultOutputCoder() { - return source.getDefaultOutputCoder(); + return source.getOutputCoder(); } @Override @@ -166,14 +166,14 @@ public Reader createReader(PipelineOptions options, Checkpoint checkpoint) } @Override - public Coder getDefaultOutputCoder() { - return boundedSource.getDefaultOutputCoder(); + public Coder getOutputCoder() { + return boundedSource.getOutputCoder(); } @SuppressWarnings({"rawtypes", "unchecked"}) @Override public Coder> getCheckpointMarkCoder() { - return new CheckpointCoder<>(boundedSource.getDefaultOutputCoder()); + return new CheckpointCoder<>(boundedSource.getOutputCoder()); } @VisibleForTesting diff --git a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/ReadTranslationTest.java b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/ReadTranslationTest.java index 740b3245dd6af..f85bd79089c5a 100644 --- a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/ReadTranslationTest.java +++ b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/ReadTranslationTest.java @@ -112,7 +112,7 @@ public BoundedReader createReader(PipelineOptions options) throws IOExce public void validate() {} @Override - public Coder getDefaultOutputCoder() { + public Coder getOutputCoder() { return StringUtf8Coder.of(); } @@ -132,7 +132,7 @@ private static class TestUnboundedSource extends UnboundedSource getDefaultOutputCoder() { + public Coder getOutputCoder() { return ByteArrayCoder.of(); } diff --git a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/UnboundedReadFromBoundedSourceTest.java b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/UnboundedReadFromBoundedSourceTest.java index 0e48a9dc26b53..62b06b7d9e66c 100644 --- a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/UnboundedReadFromBoundedSourceTest.java +++ b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/UnboundedReadFromBoundedSourceTest.java @@ -320,7 +320,7 @@ protected UnsplittableReader createSingleFileReader(PipelineOptions options) { } @Override - public Coder getDefaultOutputCoder() { + public Coder getOutputCoder() { return SerializableCoder.of(Byte.class); } 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 6180d2994a1a5..3d8188433a5d1 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 @@ -395,7 +395,7 @@ public OffsetBasedSource createSourceForSubrange(long start, long end) { } @Override - public Coder getDefaultOutputCoder() { + public Coder getOutputCoder() { return coder; } } 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 943d27c07ad1f..d3f407a29b784 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 @@ -573,8 +573,8 @@ public void validate() { } @Override - public Coder getDefaultOutputCoder() { - return underlying.getDefaultOutputCoder(); + public Coder getOutputCoder() { + return underlying.getOutputCoder(); } } } 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 2a01db55ae1ec..cc6847df55542 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 @@ -477,7 +477,7 @@ public boolean requiresDeduping() { public void validate() {} @Override - public Coder getDefaultOutputCoder() { + public Coder getOutputCoder() { return coder; } diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/UnboundedSocketSource.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/UnboundedSocketSource.java index 910a33f99d1d1..49e4ddc7019b6 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/UnboundedSocketSource.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/UnboundedSocketSource.java @@ -123,7 +123,7 @@ public void validate() { } @Override - public Coder getDefaultOutputCoder() { + public Coder getOutputCoder() { return DEFAULT_SOCKET_CODER; } diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/streaming/TestCountingSource.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/streaming/TestCountingSource.java index edf548a57fa6d..fcb9282798092 100644 --- a/runners/flink/src/test/java/org/apache/beam/runners/flink/streaming/TestCountingSource.java +++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/streaming/TestCountingSource.java @@ -238,7 +238,7 @@ public CountingSourceReader createReader( public void validate() {} @Override - public Coder> getDefaultOutputCoder() { + public Coder> getOutputCoder() { return KvCoder.of(VarIntCoder.of(), VarIntCoder.of()); } 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 f8d2c3c12b652..8fce5b4aa714c 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 @@ -1176,7 +1176,7 @@ public StreamingUnboundedRead(DataflowRunner runner, Read.Unbounded transform @Override protected Coder getDefaultOutputCoder() { - return source.getDefaultOutputCoder(); + return source.getOutputCoder(); } @Override @@ -1212,7 +1212,7 @@ public final PCollection> expand(PInput input) { @Override protected Coder> getDefaultOutputCoder() { - return ValueWithRecordId.ValueWithRecordIdCoder.of(source.getDefaultOutputCoder()); + return ValueWithRecordId.ValueWithRecordIdCoder.of(source.getOutputCoder()); } @Override @@ -1291,7 +1291,7 @@ public StreamingBoundedRead(DataflowRunner runner, Read.Bounded transform) { @Override protected Coder getDefaultOutputCoder() { - return source.getDefaultOutputCoder(); + return source.getOutputCoder(); } @Override 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 3b48caf3ef855..ae873a3954834 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 @@ -140,8 +140,8 @@ public void validate() { } @Override - public Coder getDefaultOutputCoder() { - return source.getDefaultOutputCoder(); + public Coder getOutputCoder() { + return source.getOutputCoder(); } public Coder getCheckpointMarkCoder() { 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 b31aa9f6e59ba..26af0c02ef9ce 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 @@ -116,7 +116,7 @@ public static UnboundedDataset re // output the actual (deserialized) stream. WindowedValue.FullWindowedValueCoder coder = WindowedValue.FullWindowedValueCoder.of( - source.getDefaultOutputCoder(), + source.getOutputCoder(), GlobalWindow.Coder.INSTANCE); JavaDStream> readUnboundedStream = mapWithStateDStream 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 1b54478703664..ca5471550ae6e 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 @@ -161,7 +161,7 @@ public Tuple2, Metadata> apply( final List readValues = new ArrayList<>(); WindowedValue.FullWindowedValueCoder coder = WindowedValue.FullWindowedValueCoder.of( - source.getDefaultOutputCoder(), + source.getOutputCoder(), GlobalWindow.Coder.INSTANCE); try { // measure how long a read takes per-partition. diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/AvroSource.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/AvroSource.java index d277503d7dc51..8dd312514f1fe 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/AvroSource.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/AvroSource.java @@ -315,7 +315,7 @@ protected BlockBasedReader createSingleFileReader(PipelineOptions options) { } @Override - public Coder getDefaultOutputCoder() { + public Coder getOutputCoder() { return mode.getOutputCoder(); } 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 c882447d7ccc6..8505ca476c7ff 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 @@ -119,7 +119,7 @@ public byte[] apply(ValueWithRecordId input) { @Override protected Coder getDefaultOutputCoder() { - return source.getDefaultOutputCoder(); + return source.getOutputCoder(); } @Override @@ -211,8 +211,8 @@ public long getEstimatedSizeBytes(PipelineOptions options) { } @Override - public Coder> getDefaultOutputCoder() { - return ValueWithRecordId.ValueWithRecordIdCoder.of(getSource().getDefaultOutputCoder()); + public Coder> getOutputCoder() { + return ValueWithRecordId.ValueWithRecordIdCoder.of(getSource().getOutputCoder()); } @Override diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/CompressedSource.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/CompressedSource.java index ad81b61bc54d7..6943a02ee4b08 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/CompressedSource.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/CompressedSource.java @@ -404,11 +404,11 @@ public void populateDisplayData(DisplayData.Builder builder) { } /** - * Returns the delegate source's default output coder. + * Returns the delegate source's output coder. */ @Override - public final Coder getDefaultOutputCoder() { - return sourceDelegate.getDefaultOutputCoder(); + public final Coder getOutputCoder() { + return sourceDelegate.getOutputCoder(); } public final DecompressingChannelFactory getChannelFactory() { 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 6202c2b5fad56..b47edc72899d6 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 @@ -188,7 +188,7 @@ public org.apache.beam.sdk.io.BoundedSource.BoundedReader createReader( } @Override - public Coder getDefaultOutputCoder() { + public Coder getOutputCoder() { return VarLongCoder.of(); } @@ -364,7 +364,7 @@ public Coder getCheckpointMarkCoder() { public void validate() {} @Override - public Coder getDefaultOutputCoder() { + public Coder getOutputCoder() { return VarLongCoder.of(); } 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 a07fca89457d5..6e6750ddca69f 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 @@ -96,7 +96,7 @@ private Bounded(@Nullable String name, BoundedSource source) { @Override protected Coder getDefaultOutputCoder() { - return source.getDefaultOutputCoder(); + return source.getOutputCoder(); } @Override @@ -164,7 +164,7 @@ public BoundedReadFromUnboundedSource withMaxReadTime(Duration maxReadTime) { @Override protected Coder getDefaultOutputCoder() { - return source.getDefaultOutputCoder(); + return source.getOutputCoder(); } @Override diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/Source.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/Source.java index 542d91ca791ea..32a72705c346f 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/Source.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/Source.java @@ -61,10 +61,16 @@ public abstract class Source implements Serializable, HasDisplayData { */ public abstract void validate(); - /** - * Returns the default {@code Coder} to use for the data read from this source. - */ - public abstract Coder getDefaultOutputCoder(); + /** @deprecated Override {@link #getOutputCoder()} instead. */ + @Deprecated + public Coder getDefaultOutputCoder() { + throw new UnsupportedOperationException("Source needs to override getOutputCoder()"); + } + + /** Returns the {@code Coder} to use for the data read from this source. */ + public Coder getOutputCoder() { + return getDefaultOutputCoder(); + } /** * {@inheritDoc} diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/TFRecordIO.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/TFRecordIO.java index 29b3e295e3b3b..1b2e95bc74eeb 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/TFRecordIO.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/TFRecordIO.java @@ -474,7 +474,7 @@ protected FileBasedReader createSingleFileReader(PipelineOptions options } @Override - public Coder getDefaultOutputCoder() { + public Coder getOutputCoder() { return DEFAULT_BYTE_ARRAY_CODER; } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/TextSource.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/TextSource.java index 4d9fa776e485e..86c73d5d32606 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/TextSource.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/TextSource.java @@ -69,7 +69,7 @@ protected FileBasedReader createSingleFileReader(PipelineOptions options } @Override - public Coder getDefaultOutputCoder() { + public Coder getOutputCoder() { return StringUtf8Coder.of(); } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/SourceTestUtils.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/SourceTestUtils.java index cde0b946e966c..e147221ed3383 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/SourceTestUtils.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/testing/SourceTestUtils.java @@ -212,7 +212,7 @@ public static void assertSourcesEqualReferenceSource( List> sources, PipelineOptions options) throws Exception { - Coder coder = referenceSource.getDefaultOutputCoder(); + Coder coder = referenceSource.getOutputCoder(); List referenceRecords = readFromSource(referenceSource, options); List bundleRecords = new ArrayList<>(); for (BoundedSource source : sources) { @@ -221,7 +221,7 @@ public static void assertSourcesEqualReferenceSource( + source + " is not compatible with Coder type for referenceSource " + referenceSource, - source.getDefaultOutputCoder(), + source.getOutputCoder(), equalTo(coder)); List elems = readFromSource(source, options); bundleRecords.addAll(elems); @@ -239,7 +239,7 @@ public static void assertSourcesEqualReferenceSource( */ public static void assertUnstartedReaderReadsSameAsItsSource( BoundedSource.BoundedReader reader, PipelineOptions options) throws Exception { - Coder coder = reader.getCurrentSource().getDefaultOutputCoder(); + Coder coder = reader.getCurrentSource().getOutputCoder(); List expected = readFromUnstartedReader(reader); List actual = readFromSource(reader.getCurrentSource(), options); List> expectedStructural = createStructuralValues(coder, expected); @@ -415,7 +415,7 @@ private static SourceTestUtils.SplitAtFractionResult verifySingleSplitAtFrac source, primary, residual); - Coder coder = primary.getDefaultOutputCoder(); + Coder coder = primary.getOutputCoder(); List> primaryValues = createStructuralValues(coder, primaryItems); List> currentValues = @@ -728,8 +728,8 @@ public void validate() { } @Override - public Coder getDefaultOutputCoder() { - return boundedSource.getDefaultOutputCoder(); + public Coder getOutputCoder() { + return boundedSource.getOutputCoder(); } private static class UnsplittableReader extends BoundedReader { 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 09e12efe10b49..a28e9b27e7a0b 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 @@ -411,7 +411,7 @@ public BoundedSource.BoundedReader createReader(PipelineOptions options) public void validate() {} @Override - public Coder getDefaultOutputCoder() { + public Coder getOutputCoder() { return coder; } 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 fa28e4b2d9f5c..fe6f01f6c01f8 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 @@ -638,7 +638,7 @@ protected FileBasedReader createSingleFileReader(PipelineOptions options) } @Override - public Coder getDefaultOutputCoder() { + public Coder getOutputCoder() { return SerializableCoder.of(Byte.class); } 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 c15e6678b428f..1bdb915d9d00d 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 @@ -107,7 +107,7 @@ public TestFileBasedSource( public void validate() {} @Override - public Coder getDefaultOutputCoder() { + public Coder getOutputCoder() { return StringUtf8Coder.of(); } diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/OffsetBasedSourceTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/OffsetBasedSourceTest.java index 25168a37dce68..feda3551ced1b 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/OffsetBasedSourceTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/OffsetBasedSourceTest.java @@ -65,7 +65,7 @@ public OffsetBasedSource createSourceForSubrange(long start, long end) public void validate() {} @Override - public Coder getDefaultOutputCoder() { + public Coder getOutputCoder() { return BigEndianIntegerCoder.of(); } diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/ReadTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/ReadTest.java index 74acf18764be8..4277dc3461d58 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/ReadTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/ReadTest.java @@ -171,7 +171,7 @@ public BoundedReader createReader(PipelineOptions options) throws IOExce public void validate() {} @Override - public Coder getDefaultOutputCoder() { + public Coder getOutputCoder() { return StringUtf8Coder.of(); } } @@ -207,7 +207,7 @@ public Coder getCheckpointMarkCoder() { public void validate() {} @Override - public Coder getDefaultOutputCoder() { + public Coder getOutputCoder() { return StringUtf8Coder.of(); } } diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/dataflow/TestCountingSource.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/dataflow/TestCountingSource.java index 9fcc3c596a0cf..338ea385a2eb5 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/dataflow/TestCountingSource.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/runners/dataflow/TestCountingSource.java @@ -248,7 +248,7 @@ public CountingSourceReader createReader( public void validate() {} @Override - public Coder> getDefaultOutputCoder() { + public Coder> getOutputCoder() { return KvCoder.of(VarIntCoder.of(), VarIntCoder.of()); } } 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 6a682ef658f75..6be67722281d0 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 @@ -487,12 +487,12 @@ public void testSourceSplitEmpty() throws Exception { } @Test - public void testSourceGetDefaultOutputCoderReturnsConstructorCoder() throws Exception { + public void testSourceGetOutputCoderReturnsConstructorCoder() throws Exception { Coder coder = VarIntCoder.of(); CreateSource source = CreateSource.fromIterable(ImmutableList.of(1, 2, 3, 4, 5, 6, 7, 8), coder); - Coder defaultCoder = source.getDefaultOutputCoder(); + Coder defaultCoder = source.getOutputCoder(); assertThat(defaultCoder, equalTo(coder)); } diff --git a/sdks/java/io/amqp/src/main/java/org/apache/beam/sdk/io/amqp/AmqpIO.java b/sdks/java/io/amqp/src/main/java/org/apache/beam/sdk/io/amqp/AmqpIO.java index 1f307b252cb05..508373f4d88c4 100644 --- a/sdks/java/io/amqp/src/main/java/org/apache/beam/sdk/io/amqp/AmqpIO.java +++ b/sdks/java/io/amqp/src/main/java/org/apache/beam/sdk/io/amqp/AmqpIO.java @@ -246,7 +246,7 @@ public UnboundedReader createReader(PipelineOptions pipelineOptions, } @Override - public Coder getDefaultOutputCoder() { + public Coder getOutputCoder() { return new AmqpMessageCoder(); } diff --git a/sdks/java/io/cassandra/src/main/java/org/apache/beam/sdk/io/cassandra/CassandraIO.java b/sdks/java/io/cassandra/src/main/java/org/apache/beam/sdk/io/cassandra/CassandraIO.java index 32905b77258a8..eacc3e402fc8a 100644 --- a/sdks/java/io/cassandra/src/main/java/org/apache/beam/sdk/io/cassandra/CassandraIO.java +++ b/sdks/java/io/cassandra/src/main/java/org/apache/beam/sdk/io/cassandra/CassandraIO.java @@ -289,7 +289,7 @@ static class CassandraSource extends BoundedSource { } @Override - public Coder getDefaultOutputCoder() { + public Coder getOutputCoder() { return spec.coder(); } diff --git a/sdks/java/io/elasticsearch/src/main/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIO.java b/sdks/java/io/elasticsearch/src/main/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIO.java index acc7f2f0e5043..50468887120bb 100644 --- a/sdks/java/io/elasticsearch/src/main/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIO.java +++ b/sdks/java/io/elasticsearch/src/main/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIO.java @@ -484,7 +484,7 @@ public void validate() { } @Override - public Coder getDefaultOutputCoder() { + public Coder getOutputCoder() { return StringUtf8Coder.of(); } diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQuerySourceBase.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQuerySourceBase.java index 6c118a0b116cd..abe559c5893b0 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQuerySourceBase.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQuerySourceBase.java @@ -133,7 +133,7 @@ public void validate() { } @Override - public Coder getDefaultOutputCoder() { + public Coder getOutputCoder() { return TableRowJsonCoder.of(); } @@ -184,7 +184,7 @@ public TableRow apply(GenericRecord input) { List> avroSources = Lists.newArrayList(); for (ResourceId file : files) { avroSources.add( - AvroSource.from(file.toString()).withParseFn(function, getDefaultOutputCoder())); + AvroSource.from(file.toString()).withParseFn(function, getOutputCoder())); } return ImmutableList.copyOf(avroSources); } 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 0a90dde94f43c..c5b0fbf8e6b0a 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 @@ -893,7 +893,7 @@ public void populateDisplayData(DisplayData.Builder builder) { } @Override - public Coder getDefaultOutputCoder() { + public Coder getOutputCoder() { return ProtoCoder.of(Row.class); } diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubUnboundedSource.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubUnboundedSource.java index b7df804f098ab..8da6ff4f80c5f 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubUnboundedSource.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubUnboundedSource.java @@ -1164,7 +1164,7 @@ public Coder getCheckpointMarkCoder() { } @Override - public Coder getDefaultOutputCoder() { + public Coder getOutputCoder() { return outer.getNeedsAttributes() ? PubsubMessageWithAttributesCoder.of() : PubsubMessagePayloadOnlyCoder.of(); diff --git a/sdks/java/io/hadoop/input-format/src/main/java/org/apache/beam/sdk/io/hadoop/inputformat/HadoopInputFormatIO.java b/sdks/java/io/hadoop/input-format/src/main/java/org/apache/beam/sdk/io/hadoop/inputformat/HadoopInputFormatIO.java index 0b4c23f65c831..20ca50a005f88 100644 --- a/sdks/java/io/hadoop/input-format/src/main/java/org/apache/beam/sdk/io/hadoop/inputformat/HadoopInputFormatIO.java +++ b/sdks/java/io/hadoop/input-format/src/main/java/org/apache/beam/sdk/io/hadoop/inputformat/HadoopInputFormatIO.java @@ -552,7 +552,7 @@ void setInputFormatObj(InputFormat inputFormatObj) { } @Override - public Coder> getDefaultOutputCoder() { + public Coder> getOutputCoder() { return KvCoder.of(keyCoder, valueCoder); } diff --git a/sdks/java/io/hbase/src/main/java/org/apache/beam/sdk/io/hbase/HBaseIO.java b/sdks/java/io/hbase/src/main/java/org/apache/beam/sdk/io/hbase/HBaseIO.java index 90ede4ce01347..2ba682639ab75 100644 --- a/sdks/java/io/hbase/src/main/java/org/apache/beam/sdk/io/hbase/HBaseIO.java +++ b/sdks/java/io/hbase/src/main/java/org/apache/beam/sdk/io/hbase/HBaseIO.java @@ -457,7 +457,7 @@ public void populateDisplayData(DisplayData.Builder builder) { } @Override - public Coder getDefaultOutputCoder() { + public Coder getOutputCoder() { return HBaseResultCoder.of(); } } diff --git a/sdks/java/io/hcatalog/src/main/java/org/apache/beam/sdk/io/hcatalog/HCatalogIO.java b/sdks/java/io/hcatalog/src/main/java/org/apache/beam/sdk/io/hcatalog/HCatalogIO.java index 4199b805c0e78..d8e462ba80e0d 100644 --- a/sdks/java/io/hcatalog/src/main/java/org/apache/beam/sdk/io/hcatalog/HCatalogIO.java +++ b/sdks/java/io/hcatalog/src/main/java/org/apache/beam/sdk/io/hcatalog/HCatalogIO.java @@ -210,7 +210,7 @@ static class BoundedHCatalogSource extends BoundedSource { @Override @SuppressWarnings({"unchecked", "rawtypes"}) - public Coder getDefaultOutputCoder() { + public Coder getOutputCoder() { return (Coder) WritableCoder.of(DefaultHCatRecord.class); } 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 f8cba5e0d8e17..2af0ce947ca60 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 @@ -373,7 +373,7 @@ public Coder getCheckpointMarkCoder() { } @Override - public Coder getDefaultOutputCoder() { + public Coder getOutputCoder() { return SerializableCoder.of(JmsRecord.class); } 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 026313ab2a8cb..7fb4260313c7a 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 @@ -844,7 +844,7 @@ public void validate() { } @Override - public Coder> getDefaultOutputCoder() { + public Coder> getOutputCoder() { return KafkaRecordCoder.of(spec.getKeyCoder(), spec.getValueCoder()); } } diff --git a/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/KinesisSource.java b/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/KinesisSource.java index 362792b941a47..144bd802243db 100644 --- a/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/KinesisSource.java +++ b/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/KinesisSource.java @@ -107,7 +107,7 @@ public void validate() { } @Override - public Coder getDefaultOutputCoder() { + public Coder getOutputCoder() { return KinesisRecordCoder.of(); } } 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 5b5412c9bee97..c612d5254ea3e 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 @@ -440,7 +440,7 @@ public void populateDisplayData(DisplayData.Builder builder) { } @Override - public Coder getDefaultOutputCoder() { + public Coder getOutputCoder() { return SerializableCoder.of(ObjectId.class); } 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 3b14182f27608..087123a45141d 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 @@ -277,7 +277,7 @@ private BoundedMongoDbSource(Read spec) { } @Override - public Coder getDefaultOutputCoder() { + public Coder getOutputCoder() { return SerializableCoder.of(Document.class); } diff --git a/sdks/java/io/mqtt/src/main/java/org/apache/beam/sdk/io/mqtt/MqttIO.java b/sdks/java/io/mqtt/src/main/java/org/apache/beam/sdk/io/mqtt/MqttIO.java index add5cb57f6103..5aadb80fc8011 100644 --- a/sdks/java/io/mqtt/src/main/java/org/apache/beam/sdk/io/mqtt/MqttIO.java +++ b/sdks/java/io/mqtt/src/main/java/org/apache/beam/sdk/io/mqtt/MqttIO.java @@ -387,7 +387,7 @@ public Coder getCheckpointMarkCoder() { } @Override - public Coder getDefaultOutputCoder() { + public Coder getOutputCoder() { return ByteArrayCoder.of(); } } diff --git a/sdks/java/io/xml/src/main/java/org/apache/beam/sdk/io/xml/XmlSource.java b/sdks/java/io/xml/src/main/java/org/apache/beam/sdk/io/xml/XmlSource.java index 7aa42c5731296..b893d430d07dc 100644 --- a/sdks/java/io/xml/src/main/java/org/apache/beam/sdk/io/xml/XmlSource.java +++ b/sdks/java/io/xml/src/main/java/org/apache/beam/sdk/io/xml/XmlSource.java @@ -85,7 +85,7 @@ public void populateDisplayData(DisplayData.Builder builder) { } @Override - public Coder getDefaultOutputCoder() { + public Coder getOutputCoder() { return JAXBCoder.of(spec.getRecordClass()); } From bb1bf3c19ca0baa2c04cec9863bfcaca2024f94e Mon Sep 17 00:00:00 2001 From: Eugene Kirpichov Date: Wed, 26 Jul 2017 17:14:58 -0700 Subject: [PATCH 238/346] Requires specifying a Coder on PCollection.createPrimitiveOutputInternal The coder can still be null, in which case it is left unspecified. --- .../apache/beam/runners/apex/ApexRunner.java | 5 +- .../FlattenPCollectionTranslator.java | 15 +- .../apex/translation/ParDoTranslator.java | 7 +- .../ApexGroupByKeyOperatorTest.java | 6 +- .../construction/PCollectionTranslation.java | 8 +- .../construction/PTransformReplacements.java | 6 + .../core/construction/PrimitiveCreate.java | 14 +- .../core/construction/SplittableParDo.java | 3 + .../UnboundedReadFromBoundedSource.java | 5 - .../construction/PTransformMatchersTest.java | 79 +++++----- .../construction/ReplacementOutputsTest.java | 14 +- .../core/GroupByKeyViaGroupByKeyOnly.java | 15 +- .../SplittableParDoViaKeyedWorkItems.java | 9 +- .../beam/runners/direct/DirectGroupByKey.java | 31 ++-- .../direct/ParDoMultiOverrideFactory.java | 3 + .../direct/TestStreamEvaluatorFactory.java | 8 +- .../runners/direct/ViewOverrideFactory.java | 5 +- .../runners/direct/CommittedResultTest.java | 26 +++- .../runners/direct/EvaluationContextTest.java | 8 +- .../flink/CreateStreamingFlinkView.java | 5 +- .../beam/runners/dataflow/AssignWindows.java | 4 +- .../runners/dataflow/BatchViewOverrides.java | 19 +-- .../runners/dataflow/CreateDataflowView.java | 5 +- .../beam/runners/dataflow/DataflowRunner.java | 147 +++++++++--------- .../dataflow/PrimitiveParDoSingleFactory.java | 12 +- .../DataflowPipelineTranslatorTest.java | 11 +- .../runners/dataflow/DataflowRunnerTest.java | 11 +- .../transforms/DataflowGroupByKeyTest.java | 12 +- .../dataflow/transforms/DataflowViewTest.java | 14 +- .../beam/runners/spark/io/CreateStream.java | 11 +- .../translation/StorageLevelPTransform.java | 10 +- .../util/SinglePrimitiveOutputPTransform.java | 51 ------ .../java/org/apache/beam/sdk/io/Read.java | 21 ++- .../apache/beam/sdk/testing/TestStream.java | 5 +- .../apache/beam/sdk/transforms/Flatten.java | 22 +-- .../beam/sdk/transforms/GroupByKey.java | 12 +- .../org/apache/beam/sdk/transforms/ParDo.java | 2 + .../org/apache/beam/sdk/transforms/View.java | 5 +- .../beam/sdk/transforms/windowing/Window.java | 2 +- .../apache/beam/sdk/values/PCollection.java | 9 +- .../beam/sdk/values/PCollectionTuple.java | 10 +- .../sdk/runners/TransformHierarchyTest.java | 41 ++--- .../beam/sdk/runners/TransformTreeTest.java | 12 +- .../beam/sdk/transforms/FlattenTest.java | 2 +- .../beam/sdk/transforms/GroupByKeyTest.java | 11 +- .../apache/beam/sdk/transforms/ViewTest.java | 11 +- .../beam/sdk/values/PCollectionTupleTest.java | 7 +- 47 files changed, 357 insertions(+), 394 deletions(-) delete mode 100644 runners/spark/src/main/java/org/apache/beam/runners/spark/util/SinglePrimitiveOutputPTransform.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 fd0a1c93d1e66..cee524ee29fa5 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 @@ -227,9 +227,8 @@ public static CreateApexPCollectionView of( @Override public PCollection expand(PCollection input) { - return PCollection.createPrimitiveOutputInternal( - input.getPipeline(), input.getWindowingStrategy(), input.isBounded()) - .setCoder(input.getCoder()); + return PCollection.createPrimitiveOutputInternal( + input.getPipeline(), input.getWindowingStrategy(), input.isBounded(), input.getCoder()); } public PCollectionView getView() { diff --git a/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/FlattenPCollectionTranslator.java b/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/FlattenPCollectionTranslator.java index 440b8015e1f07..189cb65638099 100644 --- a/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/FlattenPCollectionTranslator.java +++ b/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/FlattenPCollectionTranslator.java @@ -110,8 +110,12 @@ static void flattenCollections(List> collections, Map 2) { - PCollection intermediateCollection = intermediateCollection(collection, - collection.getCoder()); + PCollection intermediateCollection = + PCollection.createPrimitiveOutputInternal( + collection.getPipeline(), + collection.getWindowingStrategy(), + collection.isBounded(), + collection.getCoder()); context.addOperator(operator, operator.out, intermediateCollection); remainingCollections.add(intermediateCollection); } else { @@ -135,11 +139,4 @@ static void flattenCollections(List> collections, Map PCollection intermediateCollection(PCollection input, Coder outputCoder) { - PCollection output = PCollection.createPrimitiveOutputInternal(input.getPipeline(), - input.getWindowingStrategy(), input.isBounded()); - output.setCoder(outputCoder); - return output; - } - } diff --git a/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/ParDoTranslator.java b/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/ParDoTranslator.java index e46687a65858a..be11b023f1f6a 100644 --- a/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/ParDoTranslator.java +++ b/runners/apex/src/main/java/org/apache/beam/runners/apex/translation/ParDoTranslator.java @@ -241,8 +241,11 @@ private static PCollection unionSideInputs( } PCollection resultCollection = - FlattenPCollectionTranslator.intermediateCollection( - firstSideInput, firstSideInput.getCoder()); + PCollection.createPrimitiveOutputInternal( + firstSideInput.getPipeline(), + firstSideInput.getWindowingStrategy(), + firstSideInput.isBounded(), + firstSideInput.getCoder()); FlattenPCollectionTranslator.flattenCollections( sourceCollections, unionTags, resultCollection, context); return resultCollection; diff --git a/runners/apex/src/test/java/org/apache/beam/runners/apex/translation/ApexGroupByKeyOperatorTest.java b/runners/apex/src/test/java/org/apache/beam/runners/apex/translation/ApexGroupByKeyOperatorTest.java index 206b430017349..63a218b5c8052 100644 --- a/runners/apex/src/test/java/org/apache/beam/runners/apex/translation/ApexGroupByKeyOperatorTest.java +++ b/runners/apex/src/test/java/org/apache/beam/runners/apex/translation/ApexGroupByKeyOperatorTest.java @@ -59,9 +59,9 @@ public void testGlobalWindowMinTimestamp() throws Exception { WindowingStrategy ws = WindowingStrategy.of(FixedWindows.of( Duration.standardSeconds(10))); - PCollection> input = PCollection.createPrimitiveOutputInternal(pipeline, - ws, IsBounded.BOUNDED); - input.setCoder(KvCoder.of(StringUtf8Coder.of(), VarIntCoder.of())); + PCollection> input = + PCollection.createPrimitiveOutputInternal( + pipeline, ws, IsBounded.BOUNDED, KvCoder.of(StringUtf8Coder.of(), VarIntCoder.of())); ApexGroupByKeyOperator operator = new ApexGroupByKeyOperator<>(options, input, new ApexStateInternals.ApexStateBackend() diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/PCollectionTranslation.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/PCollectionTranslation.java index c0a5acf022a80..c256e4c63e9c9 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/PCollectionTranslation.java +++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/PCollectionTranslation.java @@ -52,10 +52,10 @@ public static PCollection fromProto( Coder coder = components.getCoder(pCollection.getCoderId()); return PCollection.createPrimitiveOutputInternal( - pipeline, - components.getWindowingStrategy(pCollection.getWindowingStrategyId()), - fromProto(pCollection.getIsBounded())) - .setCoder((Coder) coder); + pipeline, + components.getWindowingStrategy(pCollection.getWindowingStrategyId()), + fromProto(pCollection.getIsBounded()), + (Coder) coder); } public static IsBounded isBounded(RunnerApi.PCollection pCollection) { diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/PTransformReplacements.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/PTransformReplacements.java index 706a9564640bd..35bad1556c6bd 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/PTransformReplacements.java +++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/PTransformReplacements.java @@ -20,6 +20,7 @@ import static com.google.common.base.Preconditions.checkArgument; +import com.google.common.collect.Iterables; import java.util.Map; import java.util.Set; import org.apache.beam.sdk.runners.AppliedPTransform; @@ -66,4 +67,9 @@ private static PCollection getSingletonMainInput( ignoredTags); return mainInput; } + + public static PCollection getSingletonMainOutput( + AppliedPTransform, ? extends PTransform>> transform) { + return ((PCollection) Iterables.getOnlyElement(transform.getOutputs().values())); + } } diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/PrimitiveCreate.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/PrimitiveCreate.java index f43d23b88b8bd..62b6d0a9b3d78 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/PrimitiveCreate.java +++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/PrimitiveCreate.java @@ -18,7 +18,9 @@ package org.apache.beam.runners.core.construction; +import com.google.common.collect.Iterables; import java.util.Map; +import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.runners.AppliedPTransform; import org.apache.beam.sdk.runners.PTransformOverrideFactory; import org.apache.beam.sdk.transforms.Create; @@ -36,15 +38,17 @@ */ public class PrimitiveCreate extends PTransform> { private final Create.Values transform; + private final Coder coder; - private PrimitiveCreate(Create.Values transform) { + private PrimitiveCreate(Create.Values transform, Coder coder) { this.transform = transform; + this.coder = coder; } @Override public PCollection expand(PBegin input) { return PCollection.createPrimitiveOutputInternal( - input.getPipeline(), WindowingStrategy.globalDefault(), IsBounded.BOUNDED); + input.getPipeline(), WindowingStrategy.globalDefault(), IsBounded.BOUNDED, coder); } public Iterable getElements() { @@ -60,7 +64,11 @@ public static class Factory public PTransformReplacement> getReplacementTransform( AppliedPTransform, Values> transform) { return PTransformReplacement.of( - transform.getPipeline().begin(), new PrimitiveCreate(transform.getTransform())); + transform.getPipeline().begin(), + new PrimitiveCreate( + transform.getTransform(), + ((PCollection) Iterables.getOnlyElement(transform.getOutputs().values())) + .getCoder())); } @Override diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/SplittableParDo.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/SplittableParDo.java index e71187be6c256..bcc5de864187b 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/SplittableParDo.java +++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/SplittableParDo.java @@ -20,6 +20,7 @@ import static com.google.common.base.Preconditions.checkArgument; import java.io.IOException; +import java.util.Collections; import java.util.List; import java.util.Map; import java.util.UUID; @@ -273,6 +274,8 @@ public static PCollectionTuple createPrimitiveOutputFor( PCollectionTuple.ofPrimitiveOutputsInternal( input.getPipeline(), TupleTagList.of(mainOutputTag).and(additionalOutputTags.getAll()), + // TODO + Collections., Coder>emptyMap(), windowingStrategy, input.isBounded().and(signature.isBoundedPerElement())); diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/UnboundedReadFromBoundedSource.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/UnboundedReadFromBoundedSource.java index f35f4c3de08c2..55f95194cbc4d 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/UnboundedReadFromBoundedSource.java +++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/UnboundedReadFromBoundedSource.java @@ -90,11 +90,6 @@ public PCollection expand(PBegin input) { Read.from(new BoundedToUnboundedSourceAdapter<>(source))); } - @Override - protected Coder getDefaultOutputCoder() { - return source.getOutputCoder(); - } - @Override public String getKindString() { return String.format("Read(%s)", NameUtils.approximateSimpleName(source)); diff --git a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/PTransformMatchersTest.java b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/PTransformMatchersTest.java index 18626997000a3..fa7e1e915919d 100644 --- a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/PTransformMatchersTest.java +++ b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/PTransformMatchersTest.java @@ -100,15 +100,16 @@ public class PTransformMatchersTest implements Serializable { private AppliedPTransform getAppliedTransform(PTransform pardo) { PCollection> input = PCollection.createPrimitiveOutputInternal( - p, WindowingStrategy.globalDefault(), IsBounded.BOUNDED); + p, + WindowingStrategy.globalDefault(), + IsBounded.BOUNDED, + KvCoder.of(StringUtf8Coder.of(), VarIntCoder.of())); input.setName("dummy input"); - input.setCoder(KvCoder.of(StringUtf8Coder.of(), VarIntCoder.of())); PCollection output = PCollection.createPrimitiveOutputInternal( - p, WindowingStrategy.globalDefault(), IsBounded.BOUNDED); + p, WindowingStrategy.globalDefault(), IsBounded.BOUNDED, VarIntCoder.of()); output.setName("dummy output"); - output.setCoder(VarIntCoder.of()); return AppliedPTransform.of("pardo", input.expand(), output.expand(), pardo, p); } @@ -133,7 +134,7 @@ class MyPTransform extends PTransform>, PCollect @Override public PCollection expand(PCollection> input) { return PCollection.createPrimitiveOutputInternal( - input.getPipeline(), input.getWindowingStrategy(), input.isBounded()); + input.getPipeline(), input.getWindowingStrategy(), input.isBounded(), VarIntCoder.of()); } } PTransformMatcher matcher = PTransformMatchers.classEqualTo(MyPTransform.class); @@ -425,14 +426,14 @@ public void createViewWithViewFnNotCreatePCollectionView() { public void emptyFlattenWithEmptyFlatten() { AppliedPTransform application = AppliedPTransform - ., PCollection, Flatten.PCollections>of( + ., PCollection, Flatten.PCollections>of( "EmptyFlatten", Collections., PValue>emptyMap(), Collections., PValue>singletonMap( - new TupleTag(), + new TupleTag(), PCollection.createPrimitiveOutputInternal( - p, WindowingStrategy.globalDefault(), IsBounded.BOUNDED)), - Flatten.pCollections(), + p, WindowingStrategy.globalDefault(), IsBounded.BOUNDED, VarIntCoder.of())), + Flatten.pCollections(), p); assertThat(PTransformMatchers.emptyFlatten().matches(application), is(true)); @@ -442,17 +443,17 @@ public void emptyFlattenWithEmptyFlatten() { public void emptyFlattenWithNonEmptyFlatten() { AppliedPTransform application = AppliedPTransform - ., PCollection, Flatten.PCollections>of( + ., PCollection, Flatten.PCollections>of( "Flatten", Collections., PValue>singletonMap( - new TupleTag(), + new TupleTag(), PCollection.createPrimitiveOutputInternal( - p, WindowingStrategy.globalDefault(), IsBounded.BOUNDED)), + p, WindowingStrategy.globalDefault(), IsBounded.BOUNDED, VarIntCoder.of())), Collections., PValue>singletonMap( - new TupleTag(), + new TupleTag(), PCollection.createPrimitiveOutputInternal( - p, WindowingStrategy.globalDefault(), IsBounded.BOUNDED)), - Flatten.pCollections(), + p, WindowingStrategy.globalDefault(), IsBounded.BOUNDED, VarIntCoder.of())), + Flatten.pCollections(), p); assertThat(PTransformMatchers.emptyFlatten().matches(application), is(false)); @@ -462,15 +463,15 @@ public void emptyFlattenWithNonEmptyFlatten() { public void emptyFlattenWithNonFlatten() { AppliedPTransform application = AppliedPTransform - .>, PCollection, Flatten.Iterables>of( + .>, PCollection, Flatten.Iterables>of( "EmptyFlatten", Collections., PValue>emptyMap(), Collections., PValue>singletonMap( - new TupleTag(), + new TupleTag(), PCollection.createPrimitiveOutputInternal( - p, WindowingStrategy.globalDefault(), IsBounded.BOUNDED)), - Flatten.iterables() /* This isn't actually possible to construct, - * but for the sake of example */, + p, WindowingStrategy.globalDefault(), IsBounded.BOUNDED, VarIntCoder.of())), + /* This isn't actually possible to construct, but for the sake of example */ + Flatten.iterables(), p); assertThat(PTransformMatchers.emptyFlatten().matches(application), is(false)); @@ -480,17 +481,17 @@ public void emptyFlattenWithNonFlatten() { public void flattenWithDuplicateInputsWithoutDuplicates() { AppliedPTransform application = AppliedPTransform - ., PCollection, Flatten.PCollections>of( + ., PCollection, Flatten.PCollections>of( "Flatten", Collections., PValue>singletonMap( - new TupleTag(), + new TupleTag(), PCollection.createPrimitiveOutputInternal( - p, WindowingStrategy.globalDefault(), IsBounded.BOUNDED)), + p, WindowingStrategy.globalDefault(), IsBounded.BOUNDED, VarIntCoder.of())), Collections., PValue>singletonMap( - new TupleTag(), + new TupleTag(), PCollection.createPrimitiveOutputInternal( - p, WindowingStrategy.globalDefault(), IsBounded.BOUNDED)), - Flatten.pCollections(), + p, WindowingStrategy.globalDefault(), IsBounded.BOUNDED, VarIntCoder.of())), + Flatten.pCollections(), p); assertThat(PTransformMatchers.flattenWithDuplicateInputs().matches(application), is(false)); @@ -498,22 +499,22 @@ public void flattenWithDuplicateInputsWithoutDuplicates() { @Test public void flattenWithDuplicateInputsWithDuplicates() { - PCollection duplicate = + PCollection duplicate = PCollection.createPrimitiveOutputInternal( - p, WindowingStrategy.globalDefault(), IsBounded.BOUNDED); + p, WindowingStrategy.globalDefault(), IsBounded.BOUNDED, VarIntCoder.of()); AppliedPTransform application = AppliedPTransform - ., PCollection, Flatten.PCollections>of( + ., PCollection, Flatten.PCollections>of( "Flatten", ImmutableMap., PValue>builder() - .put(new TupleTag(), duplicate) - .put(new TupleTag(), duplicate) + .put(new TupleTag(), duplicate) + .put(new TupleTag(), duplicate) .build(), Collections., PValue>singletonMap( - new TupleTag(), + new TupleTag(), PCollection.createPrimitiveOutputInternal( - p, WindowingStrategy.globalDefault(), IsBounded.BOUNDED)), - Flatten.pCollections(), + p, WindowingStrategy.globalDefault(), IsBounded.BOUNDED, VarIntCoder.of())), + Flatten.pCollections(), p); assertThat(PTransformMatchers.flattenWithDuplicateInputs().matches(application), is(true)); @@ -523,15 +524,15 @@ public void flattenWithDuplicateInputsWithDuplicates() { public void flattenWithDuplicateInputsNonFlatten() { AppliedPTransform application = AppliedPTransform - .>, PCollection, Flatten.Iterables>of( + .>, PCollection, Flatten.Iterables>of( "EmptyFlatten", Collections., PValue>emptyMap(), Collections., PValue>singletonMap( - new TupleTag(), + new TupleTag(), PCollection.createPrimitiveOutputInternal( - p, WindowingStrategy.globalDefault(), IsBounded.BOUNDED)), - Flatten.iterables() /* This isn't actually possible to construct, - * but for the sake of example */, + p, WindowingStrategy.globalDefault(), IsBounded.BOUNDED, VarIntCoder.of())), + /* This isn't actually possible to construct, but for the sake of example */ + Flatten.iterables(), p); assertThat(PTransformMatchers.flattenWithDuplicateInputs().matches(application), is(false)); diff --git a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/ReplacementOutputsTest.java b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/ReplacementOutputsTest.java index f8d01e9a016ff..0165e4bbe420d 100644 --- a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/ReplacementOutputsTest.java +++ b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/ReplacementOutputsTest.java @@ -24,6 +24,8 @@ import com.google.common.collect.ImmutableMap; import com.google.common.collect.Iterables; import java.util.Map; +import org.apache.beam.sdk.coders.StringUtf8Coder; +import org.apache.beam.sdk.coders.VarIntCoder; import org.apache.beam.sdk.runners.PTransformOverrideFactory.ReplacementOutput; import org.apache.beam.sdk.testing.TestPipeline; import org.apache.beam.sdk.values.PCollection; @@ -50,23 +52,23 @@ public class ReplacementOutputsTest { private PCollection ints = PCollection.createPrimitiveOutputInternal( - p, WindowingStrategy.globalDefault(), IsBounded.BOUNDED); + p, WindowingStrategy.globalDefault(), IsBounded.BOUNDED, VarIntCoder.of()); private PCollection moreInts = PCollection.createPrimitiveOutputInternal( - p, WindowingStrategy.globalDefault(), IsBounded.BOUNDED); + p, WindowingStrategy.globalDefault(), IsBounded.BOUNDED, VarIntCoder.of()); private PCollection strs = PCollection.createPrimitiveOutputInternal( - p, WindowingStrategy.globalDefault(), IsBounded.BOUNDED); + p, WindowingStrategy.globalDefault(), IsBounded.BOUNDED, StringUtf8Coder.of()); private PCollection replacementInts = PCollection.createPrimitiveOutputInternal( - p, WindowingStrategy.globalDefault(), IsBounded.BOUNDED); + p, WindowingStrategy.globalDefault(), IsBounded.BOUNDED, VarIntCoder.of()); private PCollection moreReplacementInts = PCollection.createPrimitiveOutputInternal( - p, WindowingStrategy.globalDefault(), IsBounded.BOUNDED); + p, WindowingStrategy.globalDefault(), IsBounded.BOUNDED, VarIntCoder.of()); private PCollection replacementStrs = PCollection.createPrimitiveOutputInternal( - p, WindowingStrategy.globalDefault(), IsBounded.BOUNDED); + p, WindowingStrategy.globalDefault(), IsBounded.BOUNDED, StringUtf8Coder.of()); @Test public void singletonSucceeds() { 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 fca3c76423d20..1fdf07c12b220 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 @@ -111,12 +111,10 @@ public static class GroupByKeyOnly @Override public PCollection>>> expand(PCollection> input) { return PCollection.createPrimitiveOutputInternal( - input.getPipeline(), input.getWindowingStrategy(), input.isBounded()); - } - - @Override - public Coder>> getDefaultOutputCoder(PCollection> input) { - return GroupByKey.getOutputKvCoder(input.getCoder()); + input.getPipeline(), + input.getWindowingStrategy(), + input.isBounded(), + (Coder) GroupByKey.getOutputKvCoder(input.getCoder())); } } @@ -244,9 +242,8 @@ public PCollection>> expand( Coder> outputValueCoder = IterableCoder.of(inputIterableElementValueCoder); Coder>> outputKvCoder = KvCoder.of(keyCoder, outputValueCoder); - return PCollection.>>createPrimitiveOutputInternal( - input.getPipeline(), windowingStrategy, input.isBounded()) - .setCoder(outputKvCoder); + return PCollection.createPrimitiveOutputInternal( + input.getPipeline(), windowingStrategy, input.isBounded(), outputKvCoder); } } } diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/SplittableParDoViaKeyedWorkItems.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/SplittableParDoViaKeyedWorkItems.java index 6e976455a9414..af720fd591832 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/SplittableParDoViaKeyedWorkItems.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/SplittableParDoViaKeyedWorkItems.java @@ -72,8 +72,15 @@ public static class GBKIntoKeyedWorkItems PCollection>, PCollection>> { @Override public PCollection> expand(PCollection> input) { + KvCoder kvCoder = (KvCoder) input.getCoder(); return PCollection.createPrimitiveOutputInternal( - input.getPipeline(), WindowingStrategy.globalDefault(), input.isBounded()); + input.getPipeline(), + WindowingStrategy.globalDefault(), + input.isBounded(), + KeyedWorkItemCoder.of( + kvCoder.getKeyCoder(), + kvCoder.getValueCoder(), + input.getWindowingStrategy().getWindowFn().windowCoder())); } @Override 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 06b8e29962be4..3ba04e7c5e579 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 @@ -24,7 +24,6 @@ import org.apache.beam.runners.core.KeyedWorkItemCoder; import org.apache.beam.runners.core.construction.ForwardingPTransform; import org.apache.beam.runners.core.construction.PTransformTranslation; -import org.apache.beam.sdk.coders.CannotProvideCoderException; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.IterableCoder; import org.apache.beam.sdk.coders.KvCoder; @@ -78,21 +77,17 @@ static final class DirectGroupByKeyOnly @Override public PCollection> expand(PCollection> input) { return PCollection.createPrimitiveOutputInternal( - input.getPipeline(), WindowingStrategy.globalDefault(), input.isBounded()); + input.getPipeline(), + WindowingStrategy.globalDefault(), + input.isBounded(), + KeyedWorkItemCoder.of( + GroupByKey.getKeyCoder(input.getCoder()), + GroupByKey.getInputValueCoder(input.getCoder()), + input.getWindowingStrategy().getWindowFn().windowCoder())); } DirectGroupByKeyOnly() {} - @Override - protected Coder getDefaultOutputCoder( - @SuppressWarnings("unused") PCollection> input) - throws CannotProvideCoderException { - return KeyedWorkItemCoder.of( - GroupByKey.getKeyCoder(input.getCoder()), - GroupByKey.getInputValueCoder(input.getCoder()), - input.getWindowingStrategy().getWindowFn().windowCoder()); - } - @Override public String getUrn() { return DIRECT_GBKO_URN; @@ -134,18 +129,12 @@ public Coder getValueCoder(Coder> inputCoder) { return getKeyedWorkItemCoder(inputCoder).getElementCoder(); } - @Override - protected Coder getDefaultOutputCoder( - @SuppressWarnings("unused") PCollection> input) - throws CannotProvideCoderException { - KeyedWorkItemCoder inputCoder = getKeyedWorkItemCoder(input.getCoder()); - return KvCoder.of(inputCoder.getKeyCoder(), IterableCoder.of(inputCoder.getElementCoder())); - } - @Override public PCollection>> expand(PCollection> input) { + KeyedWorkItemCoder inputCoder = getKeyedWorkItemCoder(input.getCoder()); return PCollection.createPrimitiveOutputInternal( - input.getPipeline(), outputWindowingStrategy, input.isBounded()); + input.getPipeline(), outputWindowingStrategy, input.isBounded(), + KvCoder.of(inputCoder.getKeyCoder(), IterableCoder.of(inputCoder.getElementCoder()))); } @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 891d1020787b4..3f04b56123ae0 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,6 +20,7 @@ import static com.google.common.base.Preconditions.checkState; import java.io.IOException; +import java.util.Collections; import java.util.List; import java.util.Map; import org.apache.beam.runners.core.KeyedWorkItem; @@ -248,6 +249,8 @@ public PCollectionTuple expand(PCollection, Coder>emptyMap(), input.getWindowingStrategy(), input.isBounded()); 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 16c8589c5f749..49e7be75a8370 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 @@ -207,9 +207,11 @@ static class DirectTestStream @Override public PCollection expand(PBegin input) { runner.setClockSupplier(new TestClockSupplier()); - return PCollection.createPrimitiveOutputInternal( - input.getPipeline(), WindowingStrategy.globalDefault(), IsBounded.UNBOUNDED) - .setCoder(original.getValueCoder()); + return PCollection.createPrimitiveOutputInternal( + input.getPipeline(), + WindowingStrategy.globalDefault(), + IsBounded.UNBOUNDED, + original.getValueCoder()); } @Override diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ViewOverrideFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ViewOverrideFactory.java index 5dcf0165c0f07..c2255fe4b2532 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ViewOverrideFactory.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ViewOverrideFactory.java @@ -115,9 +115,8 @@ static final class WriteView @Override @SuppressWarnings("deprecation") public PCollection> expand(PCollection> input) { - return PCollection.>createPrimitiveOutputInternal( - input.getPipeline(), input.getWindowingStrategy(), input.isBounded()) - .setCoder(input.getCoder()); + return PCollection.createPrimitiveOutputInternal( + input.getPipeline(), input.getWindowingStrategy(), input.isBounded(), input.getCoder()); } @SuppressWarnings("deprecation") 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 8b95b345d5548..29ed55defd354 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 @@ -27,6 +27,7 @@ import java.util.EnumSet; import java.util.List; import org.apache.beam.runners.direct.CommittedResult.OutputType; +import org.apache.beam.sdk.coders.VarIntCoder; import org.apache.beam.sdk.runners.AppliedPTransform; import org.apache.beam.sdk.testing.TestPipeline; import org.apache.beam.sdk.transforms.Create; @@ -113,13 +114,24 @@ public void getUncommittedElementsNull() { @Test public void getOutputsEqualInput() { - List> outputs = - ImmutableList.of(bundleFactory.createBundle(PCollection.createPrimitiveOutputInternal(p, - WindowingStrategy.globalDefault(), - PCollection.IsBounded.BOUNDED)).commit(Instant.now()), - bundleFactory.createBundle(PCollection.createPrimitiveOutputInternal(p, - WindowingStrategy.globalDefault(), - PCollection.IsBounded.UNBOUNDED)).commit(Instant.now())); + List> outputs = + ImmutableList.of( + bundleFactory + .createBundle( + PCollection.createPrimitiveOutputInternal( + p, + WindowingStrategy.globalDefault(), + PCollection.IsBounded.BOUNDED, + VarIntCoder.of())) + .commit(Instant.now()), + bundleFactory + .createBundle( + PCollection.createPrimitiveOutputInternal( + p, + WindowingStrategy.globalDefault(), + PCollection.IsBounded.UNBOUNDED, + VarIntCoder.of())) + .commit(Instant.now())); CommittedResult result = CommittedResult.create( StepTransformResult.withoutHold(transform).build(), 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 699a31870d65e..cc9ce60f577cc 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 @@ -40,6 +40,7 @@ import org.apache.beam.runners.direct.WatermarkManager.FiredTimers; import org.apache.beam.runners.direct.WatermarkManager.TimerUpdate; import org.apache.beam.sdk.coders.ByteArrayCoder; +import org.apache.beam.sdk.coders.IterableCoder; import org.apache.beam.sdk.coders.StringUtf8Coder; import org.apache.beam.sdk.coders.VarIntCoder; import org.apache.beam.sdk.io.GenerateSequence; @@ -127,8 +128,11 @@ public void setup() { public void writeToViewWriterThenReadReads() { PCollectionViewWriter> viewWriter = context.createPCollectionViewWriter( - PCollection.>createPrimitiveOutputInternal( - p, WindowingStrategy.globalDefault(), IsBounded.BOUNDED), + PCollection.createPrimitiveOutputInternal( + p, + WindowingStrategy.globalDefault(), + IsBounded.BOUNDED, + IterableCoder.of(VarIntCoder.of())), view); BoundedWindow window = new TestBoundedWindow(new Instant(1024L)); BoundedWindow second = new TestBoundedWindow(new Instant(899999L)); diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/CreateStreamingFlinkView.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/CreateStreamingFlinkView.java index 0cc3aec1d3aff..3114a6fb20413 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/CreateStreamingFlinkView.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/CreateStreamingFlinkView.java @@ -120,9 +120,8 @@ public static CreateFlinkPCollectionView of( @Override public PCollection> expand(PCollection> input) { - return PCollection.>createPrimitiveOutputInternal( - input.getPipeline(), input.getWindowingStrategy(), input.isBounded()) - .setCoder(input.getCoder()); + return PCollection.createPrimitiveOutputInternal( + input.getPipeline(), input.getWindowingStrategy(), input.isBounded(), input.getCoder()); } public PCollectionView getView() { diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/AssignWindows.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/AssignWindows.java index 572b00561770e..d015d2bbebf23 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/AssignWindows.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/AssignWindows.java @@ -59,8 +59,8 @@ public PCollection expand(PCollection input) { transform.getOutputStrategyInternal(input.getWindowingStrategy()); if (transform.getWindowFn() != null) { // If the windowFn changed, we create a primitive, and run the AssignWindows operation here. - return PCollection.createPrimitiveOutputInternal( - input.getPipeline(), outputStrategy, input.isBounded()); + return PCollection.createPrimitiveOutputInternal( + input.getPipeline(), outputStrategy, input.isBounded(), input.getCoder()); } else { // If the windowFn didn't change, we just run a pass-through transform and then set the // new windowing strategy. diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/BatchViewOverrides.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/BatchViewOverrides.java index ad3faed1cca14..9a77b4b20eb32 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/BatchViewOverrides.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/BatchViewOverrides.java @@ -1258,18 +1258,13 @@ static class GroupByKeyAndSortValuesOnly @Override public PCollection>>> expand(PCollection>> input) { - PCollection>>> rval = - PCollection.>>>createPrimitiveOutputInternal( - input.getPipeline(), - WindowingStrategy.globalDefault(), - IsBounded.BOUNDED); - - @SuppressWarnings({"unchecked", "rawtypes"}) - KvCoder> inputCoder = (KvCoder) input.getCoder(); - rval.setCoder( - KvCoder.of(inputCoder.getKeyCoder(), - IterableCoder.of(inputCoder.getValueCoder()))); - return rval; + @SuppressWarnings("unchecked") + KvCoder> inputCoder = (KvCoder>) input.getCoder(); + return PCollection.createPrimitiveOutputInternal( + input.getPipeline(), + WindowingStrategy.globalDefault(), + IsBounded.BOUNDED, + KvCoder.of(inputCoder.getKeyCoder(), IterableCoder.of(inputCoder.getValueCoder()))); } } diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/CreateDataflowView.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/CreateDataflowView.java index caad7f8406c81..3b01d6931e053 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/CreateDataflowView.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/CreateDataflowView.java @@ -37,9 +37,8 @@ private CreateDataflowView(PCollectionView view) { @Override public PCollection expand(PCollection input) { - return PCollection.createPrimitiveOutputInternal( - input.getPipeline(), input.getWindowingStrategy(), input.isBounded()) - .setCoder(input.getCoder()); + return PCollection.createPrimitiveOutputInternal( + input.getPipeline(), input.getWindowingStrategy(), input.isBounded(), input.getCoder()); } public PCollectionView getView() { 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 8fce5b4aa714c..6999616f667a8 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 @@ -321,7 +321,7 @@ private List getOverrides(boolean streaming) { overridesBuilder.add( PTransformOverride.of( PTransformMatchers.classEqualTo(PubsubUnboundedSource.class), - new ReflectiveRootOverrideFactory(StreamingPubsubIORead.class, this))); + new StreamingPubsubIOReadOverrideFactory())); } if (!hasExperiment(options, "enable_custom_pubsub_sink")) { overridesBuilder.add( @@ -359,11 +359,11 @@ private List getOverrides(boolean streaming) { // must precede it PTransformOverride.of( PTransformMatchers.classEqualTo(Read.Bounded.class), - new ReflectiveRootOverrideFactory(StreamingBoundedRead.class, this))) + new StreamingBoundedReadOverrideFactory())) .add( PTransformOverride.of( PTransformMatchers.classEqualTo(Read.Unbounded.class), - new ReflectiveRootOverrideFactory(StreamingUnboundedRead.class, this))) + new StreamingUnboundedReadOverrideFactory())) .add( PTransformOverride.of( PTransformMatchers.classEqualTo(View.CreatePCollectionView.class), @@ -448,38 +448,6 @@ public PTransformReplacement, PCollection> getRepla } } - private static class ReflectiveRootOverrideFactory - implements PTransformOverrideFactory< - PBegin, PCollection, PTransform>> { - private final Class>> replacement; - private final DataflowRunner runner; - - private ReflectiveRootOverrideFactory( - Class>> replacement, DataflowRunner runner) { - this.replacement = replacement; - this.runner = runner; - } - - @Override - public PTransformReplacement> getReplacementTransform( - AppliedPTransform, PTransform>> transform) { - PTransform> original = transform.getTransform(); - return PTransformReplacement.of( - transform.getPipeline().begin(), - InstanceBuilder.ofType(replacement) - .withArg(DataflowRunner.class, runner) - .withArg( - (Class>>) original.getClass(), original) - .build()); - } - - @Override - public Map mapOutputs( - Map, PValue> outputs, PCollection newOutput) { - return ReplacementOutputs.singleton(outputs, newOutput); - } - } - private String debuggerMessage(String projectId, String uniquifier) { return String.format("To debug your job, visit Google Cloud Debugger at: " + "https://console.developers.google.com/debug?project=%s&dbgee=%s", @@ -838,6 +806,24 @@ void recordViewUsesNonDeterministicKeyCoder(PTransform ptransform) { // PubsubIO translations // ================================================================================ + private static class StreamingPubsubIOReadOverrideFactory + implements PTransformOverrideFactory< + PBegin, PCollection, PubsubUnboundedSource> { + @Override + public PTransformReplacement> getReplacementTransform( + AppliedPTransform, PubsubUnboundedSource> transform) { + return PTransformReplacement.of( + transform.getPipeline().begin(), new StreamingPubsubIORead(transform.getTransform())); + } + + @Override + public Map mapOutputs( + Map, PValue> outputs, PCollection newOutput) { + return ReplacementOutputs.singleton(outputs, newOutput); + } + } + + /** * Suppress application of {@link PubsubUnboundedSource#expand} in streaming mode so that we can * instead defer to Windmill's implementation. @@ -846,9 +832,7 @@ private static class StreamingPubsubIORead extends PTransform> { private final PubsubUnboundedSource transform; - /** Builds an instance of this class from the overridden transform. */ - @SuppressWarnings("unused") // used via reflection in DataflowRunner#apply() - public StreamingPubsubIORead(DataflowRunner runner, PubsubUnboundedSource transform) { + public StreamingPubsubIORead(PubsubUnboundedSource transform) { this.transform = transform; } @@ -858,9 +842,11 @@ PubsubUnboundedSource getOverriddenTransform() { @Override public PCollection expand(PBegin input) { - return PCollection.createPrimitiveOutputInternal( - input.getPipeline(), WindowingStrategy.globalDefault(), IsBounded.UNBOUNDED) - .setCoder(new PubsubMessageWithAttributesCoder()); + return PCollection.createPrimitiveOutputInternal( + input.getPipeline(), + WindowingStrategy.globalDefault(), + IsBounded.UNBOUNDED, + new PubsubMessageWithAttributesCoder()); } @Override @@ -1129,12 +1115,7 @@ private Impulse(IsBounded isBounded) { @Override public PCollection expand(PBegin input) { return PCollection.createPrimitiveOutputInternal( - input.getPipeline(), WindowingStrategy.globalDefault(), isBounded); - } - - @Override - protected Coder getDefaultOutputCoder() { - return ByteArrayCoder.of(); + input.getPipeline(), WindowingStrategy.globalDefault(), isBounded, ByteArrayCoder.of()); } private static class Translator implements TransformTranslator { @@ -1157,6 +1138,22 @@ public void translate(Impulse transform, TranslationContext context) { } } + private static class StreamingUnboundedReadOverrideFactory + implements PTransformOverrideFactory, Read.Unbounded> { + @Override + public PTransformReplacement> getReplacementTransform( + AppliedPTransform, Read.Unbounded> transform) { + return PTransformReplacement.of( + transform.getPipeline().begin(), new StreamingUnboundedRead<>(transform.getTransform())); + } + + @Override + public Map mapOutputs( + Map, PValue> outputs, PCollection newOutput) { + return ReplacementOutputs.singleton(outputs, newOutput); + } + } + /** * Specialized implementation for * {@link org.apache.beam.sdk.io.Read.Unbounded Read.Unbounded} for the @@ -1168,17 +1165,10 @@ public void translate(Impulse transform, TranslationContext context) { private static class StreamingUnboundedRead extends PTransform> { private final UnboundedSource source; - /** Builds an instance of this class from the overridden transform. */ - @SuppressWarnings("unused") // used via reflection in DataflowRunner#apply() - public StreamingUnboundedRead(DataflowRunner runner, Read.Unbounded transform) { + public StreamingUnboundedRead(Read.Unbounded transform) { this.source = transform.getSource(); } - @Override - protected Coder getDefaultOutputCoder() { - return source.getOutputCoder(); - } - @Override public final PCollection expand(PBegin input) { source.validate(); @@ -1206,13 +1196,9 @@ private ReadWithIds(UnboundedSource source) { @Override public final PCollection> expand(PInput input) { - return PCollection.>createPrimitiveOutputInternal( - input.getPipeline(), WindowingStrategy.globalDefault(), IsBounded.UNBOUNDED); - } - - @Override - protected Coder> getDefaultOutputCoder() { - return ValueWithRecordId.ValueWithRecordIdCoder.of(source.getOutputCoder()); + return PCollection.createPrimitiveOutputInternal( + input.getPipeline(), WindowingStrategy.globalDefault(), IsBounded.UNBOUNDED, + ValueWithRecordId.ValueWithRecordIdCoder.of(source.getOutputCoder())); } @Override @@ -1276,6 +1262,22 @@ public void processElement(ProcessContext c) { } } + private static class StreamingBoundedReadOverrideFactory + implements PTransformOverrideFactory, Read.Bounded> { + @Override + public PTransformReplacement> getReplacementTransform( + AppliedPTransform, Read.Bounded> transform) { + return PTransformReplacement.of( + transform.getPipeline().begin(), new StreamingBoundedRead<>(transform.getTransform())); + } + + @Override + public Map mapOutputs( + Map, PValue> outputs, PCollection newOutput) { + return ReplacementOutputs.singleton(outputs, newOutput); + } + } + /** * Specialized implementation for {@link org.apache.beam.sdk.io.Read.Bounded Read.Bounded} for the * Dataflow runner in streaming mode. @@ -1283,17 +1285,10 @@ public void processElement(ProcessContext c) { private static class StreamingBoundedRead extends PTransform> { private final BoundedSource source; - /** Builds an instance of this class from the overridden transform. */ - @SuppressWarnings("unused") // used via reflection in DataflowRunner#apply() - public StreamingBoundedRead(DataflowRunner runner, Read.Bounded transform) { + public StreamingBoundedRead(Read.Bounded transform) { this.source = transform.getSource(); } - @Override - protected Coder getDefaultOutputCoder() { - return source.getOutputCoder(); - } - @Override public final PCollection expand(PBegin input) { source.validate(); @@ -1404,15 +1399,19 @@ private String getJobIdFromName(String jobName) { static class CombineGroupedValues extends PTransform>>, PCollection>> { private final Combine.GroupedValues original; + private final Coder> outputCoder; - CombineGroupedValues(GroupedValues original) { + CombineGroupedValues( + GroupedValues original, Coder> outputCoder) { this.original = original; + this.outputCoder = outputCoder; } @Override public PCollection> expand(PCollection>> input) { return PCollection.createPrimitiveOutputInternal( - input.getPipeline(), input.getWindowingStrategy(), input.isBounded()); + input.getPipeline(), input.getWindowingStrategy(), input.isBounded(), + outputCoder); } public Combine.GroupedValues getOriginalCombine() { @@ -1433,7 +1432,9 @@ private static class PrimitiveCombineGroupedValuesOverrideFactory(transform.getTransform())); + new CombineGroupedValues<>( + transform.getTransform(), + PTransformReplacements.getSingletonMainOutput(transform).getCoder())); } @Override diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/PrimitiveParDoSingleFactory.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/PrimitiveParDoSingleFactory.java index 8611d3cad945a..9252c6451471d 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/PrimitiveParDoSingleFactory.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/PrimitiveParDoSingleFactory.java @@ -22,6 +22,7 @@ import org.apache.beam.runners.core.construction.ForwardingPTransform; import org.apache.beam.runners.core.construction.PTransformReplacements; import org.apache.beam.runners.core.construction.SingleInputOutputOverrideFactory; +import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.common.runner.v1.RunnerApi.DisplayData; import org.apache.beam.sdk.runners.AppliedPTransform; import org.apache.beam.sdk.runners.PTransformOverrideFactory; @@ -49,7 +50,9 @@ public class PrimitiveParDoSingleFactory transform) { return PTransformReplacement.of( PTransformReplacements.getSingletonMainInput(transform), - new ParDoSingle<>(transform.getTransform())); + new ParDoSingle<>( + transform.getTransform(), + PTransformReplacements.getSingletonMainOutput(transform).getCoder())); } /** @@ -58,15 +61,18 @@ public class PrimitiveParDoSingleFactory public static class ParDoSingle extends ForwardingPTransform, PCollection> { private final ParDo.SingleOutput original; + private final Coder outputCoder; - private ParDoSingle(ParDo.SingleOutput original) { + private ParDoSingle(SingleOutput original, Coder outputCoder) { this.original = original; + this.outputCoder = outputCoder; } @Override public PCollection expand(PCollection input) { return PCollection.createPrimitiveOutputInternal( - input.getPipeline(), input.getWindowingStrategy(), input.isBounded()); + input.getPipeline(), input.getWindowingStrategy(), input.isBounded(), + outputCoder); } public DoFn getFn() { 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 9a0bdf80862dc..7a99f750cf097 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 @@ -650,10 +650,13 @@ public PCollectionTuple expand(PCollection input) { // Fails here when attempting to construct a tuple with an unbound object. return PCollectionTuple.of(sumTag, sum) - .and(doneTag, PCollection.createPrimitiveOutputInternal( - input.getPipeline(), - WindowingStrategy.globalDefault(), - input.isBounded())); + .and( + doneTag, + PCollection.createPrimitiveOutputInternal( + input.getPipeline(), + WindowingStrategy.globalDefault(), + input.isBounded(), + VoidCoder.of())); } } 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 9db73c6501b86..55264a1f6f767 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 @@ -72,7 +72,6 @@ import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.Pipeline.PipelineVisitor; import org.apache.beam.sdk.coders.BigEndianIntegerCoder; -import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.VoidCoder; import org.apache.beam.sdk.extensions.gcp.auth.NoopCredentialFactory; import org.apache.beam.sdk.extensions.gcp.auth.TestCredential; @@ -953,15 +952,11 @@ public static class TestTransform @Override public PCollection expand(PCollection input) { - return PCollection.createPrimitiveOutputInternal( + return PCollection.createPrimitiveOutputInternal( input.getPipeline(), WindowingStrategy.globalDefault(), - input.isBounded()); - } - - @Override - protected Coder getDefaultOutputCoder(PCollection input) { - return input.getCoder(); + input.isBounded(), + input.getCoder()); } } 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 737b40813f33d..c198ebf631898 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 @@ -26,6 +26,7 @@ import org.apache.beam.sdk.coders.BigEndianIntegerCoder; 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.extensions.gcp.storage.NoopPathValidator; import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.transforms.Create; @@ -36,7 +37,6 @@ import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PBegin; import org.apache.beam.sdk.values.PCollection; -import org.apache.beam.sdk.values.TypeDescriptor; import org.apache.beam.sdk.values.WindowingStrategy; import org.joda.time.Duration; import org.junit.Before; @@ -105,11 +105,11 @@ public void testGroupByKeyServiceUnbounded() { new PTransform>>() { @Override public PCollection> expand(PBegin input) { - return PCollection.>createPrimitiveOutputInternal( - input.getPipeline(), - WindowingStrategy.globalDefault(), - PCollection.IsBounded.UNBOUNDED) - .setTypeDescriptor(new TypeDescriptor>() {}); + return PCollection.createPrimitiveOutputInternal( + input.getPipeline(), + WindowingStrategy.globalDefault(), + PCollection.IsBounded.UNBOUNDED, + KvCoder.of(StringUtf8Coder.of(), VarIntCoder.of())); } }); 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 dea96b96ea21e..e2e42a6684ab8 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 @@ -21,6 +21,9 @@ import org.apache.beam.runners.dataflow.DataflowRunner; import org.apache.beam.runners.dataflow.options.DataflowPipelineOptions; 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; import org.apache.beam.sdk.extensions.gcp.storage.NoopPathValidator; import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.transforms.Create; @@ -33,7 +36,6 @@ 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.TypeDescriptor; import org.apache.beam.sdk.values.WindowingStrategy; import org.hamcrest.Matchers; import org.joda.time.Duration; @@ -94,11 +96,11 @@ private void testViewUnbounded( new PTransform>>() { @Override public PCollection> expand(PBegin input) { - return PCollection.>createPrimitiveOutputInternal( - input.getPipeline(), - WindowingStrategy.globalDefault(), - PCollection.IsBounded.UNBOUNDED) - .setTypeDescriptor(new TypeDescriptor>() {}); + return PCollection.createPrimitiveOutputInternal( + input.getPipeline(), + WindowingStrategy.globalDefault(), + PCollection.IsBounded.UNBOUNDED, + KvCoder.of(StringUtf8Coder.of(), VarIntCoder.of())); } }) .apply(view); 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 fdcea99b91651..d485d25b019f7 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 @@ -27,7 +27,6 @@ import java.util.List; import java.util.Queue; import org.apache.beam.runners.spark.util.GlobalWatermarkHolder.SparkWatermarks; -import org.apache.beam.sdk.coders.CannotProvideCoderException; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; @@ -203,11 +202,9 @@ public Queue getTimes() { @Override public PCollection expand(PBegin input) { return PCollection.createPrimitiveOutputInternal( - input.getPipeline(), WindowingStrategy.globalDefault(), PCollection.IsBounded.UNBOUNDED); - } - - @Override - protected Coder getDefaultOutputCoder() throws CannotProvideCoderException { - return coder; + input.getPipeline(), + WindowingStrategy.globalDefault(), + PCollection.IsBounded.UNBOUNDED, + coder); } } 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 0ecfa7571ca4a..b236ce7f2594a 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 @@ -17,7 +17,6 @@ */ 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.values.PCollection; @@ -32,12 +31,7 @@ public final class StorageLevelPTransform extends PTransform, PCo public PCollection expand(PCollection input) { return PCollection.createPrimitiveOutputInternal(input.getPipeline(), WindowingStrategy.globalDefault(), - PCollection.IsBounded.BOUNDED); + PCollection.IsBounded.BOUNDED, + StringUtf8Coder.of()); } - - @Override - public Coder getDefaultOutputCoder() { - return StringUtf8Coder.of(); - } - } 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 deleted file mode 100644 index 299f5ba97a887..0000000000000 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/util/SinglePrimitiveOutputPTransform.java +++ /dev/null @@ -1,51 +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.util; - -import org.apache.beam.sdk.coders.CannotProvideCoderException; -import org.apache.beam.sdk.transforms.PTransform; -import org.apache.beam.sdk.values.PCollection; -import org.apache.beam.sdk.values.PCollection.IsBounded; -import org.apache.beam.sdk.values.PInput; -import org.apache.beam.sdk.values.WindowingStrategy; - -/** - * A {@link PTransform} wrapping another transform. - */ -public class SinglePrimitiveOutputPTransform extends PTransform> { - private PTransform> transform; - - public SinglePrimitiveOutputPTransform(PTransform> transform) { - this.transform = transform; - } - - @Override - public PCollection expand(PInput input) { - try { - PCollection collection = PCollection.createPrimitiveOutputInternal( - input.getPipeline(), WindowingStrategy.globalDefault(), IsBounded.BOUNDED); - collection.setCoder(transform.getDefaultOutputCoder(input, collection)); - return collection; - } catch (CannotProvideCoderException e) { - throw new IllegalArgumentException( - "Unable to infer a coder and no Coder was specified. " - + "Please set a coder by invoking Create.withCoder() explicitly.", - e); - } - } -} 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 6e6750ddca69f..574ba0c4935f4 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 @@ -94,18 +94,15 @@ private Bounded(@Nullable String name, BoundedSource source) { this.source = SerializableUtils.ensureSerializable(source); } - @Override - protected Coder getDefaultOutputCoder() { - return source.getOutputCoder(); - } - @Override public final PCollection expand(PBegin input) { source.validate(); - return PCollection.createPrimitiveOutputInternal(input.getPipeline(), - WindowingStrategy.globalDefault(), IsBounded.BOUNDED) - .setCoder(getDefaultOutputCoder()); + return PCollection.createPrimitiveOutputInternal( + input.getPipeline(), + WindowingStrategy.globalDefault(), + IsBounded.BOUNDED, + source.getOutputCoder()); } /** @@ -170,9 +167,11 @@ protected Coder getDefaultOutputCoder() { @Override public final PCollection expand(PBegin input) { source.validate(); - - return PCollection.createPrimitiveOutputInternal( - input.getPipeline(), WindowingStrategy.globalDefault(), IsBounded.UNBOUNDED); + return PCollection.createPrimitiveOutputInternal( + input.getPipeline(), + WindowingStrategy.globalDefault(), + IsBounded.UNBOUNDED, + source.getOutputCoder()); } /** 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 d13fcf1e86b7f..45f441361351e 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 @@ -253,9 +253,8 @@ public static Event advanceBy(Duration amount) { @Override public PCollection expand(PBegin input) { - return PCollection.createPrimitiveOutputInternal( - input.getPipeline(), WindowingStrategy.globalDefault(), IsBounded.UNBOUNDED) - .setCoder(coder); + return PCollection.createPrimitiveOutputInternal( + input.getPipeline(), WindowingStrategy.globalDefault(), IsBounded.UNBOUNDED, coder); } public Coder getValueCoder() { 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 25d9c0519727d..8247a58767148 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 @@ -17,7 +17,6 @@ */ package org.apache.beam.sdk.transforms; -import org.apache.beam.sdk.coders.CannotProvideCoderException; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.IterableLikeCoder; import org.apache.beam.sdk.transforms.windowing.WindowFn; @@ -129,25 +128,12 @@ public PCollection expand(PCollectionList inputs) { windowingStrategy = WindowingStrategy.globalDefault(); } - return PCollection.createPrimitiveOutputInternal( + return PCollection.createPrimitiveOutputInternal( inputs.getPipeline(), windowingStrategy, - isBounded); - } - - @Override - protected Coder getDefaultOutputCoder(PCollectionList input) - throws CannotProvideCoderException { - - // Take coder from first collection - for (PCollection pCollection : input.getAll()) { - return pCollection.getCoder(); - } - - // No inputs - throw new CannotProvideCoderException( - this.getClass().getSimpleName() + " cannot provide a Coder for" - + " empty " + PCollectionList.class.getSimpleName()); + isBounded, + // Take coder from first collection. If there are none, will be left unspecified. + inputs.getAll().isEmpty() ? null : inputs.get(0).getCoder()); } } 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 7516b255bb564..3cb0d23733ede 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 @@ -217,13 +217,11 @@ public PCollection>> expand(PCollection> input) { // merging windows as needed, using the windows assigned to the // key/value input elements and the window merge operation of the // window function associated with the input PCollection. - return PCollection.createPrimitiveOutputInternal(input.getPipeline(), - updateWindowingStrategy(input.getWindowingStrategy()), input.isBounded()); - } - - @Override - protected Coder>> getDefaultOutputCoder(PCollection> input) { - return getOutputKvCoder(input.getCoder()); + return PCollection.createPrimitiveOutputInternal( + input.getPipeline(), + updateWindowingStrategy(input.getWindowingStrategy()), + input.isBounded(), + getOutputKvCoder(input.getCoder())); } /** 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 0d03835bb7fb8..bc4f62913cc70 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 @@ -767,6 +767,8 @@ public PCollectionTuple expand(PCollection input) { PCollectionTuple outputs = PCollectionTuple.ofPrimitiveOutputsInternal( input.getPipeline(), TupleTagList.of(mainOutputTag).and(additionalOutputTags.getAll()), + // TODO + Collections., Coder>emptyMap(), input.getWindowingStrategy(), input.isBounded()); 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 57dccbcde062d..f6f3af57d2a7f 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 @@ -509,9 +509,8 @@ public PCollectionView getView() { @Override public PCollection expand(PCollection input) { - return PCollection.createPrimitiveOutputInternal( - input.getPipeline(), input.getWindowingStrategy(), input.isBounded()) - .setCoder(input.getCoder()); + return PCollection.createPrimitiveOutputInternal( + input.getPipeline(), input.getWindowingStrategy(), input.isBounded(), input.getCoder()); } } } 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 a12be6d3b4bf0..af583e52079c1 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 @@ -484,7 +484,7 @@ public static class Assign extends PTransform, PCollection> @Override public PCollection expand(PCollection input) { return PCollection.createPrimitiveOutputInternal( - input.getPipeline(), updatedStrategy, input.isBounded()); + input.getPipeline(), updatedStrategy, input.isBounded(), input.getCoder()); } @Override 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 4063d110cb7b3..e8bf9b886fae3 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 @@ -366,10 +366,15 @@ public PCollection setIsBoundedInternal(IsBounded isBounded) { public static PCollection createPrimitiveOutputInternal( Pipeline pipeline, WindowingStrategy windowingStrategy, - IsBounded isBounded) { - return new PCollection(pipeline) + IsBounded isBounded, + @Nullable Coder coder) { + PCollection res = new PCollection(pipeline) .setWindowingStrategyInternal(windowingStrategy) .setIsBoundedInternal(isBounded); + if (coder != null) { + res.setCoder(coder); + } + return res; } private static class CoderOrFailure { 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 793994f102e9e..9799d0e87372e 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 @@ -24,6 +24,7 @@ import java.util.Objects; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.annotations.Internal; +import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.ParDo; import org.apache.beam.sdk.values.PCollection.IsBounded; @@ -201,6 +202,7 @@ public OutputT apply( public static PCollectionTuple ofPrimitiveOutputsInternal( Pipeline pipeline, TupleTagList outputTags, + Map, Coder> coders, WindowingStrategy windowingStrategy, IsBounded isBounded) { Map, PCollection> pcollectionMap = new LinkedHashMap<>(); @@ -217,10 +219,10 @@ public static PCollectionTuple ofPrimitiveOutputsInternal( // erasure as the correct type. When a transform adds // elements to `outputCollection` they will be of type T. @SuppressWarnings("unchecked") - TypeDescriptor token = (TypeDescriptor) outputTag.getTypeDescriptor(); - PCollection outputCollection = PCollection - .createPrimitiveOutputInternal(pipeline, windowingStrategy, isBounded) - .setTypeDescriptor(token); + PCollection outputCollection = + PCollection.createPrimitiveOutputInternal( + pipeline, windowingStrategy, isBounded, coders.get(outputTag)) + .setTypeDescriptor((TypeDescriptor) outputTag.getTypeDescriptor()); pcollectionMap.put(outputTag, outputCollection); } 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 93650dd8c9836..12fe633fa1ac5 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 @@ -35,6 +35,7 @@ import org.apache.beam.sdk.Pipeline.PipelineVisitor.Defaults; import org.apache.beam.sdk.coders.StringUtf8Coder; import org.apache.beam.sdk.coders.VarIntCoder; +import org.apache.beam.sdk.coders.VarLongCoder; import org.apache.beam.sdk.io.CountingSource; import org.apache.beam.sdk.io.GenerateSequence; import org.apache.beam.sdk.io.Read; @@ -110,7 +111,7 @@ public void pushThenPopSucceeds() { public void emptyCompositeSucceeds() { PCollection created = PCollection.createPrimitiveOutputInternal( - pipeline, WindowingStrategy.globalDefault(), IsBounded.BOUNDED); + pipeline, WindowingStrategy.globalDefault(), IsBounded.BOUNDED, VarLongCoder.of()); TransformHierarchy.Node node = hierarchy.pushNode("Create", PBegin.in(pipeline), Create.of(1)); hierarchy.setOutput(created); hierarchy.popNode(); @@ -139,7 +140,7 @@ public PCollection expand(PCollectionList input) { public void producingOwnAndOthersOutputsFails() { PCollection created = PCollection.createPrimitiveOutputInternal( - pipeline, WindowingStrategy.globalDefault(), IsBounded.BOUNDED); + pipeline, WindowingStrategy.globalDefault(), IsBounded.BOUNDED, VarLongCoder.of()); hierarchy.pushNode("Create", PBegin.in(pipeline), Create.of(1)); hierarchy.setOutput(created); hierarchy.popNode(); @@ -147,8 +148,11 @@ public void producingOwnAndOthersOutputsFails() { final PCollectionList appended = pcList.and( - PCollection.createPrimitiveOutputInternal( - pipeline, WindowingStrategy.globalDefault(), IsBounded.BOUNDED) + PCollection.createPrimitiveOutputInternal( + pipeline, + WindowingStrategy.globalDefault(), + IsBounded.BOUNDED, + VarLongCoder.of()) .setName("prim")); hierarchy.pushNode( "AddPc", @@ -171,7 +175,7 @@ public PCollectionList expand(PCollectionList input) { public void producingOwnOutputWithCompositeFails() { final PCollection comp = PCollection.createPrimitiveOutputInternal( - pipeline, WindowingStrategy.globalDefault(), IsBounded.BOUNDED); + pipeline, WindowingStrategy.globalDefault(), IsBounded.BOUNDED, VarLongCoder.of()); PTransform> root = new PTransform>() { @Override @@ -327,7 +331,7 @@ public void visitVisitsAllPushed() { PCollection created = PCollection.createPrimitiveOutputInternal( - pipeline, WindowingStrategy.globalDefault(), IsBounded.BOUNDED); + pipeline, WindowingStrategy.globalDefault(), IsBounded.BOUNDED, VarLongCoder.of()); SingleOutput pardo = ParDo.of( @@ -340,7 +344,7 @@ public void processElement(ProcessContext ctxt) { PCollection mapped = PCollection.createPrimitiveOutputInternal( - pipeline, WindowingStrategy.globalDefault(), IsBounded.BOUNDED); + pipeline, WindowingStrategy.globalDefault(), IsBounded.BOUNDED, VarLongCoder.of()); TransformHierarchy.Node compositeNode = hierarchy.pushNode("Create", begin, create); hierarchy.finishSpecifyingInput(); @@ -499,13 +503,11 @@ public void visitPrimitiveTransform(Node node) { @Test public void visitIsTopologicallyOrdered() { PCollection one = - PCollection.createPrimitiveOutputInternal( - pipeline, WindowingStrategy.globalDefault(), IsBounded.BOUNDED) - .setCoder(StringUtf8Coder.of()); + PCollection.createPrimitiveOutputInternal( + pipeline, WindowingStrategy.globalDefault(), IsBounded.BOUNDED, StringUtf8Coder.of()); final PCollection two = - PCollection.createPrimitiveOutputInternal( - pipeline, WindowingStrategy.globalDefault(), IsBounded.UNBOUNDED) - .setCoder(VarIntCoder.of()); + PCollection.createPrimitiveOutputInternal( + pipeline, WindowingStrategy.globalDefault(), IsBounded.UNBOUNDED, VarIntCoder.of()); final PDone done = PDone.in(pipeline); final TupleTag oneTag = new TupleTag() {}; final TupleTag twoTag = new TupleTag() {}; @@ -617,13 +619,14 @@ public void visitValue(PValue value, Node producer) { @Test public void visitDoesNotVisitSkippedNodes() { PCollection one = - PCollection.createPrimitiveOutputInternal( - pipeline, WindowingStrategy.globalDefault(), IsBounded.BOUNDED) - .setCoder(StringUtf8Coder.of()); + PCollection.createPrimitiveOutputInternal( + pipeline, + WindowingStrategy.globalDefault(), + IsBounded.BOUNDED, + StringUtf8Coder.of()); final PCollection two = - PCollection.createPrimitiveOutputInternal( - pipeline, WindowingStrategy.globalDefault(), IsBounded.UNBOUNDED) - .setCoder(VarIntCoder.of()); + PCollection.createPrimitiveOutputInternal( + pipeline, WindowingStrategy.globalDefault(), IsBounded.UNBOUNDED, VarIntCoder.of()); final PDone done = PDone.in(pipeline); final TupleTag oneTag = new TupleTag() {}; final TupleTag twoTag = new TupleTag() {}; 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 e7b680ab6c0ae..bf06d7820ad56 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 @@ -28,6 +28,7 @@ import java.util.EnumSet; 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.coders.VoidCoder; import org.apache.beam.sdk.io.Read; import org.apache.beam.sdk.io.TextIO; @@ -85,10 +86,13 @@ public PCollectionList expand(PBegin b) { // Issue below: PCollection.createPrimitiveOutput should not be used // from within a composite transform. return PCollectionList.of( - Arrays.asList(result, PCollection.createPrimitiveOutputInternal( - b.getPipeline(), - WindowingStrategy.globalDefault(), - result.isBounded()))); + Arrays.asList( + result, + PCollection.createPrimitiveOutputInternal( + b.getPipeline(), + WindowingStrategy.globalDefault(), + result.isBounded(), + StringUtf8Coder.of()))); } } 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 a8cb843709320..5dbe176aab4a5 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 @@ -228,7 +228,7 @@ public void testFlattenPCollectionsEmptyThenParDo() { public void testFlattenNoListsNoCoder() { // not ValidatesRunner because it should fail at pipeline construction time anyhow. thrown.expect(IllegalStateException.class); - thrown.expectMessage("cannot provide a Coder for empty"); + thrown.expectMessage("Unable to return a default Coder"); PCollectionList.empty(p) .apply(Flatten.pCollections()); 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 8fcb4c0debd8b..a76714c2bc855 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 @@ -70,7 +70,6 @@ import org.apache.beam.sdk.values.PBegin; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.TimestampedValue; -import org.apache.beam.sdk.values.TypeDescriptor; import org.apache.beam.sdk.values.WindowingStrategy; import org.hamcrest.Matcher; import org.joda.time.Duration; @@ -423,11 +422,11 @@ public void testGroupByKeyDirectUnbounded() { new PTransform>>() { @Override public PCollection> expand(PBegin input) { - return PCollection.>createPrimitiveOutputInternal( - input.getPipeline(), - WindowingStrategy.globalDefault(), - PCollection.IsBounded.UNBOUNDED) - .setTypeDescriptor(new TypeDescriptor>() {}); + return PCollection.createPrimitiveOutputInternal( + input.getPipeline(), + WindowingStrategy.globalDefault(), + PCollection.IsBounded.UNBOUNDED, + KvCoder.of(StringUtf8Coder.of(), VarIntCoder.of())); } }); 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 cdd03d9714fd0..bfb8b5accb05e 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 @@ -60,7 +60,6 @@ import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TimestampedValue; -import org.apache.beam.sdk.values.TypeDescriptor; import org.apache.beam.sdk.values.WindowingStrategy; import org.hamcrest.Matchers; import org.joda.time.Duration; @@ -1340,11 +1339,11 @@ private void testViewUnbounded( new PTransform>>() { @Override public PCollection> expand(PBegin input) { - return PCollection.>createPrimitiveOutputInternal( - input.getPipeline(), - WindowingStrategy.globalDefault(), - PCollection.IsBounded.UNBOUNDED) - .setTypeDescriptor(new TypeDescriptor>() {}); + return PCollection.createPrimitiveOutputInternal( + input.getPipeline(), + WindowingStrategy.globalDefault(), + PCollection.IsBounded.UNBOUNDED, + KvCoder.of(StringUtf8Coder.of(), VarIntCoder.of())); } }) .apply(view); 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 58e2bbdc859aa..33503b673587a 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 @@ -31,6 +31,7 @@ import java.util.Map; import java.util.Map.Entry; import org.apache.beam.sdk.Pipeline; +import org.apache.beam.sdk.coders.VarIntCoder; import org.apache.beam.sdk.io.GenerateSequence; import org.apache.beam.sdk.testing.PAssert; import org.apache.beam.sdk.testing.TestPipeline; @@ -59,9 +60,9 @@ public final class PCollectionTupleTest implements Serializable { @Test public void testOfThenHas() { - PCollection pCollection = PCollection.createPrimitiveOutputInternal( - pipeline, WindowingStrategy.globalDefault(), IsBounded.BOUNDED); - TupleTag tag = new TupleTag<>(); + PCollection pCollection = PCollection.createPrimitiveOutputInternal( + pipeline, WindowingStrategy.globalDefault(), IsBounded.BOUNDED, VarIntCoder.of()); + TupleTag tag = new TupleTag<>(); assertTrue(PCollectionTuple.of(tag, pCollection).has(tag)); } From 95e2a00a807caaf3b4a9532e29dc38fd9d32e700 Mon Sep 17 00:00:00 2001 From: Eugene Kirpichov Date: Wed, 26 Jul 2017 16:33:36 -0700 Subject: [PATCH 239/346] Removes all overrides of PTransform.getDefaultOutputCoder and deprecates it --- .../apache/beam/examples/complete/TfIdf.java | 14 ++-- .../construction/ForwardingPTransform.java | 18 ++--- .../ForwardingPTransformTest.java | 17 ++++- .../beam/runners/direct/MultiStepCombine.java | 10 +-- .../beam/runners/dataflow/AssignWindows.java | 8 +-- .../DataflowPipelineTranslatorTest.java | 10 --- .../java/org/apache/beam/sdk/io/AvroIO.java | 6 -- .../io/BoundedReadFromUnboundedSource.java | 8 +-- .../java/org/apache/beam/sdk/io/Read.java | 6 -- .../org/apache/beam/sdk/io/TFRecordIO.java | 18 +---- .../java/org/apache/beam/sdk/io/TextIO.java | 12 ---- .../apache/beam/sdk/transforms/Combine.java | 22 ++---- .../apache/beam/sdk/transforms/Create.java | 70 ++++++++----------- .../apache/beam/sdk/transforms/Filter.java | 26 ++++--- .../beam/sdk/transforms/PTransform.java | 13 +++- .../org/apache/beam/sdk/transforms/ParDo.java | 59 ++++++++-------- .../beam/sdk/transforms/windowing/Window.java | 6 -- .../beam/sdk/runners/TransformTreeTest.java | 7 -- .../beam/sdk/transforms/CreateTest.java | 27 +++---- .../sdk/extensions/sorter/SortValues.java | 20 +++--- .../beam/sdk/io/gcp/bigquery/BigQueryIO.java | 18 +---- .../gcp/bigquery/PassThroughThenCleanup.java | 2 +- .../sdk/io/gcp/bigquery/StreamingInserts.java | 8 --- .../beam/sdk/io/gcp/pubsub/PubsubIO.java | 13 +--- 24 files changed, 149 insertions(+), 269 deletions(-) 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 435ffab45063f..cfc413ccde393 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 @@ -25,7 +25,6 @@ import java.util.HashSet; import java.util.Set; import org.apache.beam.sdk.Pipeline; -import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.KvCoder; import org.apache.beam.sdk.coders.StringDelegateCoder; import org.apache.beam.sdk.coders.StringUtf8Coder; @@ -154,11 +153,6 @@ public ReadDocuments(Iterable uris) { this.uris = uris; } - @Override - public Coder getDefaultOutputCoder() { - return KvCoder.of(StringDelegateCoder.of(URI.class), StringUtf8Coder.of()); - } - @Override public PCollection> expand(PBegin input) { Pipeline pipeline = input.getPipeline(); @@ -179,9 +173,11 @@ public PCollection> expand(PBegin input) { uriString = uri.toString(); } - PCollection> oneUriToLines = pipeline - .apply("TextIO.Read(" + uriString + ")", TextIO.read().from(uriString)) - .apply("WithKeys(" + uriString + ")", WithKeys.of(uri)); + PCollection> oneUriToLines = + pipeline + .apply("TextIO.Read(" + uriString + ")", TextIO.read().from(uriString)) + .apply("WithKeys(" + uriString + ")", WithKeys.of(uri)) + .setCoder(KvCoder.of(StringDelegateCoder.of(URI.class), StringUtf8Coder.of())); urisToLines = urisToLines.and(oneUriToLines); } diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ForwardingPTransform.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ForwardingPTransform.java index ca25ba7ebbffd..ccf41f3adceca 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ForwardingPTransform.java +++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ForwardingPTransform.java @@ -18,7 +18,6 @@ package org.apache.beam.runners.core.construction; import org.apache.beam.sdk.coders.CannotProvideCoderException; -import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.display.DisplayData; @@ -37,7 +36,16 @@ public abstract class ForwardingPTransform Coder getDefaultOutputCoder(InputT input, PCollection output) - throws CannotProvideCoderException { - return delegate().getDefaultOutputCoder(input, output); - } - @Override public void populateDisplayData(DisplayData.Builder builder) { builder.delegate(delegate()); diff --git a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/ForwardingPTransformTest.java b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/ForwardingPTransformTest.java index 74c056ced437b..4741b6b469a8e 100644 --- a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/ForwardingPTransformTest.java +++ b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/ForwardingPTransformTest.java @@ -26,6 +26,7 @@ import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.display.DisplayData; import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.WindowingStrategy; import org.junit.Before; import org.junit.Rule; import org.junit.Test; @@ -90,14 +91,24 @@ public void validateDelegates() { @Test public void getDefaultOutputCoderDelegates() throws Exception { @SuppressWarnings("unchecked") - PCollection input = Mockito.mock(PCollection.class); + PCollection input = + PCollection.createPrimitiveOutputInternal( + null /* pipeline */, + WindowingStrategy.globalDefault(), + PCollection.IsBounded.BOUNDED, + null /* coder */); @SuppressWarnings("unchecked") - PCollection output = Mockito.mock(PCollection.class); + PCollection output = PCollection.createPrimitiveOutputInternal( + null /* pipeline */, + WindowingStrategy.globalDefault(), + PCollection.IsBounded.BOUNDED, + null /* coder */); @SuppressWarnings("unchecked") Coder outputCoder = Mockito.mock(Coder.class); + Mockito.when(delegate.expand(input)).thenReturn(output); Mockito.when(delegate.getDefaultOutputCoder(input, output)).thenReturn(outputCoder); - assertThat(forwarding.getDefaultOutputCoder(input, output), equalTo(outputCoder)); + assertThat(forwarding.expand(input).getCoder(), equalTo(outputCoder)); } @Test diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/MultiStepCombine.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/MultiStepCombine.java index 6f49e9449fed5..ae21b4dad75b1 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/MultiStepCombine.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/MultiStepCombine.java @@ -213,8 +213,7 @@ public PCollection> expand(PCollection> input) { inputCoder.getKeyCoder()))) .setCoder(KvCoder.of(inputCoder.getKeyCoder(), accumulatorCoder)) .apply(GroupByKey.create()) - .apply(new MergeAndExtractAccumulatorOutput(combineFn)) - .setCoder(outputCoder); + .apply(new MergeAndExtractAccumulatorOutput<>(combineFn, outputCoder)); } private static class CombineInputs extends DoFn, KV> { @@ -320,9 +319,12 @@ public int hashCode() { static class MergeAndExtractAccumulatorOutput extends RawPTransform>>, PCollection>> { private final CombineFn combineFn; + private final Coder> outputCoder; - private MergeAndExtractAccumulatorOutput(CombineFn combineFn) { + private MergeAndExtractAccumulatorOutput( + CombineFn combineFn, Coder> outputCoder) { this.combineFn = combineFn; + this.outputCoder = outputCoder; } CombineFn getCombineFn() { @@ -332,7 +334,7 @@ CombineFn getCombineFn() { @Override public PCollection> expand(PCollection>> input) { return PCollection.createPrimitiveOutputInternal( - input.getPipeline(), input.getWindowingStrategy(), input.isBounded()); + input.getPipeline(), input.getWindowingStrategy(), input.isBounded(), outputCoder); } @Nullable diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/AssignWindows.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/AssignWindows.java index d015d2bbebf23..7d1dadbe11a4f 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/AssignWindows.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/AssignWindows.java @@ -17,7 +17,6 @@ */ package org.apache.beam.runners.dataflow; -import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.PTransform; @@ -69,7 +68,7 @@ public PCollection expand(PCollection input) { public void processElement(ProcessContext c) throws Exception { c.output(c.element()); } - })).setWindowingStrategyInternal(outputStrategy); + })).setWindowingStrategyInternal(outputStrategy).setCoder(input.getCoder()); } } @@ -78,11 +77,6 @@ public void validate(PipelineOptions options) { transform.validate(options); } - @Override - protected Coder getDefaultOutputCoder(PCollection input) { - return input.getCoder(); - } - @Override protected String getKindString() { return "Window.Into()"; 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 7a99f750cf097..f756065303f74 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 @@ -606,11 +606,6 @@ public PCollection expand(PCollection input) { // Return a value unrelated to the input. return input.getPipeline().apply(Create.of(1, 2, 3, 4)); } - - @Override - protected Coder getDefaultOutputCoder() { - return VarIntCoder.of(); - } } /** @@ -626,11 +621,6 @@ public PDone expand(PCollection input) { return PDone.in(input.getPipeline()); } - - @Override - protected Coder getDefaultOutputCoder() { - return VoidCoder.of(); - } } /** 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 824f725fe7120..cd5857c902589 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 @@ -38,7 +38,6 @@ import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.CoderRegistry; import org.apache.beam.sdk.coders.StringUtf8Coder; -import org.apache.beam.sdk.coders.VoidCoder; import org.apache.beam.sdk.io.FileBasedSink.FilenamePolicy; import org.apache.beam.sdk.io.fs.ResourceId; import org.apache.beam.sdk.options.ValueProvider; @@ -996,11 +995,6 @@ public void populateDisplayData(DisplayData.Builder builder) { DisplayData.item("tempDirectory", tempDirectory) .withLabel("Directory for temporary files")); } - - @Override - protected Coder getDefaultOutputCoder() { - return VoidCoder.of(); - } } /** 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 8505ca476c7ff..80a03eb213492 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 @@ -114,12 +114,8 @@ public byte[] apply(ValueWithRecordId input) { } })); } - return read.apply("StripIds", ParDo.of(new ValueWithRecordId.StripIdsDoFn())); - } - - @Override - protected Coder getDefaultOutputCoder() { - return source.getOutputCoder(); + return read.apply("StripIds", ParDo.of(new ValueWithRecordId.StripIdsDoFn())) + .setCoder(source.getOutputCoder()); } @Override 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 574ba0c4935f4..9b273f862146b 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 @@ -18,7 +18,6 @@ package org.apache.beam.sdk.io; import javax.annotation.Nullable; -import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.display.DisplayData; import org.apache.beam.sdk.util.NameUtils; @@ -159,11 +158,6 @@ public BoundedReadFromUnboundedSource withMaxReadTime(Duration maxReadTime) { return new BoundedReadFromUnboundedSource(source, Long.MAX_VALUE, maxReadTime); } - @Override - protected Coder getDefaultOutputCoder() { - return source.getOutputCoder(); - } - @Override public final PCollection expand(PBegin input) { source.validate(); diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/TFRecordIO.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/TFRecordIO.java index 1b2e95bc74eeb..c75051fe7bc64 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/TFRecordIO.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/TFRecordIO.java @@ -35,8 +35,6 @@ import org.apache.beam.sdk.annotations.Experimental.Kind; import org.apache.beam.sdk.coders.ByteArrayCoder; import org.apache.beam.sdk.coders.Coder; -import org.apache.beam.sdk.coders.VoidCoder; -import org.apache.beam.sdk.io.Read.Bounded; import org.apache.beam.sdk.io.fs.MatchResult; import org.apache.beam.sdk.io.fs.MatchResult.Metadata; import org.apache.beam.sdk.io.fs.ResourceId; @@ -171,11 +169,7 @@ public PCollection expand(PBegin input) { } } - final Bounded read = org.apache.beam.sdk.io.Read.from(getSource()); - PCollection pcol = input.getPipeline().apply("Read", read); - // Honor the default output coder that would have been used by this PTransform. - pcol.setCoder(getDefaultOutputCoder()); - return pcol; + return input.apply("Read", org.apache.beam.sdk.io.Read.from(getSource())); } // Helper to create a source specific to the requested compression type. @@ -212,11 +206,6 @@ public void populateDisplayData(DisplayData.Builder builder) { .addIfNotNull(DisplayData.item("filePattern", filepatternDisplay) .withLabel("File Pattern")); } - - @Override - protected Coder getDefaultOutputCoder() { - return ByteArrayCoder.of(); - } } ///////////////////////////////////////////////////////////////////////////// @@ -391,11 +380,6 @@ public void populateDisplayData(DisplayData.Builder builder) { .add(DisplayData.item("compressionType", getCompressionType().toString()) .withLabel("Compression Type")); } - - @Override - protected Coder getDefaultOutputCoder() { - return VoidCoder.of(); - } } /** 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 312dc075ac8bc..9a14ad9d12b95 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 @@ -30,9 +30,7 @@ import javax.annotation.Nullable; import org.apache.beam.sdk.annotations.Experimental; import org.apache.beam.sdk.annotations.Experimental.Kind; -import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.StringUtf8Coder; -import org.apache.beam.sdk.coders.VoidCoder; import org.apache.beam.sdk.io.CompressedSource.CompressionMode; import org.apache.beam.sdk.io.DefaultFilenamePolicy.Params; import org.apache.beam.sdk.io.FileBasedSink.DynamicDestinations; @@ -337,11 +335,6 @@ public void populateDisplayData(DisplayData.Builder builder) { .addIfNotNull(DisplayData.item("filePattern", filepatternDisplay) .withLabel("File Pattern")); } - - @Override - protected Coder getDefaultOutputCoder() { - return StringUtf8Coder.of(); - } } ///////////////////////////////////////////////////////////////////////////// @@ -813,11 +806,6 @@ public void populateDisplayData(DisplayData.Builder builder) { "writableByteChannelFactory", getWritableByteChannelFactory().toString()) .withLabel("Compression/Transformation Type")); } - - @Override - protected Coder getDefaultOutputCoder() { - return VoidCoder.of(); - } } /** 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 c19535263006c..fab98f80795e0 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 @@ -2156,8 +2156,13 @@ public void populateDisplayData(DisplayData.Builder builder) { }).withSideInputs(sideInputs)); try { - Coder> outputCoder = getDefaultOutputCoder(input); - output.setCoder(outputCoder); + KvCoder kvCoder = getKvCoder(input.getCoder()); + @SuppressWarnings("unchecked") + Coder outputValueCoder = + ((GlobalCombineFn) fn) + .getDefaultOutputCoder( + input.getPipeline().getCoderRegistry(), kvCoder.getValueCoder()); + output.setCoder(KvCoder.of(kvCoder.getKeyCoder(), outputValueCoder)); } catch (CannotProvideCoderException exc) { // let coder inference happen later, if it can } @@ -2199,19 +2204,6 @@ private KvCoder getKvCoder( return KvCoder.of(keyCoder, inputValueCoder); } - @Override - public Coder> getDefaultOutputCoder( - PCollection>> input) - throws CannotProvideCoderException { - KvCoder kvCoder = getKvCoder(input.getCoder()); - @SuppressWarnings("unchecked") - Coder outputValueCoder = - ((GlobalCombineFn) fn) - .getDefaultOutputCoder( - input.getPipeline().getCoderRegistry(), kvCoder.getValueCoder()); - return KvCoder.of(kvCoder.getKeyCoder(), outputValueCoder); - } - @Override public void populateDisplayData(DisplayData.Builder builder) { super.populateDisplayData(builder); 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 a28e9b27e7a0b..2635bc8abde9e 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 @@ -315,29 +315,25 @@ public Iterable getElements() { @Override public PCollection expand(PBegin input) { + Coder coder; try { - Coder coder = getDefaultOutputCoder(input); - try { - CreateSource source = CreateSource.fromIterable(elems, coder); - return input.getPipeline().apply(Read.from(source)); - } catch (IOException e) { - throw new RuntimeException( - String.format("Unable to apply Create %s using Coder %s.", this, coder), e); - } + CoderRegistry registry = input.getPipeline().getCoderRegistry(); + coder = + this.coder.isPresent() + ? this.coder.get() + : typeDescriptor.isPresent() + ? registry.getCoder(typeDescriptor.get()) + : getDefaultCreateCoder(registry, elems); } catch (CannotProvideCoderException e) { throw new IllegalArgumentException("Unable to infer a coder and no Coder was specified. " + "Please set a coder by invoking Create.withCoder() explicitly.", e); } - } - - @Override - public Coder getDefaultOutputCoder(PBegin input) throws CannotProvideCoderException { - if (coder.isPresent()) { - return coder.get(); - } else if (typeDescriptor.isPresent()) { - return input.getPipeline().getCoderRegistry().getCoder(typeDescriptor.get()); - } else { - return getDefaultCreateCoder(input.getPipeline().getCoderRegistry(), elems); + try { + CreateSource source = CreateSource.fromIterable(elems, coder); + return input.getPipeline().apply(Read.from(source)); + } catch (IOException e) { + throw new RuntimeException( + String.format("Unable to apply Create %s using Coder %s.", this, coder), e); } } @@ -570,7 +566,23 @@ public TimestampedValues withType(TypeDescriptor type) { @Override public PCollection expand(PBegin input) { try { - Coder coder = getDefaultOutputCoder(input); + Coder coder; + if (elementCoder.isPresent()) { + coder = elementCoder.get(); + } else if (typeDescriptor.isPresent()) { + coder = input.getPipeline().getCoderRegistry().getCoder(typeDescriptor.get()); + } else { + Iterable rawElements = + Iterables.transform( + timestampedElements, + new Function, T>() { + @Override + public T apply(TimestampedValue timestampedValue) { + return timestampedValue.getValue(); + } + }); + coder = getDefaultCreateCoder(input.getPipeline().getCoderRegistry(), rawElements); + } PCollection> intermediate = Pipeline.applyTransform(input, Create.of(timestampedElements).withCoder(TimestampedValueCoder.of(coder))); @@ -610,26 +622,6 @@ public void processElement(ProcessContext c) { c.outputWithTimestamp(c.element().getValue(), c.element().getTimestamp()); } } - - @Override - public Coder getDefaultOutputCoder(PBegin input) throws CannotProvideCoderException { - if (elementCoder.isPresent()) { - return elementCoder.get(); - } else if (typeDescriptor.isPresent()) { - return input.getPipeline().getCoderRegistry().getCoder(typeDescriptor.get()); - } else { - Iterable rawElements = - Iterables.transform( - timestampedElements, - new Function, T>() { - @Override - public T apply(TimestampedValue input) { - return input.getValue(); - } - }); - return getDefaultCreateCoder(input.getPipeline().getCoderRegistry(), rawElements); - } - } } private static Coder getDefaultCreateCoder(CoderRegistry registry, Iterable elems) 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 d0314eb8ce095..2fd12de18da80 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 @@ -17,7 +17,6 @@ */ package org.apache.beam.sdk.transforms; -import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.transforms.display.DisplayData; import org.apache.beam.sdk.values.PCollection; @@ -229,19 +228,18 @@ Filter described(String description) { @Override public PCollection expand(PCollection input) { - return input.apply(ParDo.of(new DoFn() { - @ProcessElement - public void processElement(ProcessContext c) { - if (predicate.apply(c.element())) { - c.output(c.element()); - } - } - })); - } - - @Override - protected Coder getDefaultOutputCoder(PCollection input) { - return input.getCoder(); + return input + .apply( + ParDo.of( + new DoFn() { + @ProcessElement + public void processElement(ProcessContext c) { + if (predicate.apply(c.element())) { + c.output(c.element()); + } + } + })) + .setCoder(input.getCoder()); } @Override 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 58051df782157..f5e78302f343b 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 @@ -277,13 +277,16 @@ private void readObject(ObjectInputStream oos) { } /** - * Returns the default {@code Coder} to use for the output of this - * single-output {@code PTransform}. + * Returns the default {@code Coder} to use for the output of this single-output {@code + * PTransform}. * *

              By default, always throws * * @throws CannotProvideCoderException if no coder can be inferred + * @deprecated Instead, the PTransform should explicitly call {@link PCollection#setCoder} on the + * returned PCollection. */ + @Deprecated protected Coder getDefaultOutputCoder() throws CannotProvideCoderException { throw new CannotProvideCoderException("PTransform.getOutputCoder called."); } @@ -295,7 +298,10 @@ protected Coder getDefaultOutputCoder() throws CannotProvideCoderException { *

              By default, always throws. * * @throws CannotProvideCoderException if none can be inferred. + * @deprecated Instead, the PTransform should explicitly call {@link PCollection#setCoder} on the + * returned PCollection. */ + @Deprecated protected Coder getDefaultOutputCoder(@SuppressWarnings("unused") InputT input) throws CannotProvideCoderException { return getDefaultOutputCoder(); @@ -308,7 +314,10 @@ protected Coder getDefaultOutputCoder(@SuppressWarnings("unused") InputT inpu *

              By default, always throws. * * @throws CannotProvideCoderException if none can be inferred. + * @deprecated Instead, the PTransform should explicitly call {@link PCollection#setCoder} on the + * returned PCollection. */ + @Deprecated public Coder getDefaultOutputCoder( InputT input, @SuppressWarnings("unused") PCollection output) throws CannotProvideCoderException { 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 bc4f62913cc70..a0e1eb203a398 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 @@ -636,19 +636,21 @@ public MultiOutput withOutputTags( @Override public PCollection expand(PCollection input) { - finishSpecifyingStateSpecs(fn, input.getPipeline().getCoderRegistry(), input.getCoder()); + CoderRegistry registry = input.getPipeline().getCoderRegistry(); + finishSpecifyingStateSpecs(fn, registry, input.getCoder()); TupleTag mainOutput = new TupleTag<>(); - return input.apply(withOutputTags(mainOutput, TupleTagList.empty())).get(mainOutput); - } - - @Override - @SuppressWarnings("unchecked") - protected Coder getDefaultOutputCoder(PCollection input) - throws CannotProvideCoderException { - return input.getPipeline().getCoderRegistry().getCoder( - getFn().getOutputTypeDescriptor(), - getFn().getInputTypeDescriptor(), - ((PCollection) input).getCoder()); + PCollection res = + input.apply(withOutputTags(mainOutput, TupleTagList.empty())).get(mainOutput); + try { + res.setCoder( + registry.getCoder( + getFn().getOutputTypeDescriptor(), + getFn().getInputTypeDescriptor(), + ((PCollection) input).getCoder())); + } catch (CannotProvideCoderException e) { + // Ignore and leave coder unset. + } + return res; } @Override @@ -757,7 +759,8 @@ public PCollectionTuple expand(PCollection input) { validateWindowType(input, fn); // Use coder registry to determine coders for all StateSpec defined in the fn signature. - finishSpecifyingStateSpecs(fn, input.getPipeline().getCoderRegistry(), input.getCoder()); + CoderRegistry registry = input.getPipeline().getCoderRegistry(); + finishSpecifyingStateSpecs(fn, registry, input.getCoder()); DoFnSignature signature = DoFnSignatures.getSignature(fn.getClass()); if (signature.usesState() || signature.usesTimers()) { @@ -771,6 +774,18 @@ public PCollectionTuple expand(PCollection input) { Collections., Coder>emptyMap(), input.getWindowingStrategy(), input.isBounded()); + @SuppressWarnings("unchecked") + Coder inputCoder = ((PCollection) input).getCoder(); + for (PCollection out : outputs.getAll().values()) { + try { + out.setCoder( + (Coder) + registry.getCoder( + out.getTypeDescriptor(), getFn().getInputTypeDescriptor(), inputCoder)); + } catch (CannotProvideCoderException e) { + // Ignore and let coder inference happen later. + } + } // The fn will likely be an instance of an anonymous subclass // such as DoFn { }, thus will have a high-fidelity @@ -780,24 +795,6 @@ public PCollectionTuple expand(PCollection input) { return outputs; } - @Override - protected Coder getDefaultOutputCoder() { - throw new RuntimeException( - "internal error: shouldn't be calling this on a multi-output ParDo"); - } - - @Override - public Coder getDefaultOutputCoder( - PCollection input, PCollection output) - throws CannotProvideCoderException { - @SuppressWarnings("unchecked") - Coder inputCoder = ((PCollection) input).getCoder(); - return input.getPipeline().getCoderRegistry().getCoder( - output.getTypeDescriptor(), - getFn().getInputTypeDescriptor(), - inputCoder); - } - @Override protected String getKindString() { return String.format("ParMultiDo(%s)", NameUtils.approximateSimpleName(getFn())); 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 af583e52079c1..2337798e48a3f 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 @@ -23,7 +23,6 @@ import javax.annotation.Nullable; import org.apache.beam.sdk.annotations.Experimental; import org.apache.beam.sdk.annotations.Experimental.Kind; -import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.Coder.NonDeterministicException; import org.apache.beam.sdk.transforms.Flatten; import org.apache.beam.sdk.transforms.GroupByKey; @@ -452,11 +451,6 @@ public void populateDisplayData(DisplayData.Builder builder) { } } - @Override - protected Coder getDefaultOutputCoder(PCollection input) { - return input.getCoder(); - } - @Override protected String getKindString() { return "Window.Into()"; 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 bf06d7820ad56..9956d5cfe1355 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 @@ -27,9 +27,7 @@ import java.util.Arrays; import java.util.EnumSet; 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.coders.VoidCoder; import org.apache.beam.sdk.io.Read; import org.apache.beam.sdk.io.TextIO; import org.apache.beam.sdk.io.WriteFiles; @@ -109,11 +107,6 @@ public PDone expand(PCollection input) { return PDone.in(input.getPipeline()); } - - @Override - protected Coder getDefaultOutputCoder() { - return VoidCoder.of(); - } } @Test 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 6be67722281d0..81ad947ccf466 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 @@ -62,7 +62,6 @@ import org.apache.beam.sdk.util.SerializableUtils; import org.apache.beam.sdk.util.common.ReflectHelpers; import org.apache.beam.sdk.values.KV; -import org.apache.beam.sdk.values.PBegin; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.TimestampedValue; import org.apache.beam.sdk.values.TypeDescriptor; @@ -314,28 +313,24 @@ public void testCreateTimestampedPolymorphicType() throws Exception { @Test public void testCreateTimestampedDefaultOutputCoderUsingCoder() throws Exception { Coder coder = new RecordCoder(); - PBegin pBegin = PBegin.in(p); Create.TimestampedValues values = Create.timestamped( TimestampedValue.of(new Record(), new Instant(0)), TimestampedValue.of(new Record2(), new Instant(0))) .withCoder(coder); - Coder defaultCoder = values.getDefaultOutputCoder(pBegin); - assertThat(defaultCoder, equalTo(coder)); + assertThat(p.apply(values).getCoder(), equalTo(coder)); } @Test public void testCreateTimestampedDefaultOutputCoderUsingTypeDescriptor() throws Exception { Coder coder = new RecordCoder(); p.getCoderRegistry().registerCoderForClass(Record.class, coder); - PBegin pBegin = PBegin.in(p); Create.TimestampedValues values = Create.timestamped( TimestampedValue.of(new Record(), new Instant(0)), TimestampedValue.of(new Record2(), new Instant(0))) .withType(new TypeDescriptor() {}); - Coder defaultCoder = values.getDefaultOutputCoder(pBegin); - assertThat(defaultCoder, equalTo(coder)); + assertThat(p.apply(values).getCoder(), equalTo(coder)); } @Test @@ -417,31 +412,25 @@ public void testCreateGetName() { public void testCreateDefaultOutputCoderUsingInference() throws Exception { Coder coder = new RecordCoder(); p.getCoderRegistry().registerCoderForClass(Record.class, coder); - PBegin pBegin = PBegin.in(p); - Create.Values values = Create.of(new Record(), new Record(), new Record()); - Coder defaultCoder = values.getDefaultOutputCoder(pBegin); - assertThat(defaultCoder, equalTo(coder)); + assertThat( + p.apply(Create.of(new Record(), new Record(), new Record())).getCoder(), equalTo(coder)); } @Test public void testCreateDefaultOutputCoderUsingCoder() throws Exception { Coder coder = new RecordCoder(); - PBegin pBegin = PBegin.in(p); - Create.Values values = - Create.of(new Record(), new Record2()).withCoder(coder); - Coder defaultCoder = values.getDefaultOutputCoder(pBegin); - assertThat(defaultCoder, equalTo(coder)); + assertThat( + p.apply(Create.of(new Record(), new Record2()).withCoder(coder)).getCoder(), + equalTo(coder)); } @Test public void testCreateDefaultOutputCoderUsingTypeDescriptor() throws Exception { Coder coder = new RecordCoder(); p.getCoderRegistry().registerCoderForClass(Record.class, coder); - PBegin pBegin = PBegin.in(p); Create.Values values = Create.of(new Record(), new Record2()).withType(new TypeDescriptor() {}); - Coder defaultCoder = values.getDefaultOutputCoder(pBegin); - assertThat(defaultCoder, equalTo(coder)); + assertThat(p.apply(values).getCoder(), equalTo(coder)); } @Test 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 d1b4d07537fd5..cb9d984272f5d 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 @@ -76,18 +76,14 @@ SortValues create( @Override public PCollection>>> expand( PCollection>>> input) { - return input.apply( - ParDo.of( - new SortValuesDoFn( - sorterOptions, - getSecondaryKeyCoder(input.getCoder()), - getValueCoder(input.getCoder())))); - } - - @Override - protected Coder>>> getDefaultOutputCoder( - PCollection>>> input) { - return input.getCoder(); + return input + .apply( + ParDo.of( + new SortValuesDoFn( + sorterOptions, + getSecondaryKeyCoder(input.getCoder()), + getValueCoder(input.getCoder())))) + .setCoder(input.getCoder()); } /** Retrieves the {@link Coder} for the secondary key-value pairs. */ 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 7ca4ce2d2cc7d..6edbd06c0ed06 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 @@ -50,7 +50,6 @@ 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.coders.VoidCoder; import org.apache.beam.sdk.extensions.gcp.options.GcpOptions; import org.apache.beam.sdk.io.BoundedSource; import org.apache.beam.sdk.io.FileSystems; @@ -543,9 +542,7 @@ public PCollection expand(PBegin input) { p.apply("TriggerIdCreation", Create.of(staticJobUuid)) .apply("ViewId", View.asSingleton()); // Apply the traditional Source model. - rows = - p.apply(org.apache.beam.sdk.io.Read.from(createSource(staticJobUuid))) - .setCoder(getDefaultOutputCoder()); + rows = p.apply(org.apache.beam.sdk.io.Read.from(createSource(staticJobUuid))); } else { // Create a singleton job ID token at execution time. jobIdTokenCollection = @@ -625,7 +622,8 @@ public void processElement(ProcessContext c) throws Exception { } } }) - .withSideInputs(schemaView, jobIdTokenView)); + .withSideInputs(schemaView, jobIdTokenView)) + .setCoder(TableRowJsonCoder.of()); } PassThroughThenCleanup.CleanupOperation cleanupOperation = new PassThroughThenCleanup.CleanupOperation() { @@ -657,11 +655,6 @@ void cleanup(PassThroughThenCleanup.ContextContainer c) throws Exception { return rows.apply(new PassThroughThenCleanup(cleanupOperation, jobIdTokenView)); } - @Override - protected Coder getDefaultOutputCoder() { - return TableRowJsonCoder.of(); - } - @Override public void populateDisplayData(DisplayData.Builder builder) { super.populateDisplayData(builder); @@ -1140,11 +1133,6 @@ private WriteResult expandTyped( } } - @Override - protected Coder getDefaultOutputCoder() { - return VoidCoder.of(); - } - @Override public void populateDisplayData(DisplayData.Builder builder) { super.populateDisplayData(builder); diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/PassThroughThenCleanup.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/PassThroughThenCleanup.java index de26c8dc7a447..2f7da08b9cb1a 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/PassThroughThenCleanup.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/PassThroughThenCleanup.java @@ -74,7 +74,7 @@ public void processElement(ProcessContext c) throws Exception { }) .withSideInputs(jobIdSideInput, cleanupSignalView)); - return outputs.get(mainOutput); + return outputs.get(mainOutput).setCoder(input.getCoder()); } private static class IdentityFn extends DoFn { diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StreamingInserts.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StreamingInserts.java index ba09cb38d8d23..747f2b0525815 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StreamingInserts.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StreamingInserts.java @@ -19,8 +19,6 @@ package org.apache.beam.sdk.io.gcp.bigquery; import com.google.api.services.bigquery.model.TableRow; -import org.apache.beam.sdk.coders.Coder; -import org.apache.beam.sdk.coders.VoidCoder; import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.Write.CreateDisposition; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.values.KV; @@ -67,12 +65,6 @@ StreamingInserts withTestServices(BigQueryServices bigQueryService return new StreamingInserts<>( createDisposition, dynamicDestinations, bigQueryServices, retryPolicy); } - - @Override - protected Coder getDefaultOutputCoder() { - return VoidCoder.of(); - } - @Override public WriteResult expand(PCollection> input) { PCollection> writes = diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubIO.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubIO.java index 4f33d61ae4034..46c2df4b5f18f 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubIO.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubIO.java @@ -36,7 +36,6 @@ import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.CoderException; import org.apache.beam.sdk.coders.StringUtf8Coder; -import org.apache.beam.sdk.coders.VoidCoder; import org.apache.beam.sdk.extensions.protobuf.ProtoCoder; import org.apache.beam.sdk.io.gcp.pubsub.PubsubClient.OutgoingMessage; import org.apache.beam.sdk.io.gcp.pubsub.PubsubClient.ProjectPath; @@ -727,7 +726,7 @@ public PCollection expand(PBegin input) { getTimestampAttribute(), getIdAttribute(), getNeedsAttributes()); - return input.apply(source).apply(MapElements.via(getParseFn())); + return input.apply(source).apply(MapElements.via(getParseFn())).setCoder(getCoder()); } @Override @@ -743,11 +742,6 @@ public void populateDisplayData(DisplayData.Builder builder) { .withLabel("Pubsub Subscription")); } } - - @Override - protected Coder getDefaultOutputCoder() { - return getCoder(); - } } ///////////////////////////////////////////////////////////////////////////// @@ -870,11 +864,6 @@ public void populateDisplayData(DisplayData.Builder builder) { builder, getTimestampAttribute(), getIdAttribute(), getTopicProvider()); } - @Override - protected Coder getDefaultOutputCoder() { - return VoidCoder.of(); - } - /** * Writer to Pubsub which batches messages from bounded collections. * From 48690bc61673e767d4a1fa72e0499c32f160db39 Mon Sep 17 00:00:00 2001 From: Eugene Kirpichov Date: Thu, 3 Aug 2017 17:35:03 -0700 Subject: [PATCH 240/346] Remembers the output coders of SplittableParDo --- .../apache/beam/runners/apex/ApexRunner.java | 5 +- .../core/construction/SplittableParDo.java | 65 ++++++++++--------- .../construction/SplittableParDoTest.java | 33 +++++----- .../SplittableParDoViaKeyedWorkItems.java | 1 + .../direct/ParDoMultiOverrideFactory.java | 2 +- .../dataflow/SplittableParDoOverrides.java | 2 +- 6 files changed, 56 insertions(+), 52 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 cee524ee29fa5..57d259301b135 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 @@ -379,8 +379,9 @@ static class SplittableParDoOverrideFactory implements PTransfo public PTransformReplacement, PCollectionTuple> getReplacementTransform( AppliedPTransform, PCollectionTuple, MultiOutput> transform) { - return PTransformReplacement.of(PTransformReplacements.getSingletonMainInput(transform), - SplittableParDo.forJavaParDo(transform.getTransform())); + return PTransformReplacement.of( + PTransformReplacements.getSingletonMainInput(transform), + SplittableParDo.forAppliedParDo(transform)); } @Override diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/SplittableParDo.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/SplittableParDo.java index bcc5de864187b..32d3409359e16 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/SplittableParDo.java +++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/SplittableParDo.java @@ -19,8 +19,8 @@ import static com.google.common.base.Preconditions.checkArgument; +import com.google.common.collect.Maps; import java.io.IOException; -import java.util.Collections; import java.util.List; import java.util.Map; import java.util.UUID; @@ -74,6 +74,7 @@ public class SplittableParDo private final List> sideInputs; private final TupleTag mainOutputTag; private final TupleTagList additionalOutputTags; + private final Map, Coder> outputTagsToCoders; public static final String SPLITTABLE_PROCESS_URN = "urn:beam:runners_core:transforms:splittable_process:v1"; @@ -86,34 +87,18 @@ public class SplittableParDo private SplittableParDo( DoFn doFn, - TupleTag mainOutputTag, List> sideInputs, - TupleTagList additionalOutputTags) { + TupleTag mainOutputTag, + TupleTagList additionalOutputTags, + Map, Coder> outputTagsToCoders) { checkArgument( DoFnSignatures.getSignature(doFn.getClass()).processElement().isSplittable(), "fn must be a splittable DoFn"); this.doFn = doFn; - this.mainOutputTag = mainOutputTag; this.sideInputs = sideInputs; + this.mainOutputTag = mainOutputTag; this.additionalOutputTags = additionalOutputTags; - } - - /** - * Creates a {@link SplittableParDo} from an original Java {@link ParDo}. - * - * @param parDo The splittable {@link ParDo} transform. - */ - public static SplittableParDo forJavaParDo( - ParDo.MultiOutput parDo) { - checkArgument(parDo != null, "parDo must not be null"); - checkArgument( - DoFnSignatures.getSignature(parDo.getFn().getClass()).processElement().isSplittable(), - "fn must be a splittable DoFn"); - return new SplittableParDo( - parDo.getFn(), - parDo.getMainOutputTag(), - parDo.getSideInputs(), - parDo.getAdditionalOutputTags()); + this.outputTagsToCoders = outputTagsToCoders; } /** @@ -122,15 +107,22 @@ private SplittableParDo( *

              The input may generally be a deserialized transform so it may not actually be a {@link * ParDo}. Instead {@link ParDoTranslation} will be used to extract fields. */ - public static SplittableParDo forAppliedParDo(AppliedPTransform parDo) { + @SuppressWarnings({"unchecked", "rawtypes"}) + public static SplittableParDo forAppliedParDo( + AppliedPTransform, PCollectionTuple, ?> parDo) { checkArgument(parDo != null, "parDo must not be null"); try { - return new SplittableParDo<>( + Map, Coder> outputTagsToCoders = Maps.newHashMap(); + for (Map.Entry, PValue> entry : parDo.getOutputs().entrySet()) { + outputTagsToCoders.put(entry.getKey(), ((PCollection) entry.getValue()).getCoder()); + } + return new SplittableParDo( ParDoTranslation.getDoFn(parDo), - (TupleTag) ParDoTranslation.getMainOutputTag(parDo), ParDoTranslation.getSideInputs(parDo), - ParDoTranslation.getAdditionalOutputTags(parDo)); + ParDoTranslation.getMainOutputTag(parDo), + ParDoTranslation.getAdditionalOutputTags(parDo), + outputTagsToCoders); } catch (IOException exc) { throw new RuntimeException(exc); } @@ -169,7 +161,8 @@ public PCollectionTuple expand(PCollection input) { (WindowingStrategy) input.getWindowingStrategy(), sideInputs, mainOutputTag, - additionalOutputTags)); + additionalOutputTags, + outputTagsToCoders)); } @Override @@ -203,6 +196,7 @@ public static class ProcessKeyedElements private final List> sideInputs; private final TupleTag mainOutputTag; private final TupleTagList additionalOutputTags; + private final Map, Coder> outputTagsToCoders; /** * @param fn the splittable {@link DoFn}. @@ -210,7 +204,8 @@ public static class ProcessKeyedElements * @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 additionalOutputTags {@link TupleTagList Tags} of the {@link DoFn DoFn's} additional - * outputs. + * @param outputTagsToCoders A map from output tag to the coder for that output, which should + * provide mappings for the main and all additional tags. */ public ProcessKeyedElements( DoFn fn, @@ -219,7 +214,8 @@ public ProcessKeyedElements( WindowingStrategy windowingStrategy, List> sideInputs, TupleTag mainOutputTag, - TupleTagList additionalOutputTags) { + TupleTagList additionalOutputTags, + Map, Coder> outputTagsToCoders) { this.fn = fn; this.elementCoder = elementCoder; this.restrictionCoder = restrictionCoder; @@ -227,6 +223,7 @@ public ProcessKeyedElements( this.sideInputs = sideInputs; this.mainOutputTag = mainOutputTag; this.additionalOutputTags = additionalOutputTags; + this.outputTagsToCoders = outputTagsToCoders; } public DoFn getFn() { @@ -257,10 +254,14 @@ public TupleTagList getAdditionalOutputTags() { return additionalOutputTags; } + public Map, Coder> getOutputTagsToCoders() { + return outputTagsToCoders; + } + @Override public PCollectionTuple expand(PCollection>> input) { return createPrimitiveOutputFor( - input, fn, mainOutputTag, additionalOutputTags, windowingStrategy); + input, fn, mainOutputTag, additionalOutputTags, outputTagsToCoders, windowingStrategy); } public static PCollectionTuple createPrimitiveOutputFor( @@ -268,14 +269,14 @@ public static PCollectionTuple createPrimitiveOutputFor( DoFn fn, TupleTag mainOutputTag, TupleTagList additionalOutputTags, + Map, Coder> outputTagsToCoders, WindowingStrategy windowingStrategy) { DoFnSignature signature = DoFnSignatures.getSignature(fn.getClass()); PCollectionTuple outputs = PCollectionTuple.ofPrimitiveOutputsInternal( input.getPipeline(), TupleTagList.of(mainOutputTag).and(additionalOutputTags.getAll()), - // TODO - Collections., Coder>emptyMap(), + outputTagsToCoders, windowingStrategy, input.isBounded().and(signature.isBoundedPerElement())); diff --git a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/SplittableParDoTest.java b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/SplittableParDoTest.java index 267232c028f6a..05c471dd2da6a 100644 --- a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/SplittableParDoTest.java +++ b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/SplittableParDoTest.java @@ -22,6 +22,7 @@ import java.io.Serializable; import org.apache.beam.sdk.Pipeline; +import org.apache.beam.sdk.runners.AppliedPTransform; import org.apache.beam.sdk.testing.TestPipeline; import org.apache.beam.sdk.transforms.Create; import org.apache.beam.sdk.transforms.DoFn; @@ -29,6 +30,7 @@ import org.apache.beam.sdk.transforms.splittabledofn.HasDefaultTracker; import org.apache.beam.sdk.transforms.splittabledofn.RestrictionTracker; import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.PCollectionTuple; import org.apache.beam.sdk.values.TupleTag; import org.apache.beam.sdk.values.TupleTagList; import org.junit.Rule; @@ -106,12 +108,18 @@ private static PCollection makeBoundedCollection(Pipeline pipeline) { private static final TupleTag MAIN_OUTPUT_TAG = new TupleTag() {}; - private ParDo.MultiOutput makeParDo(DoFn fn) { - return ParDo.of(fn).withOutputTags(MAIN_OUTPUT_TAG, TupleTagList.empty()); + private PCollection applySplittableParDo( + String name, PCollection input, DoFn fn) { + ParDo.MultiOutput multiOutput = + ParDo.of(fn).withOutputTags(MAIN_OUTPUT_TAG, TupleTagList.empty()); + PCollectionTuple output = multiOutput.expand(input); + output.get(MAIN_OUTPUT_TAG).setName("main"); + AppliedPTransform, PCollectionTuple, ?> transform = + AppliedPTransform.of("ParDo", input.expand(), output.expand(), multiOutput, pipeline); + return input.apply(name, SplittableParDo.forAppliedParDo(transform)).get(MAIN_OUTPUT_TAG); } - @Rule - public TestPipeline pipeline = TestPipeline.create(); + @Rule public TestPipeline pipeline = TestPipeline.create(); @Test public void testBoundednessForBoundedFn() { @@ -121,16 +129,12 @@ public void testBoundednessForBoundedFn() { assertEquals( "Applying a bounded SDF to a bounded collection produces a bounded collection", PCollection.IsBounded.BOUNDED, - makeBoundedCollection(pipeline) - .apply("bounded to bounded", SplittableParDo.forJavaParDo(makeParDo(boundedFn))) - .get(MAIN_OUTPUT_TAG) + applySplittableParDo("bounded to bounded", makeBoundedCollection(pipeline), boundedFn) .isBounded()); assertEquals( "Applying a bounded SDF to an unbounded collection produces an unbounded collection", PCollection.IsBounded.UNBOUNDED, - makeUnboundedCollection(pipeline) - .apply("bounded to unbounded", SplittableParDo.forJavaParDo(makeParDo(boundedFn))) - .get(MAIN_OUTPUT_TAG) + applySplittableParDo("bounded to unbounded", makeUnboundedCollection(pipeline), boundedFn) .isBounded()); } @@ -142,16 +146,13 @@ public void testBoundednessForUnboundedFn() { assertEquals( "Applying an unbounded SDF to a bounded collection produces a bounded collection", PCollection.IsBounded.UNBOUNDED, - makeBoundedCollection(pipeline) - .apply("unbounded to bounded", SplittableParDo.forJavaParDo(makeParDo(unboundedFn))) - .get(MAIN_OUTPUT_TAG) + applySplittableParDo("unbounded to bounded", makeBoundedCollection(pipeline), unboundedFn) .isBounded()); assertEquals( "Applying an unbounded SDF to an unbounded collection produces an unbounded collection", PCollection.IsBounded.UNBOUNDED, - makeUnboundedCollection(pipeline) - .apply("unbounded to unbounded", SplittableParDo.forJavaParDo(makeParDo(unboundedFn))) - .get(MAIN_OUTPUT_TAG) + applySplittableParDo( + "unbounded to unbounded", makeUnboundedCollection(pipeline), unboundedFn) .isBounded()); } } diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/SplittableParDoViaKeyedWorkItems.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/SplittableParDoViaKeyedWorkItems.java index af720fd591832..251260e2aecb6 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/SplittableParDoViaKeyedWorkItems.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/SplittableParDoViaKeyedWorkItems.java @@ -184,6 +184,7 @@ public PCollectionTuple expand( original.getFn(), original.getMainOutputTag(), original.getAdditionalOutputTags(), + original.getOutputTagsToCoders(), original.getInputWindowingStrategy()); } } 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 3f04b56123ae0..26f30b0f992f1 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 @@ -96,7 +96,7 @@ private PTransform, PCollectionTuple> getReplaceme DoFnSignature signature = DoFnSignatures.getSignature(fn.getClass()); if (signature.processElement().isSplittable()) { - return (PTransform) SplittableParDo.forAppliedParDo(application); + return SplittableParDo.forAppliedParDo((AppliedPTransform) application); } else if (signature.stateDeclarations().size() > 0 || signature.timerDeclarations().size() > 0) { return new GbkThenStatefulParDo( diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/SplittableParDoOverrides.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/SplittableParDoOverrides.java index fc010f81aadd1..7b65950910426 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/SplittableParDoOverrides.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/SplittableParDoOverrides.java @@ -64,7 +64,7 @@ public PTransformReplacement, PCollectionTuple> getReplaceme appliedTransform) { return PTransformReplacement.of( PTransformReplacements.getSingletonMainInput(appliedTransform), - SplittableParDo.forJavaParDo(appliedTransform.getTransform())); + SplittableParDo.forAppliedParDo(appliedTransform)); } @Override From 5e71d53ef8c28ec14b6a282b1fe67489c2b3f243 Mon Sep 17 00:00:00 2001 From: Robert Bradshaw Date: Thu, 3 Aug 2017 09:26:49 -0700 Subject: [PATCH 241/346] Refactor FnApiRunner to operate directly on the runner API protos. This allows for optimization and execution of pipelines in other langauges over the Fn API (modulo aligning URNs and using the runner API for Coders). The only portions of the pipeline that are deserialized are the Coders. --- sdks/python/apache_beam/coders/stream.pxd | 2 +- sdks/python/apache_beam/coders/stream.pyx | 2 +- .../apache_beam/runners/pipeline_context.py | 8 +- .../runners/portability/fn_api_runner.py | 569 +++++++++++++++++- .../runners/portability/fn_api_runner_test.py | 2 +- .../runners/worker/bundle_processor.py | 107 +++- sdks/python/apache_beam/transforms/core.py | 6 +- 7 files changed, 678 insertions(+), 18 deletions(-) diff --git a/sdks/python/apache_beam/coders/stream.pxd b/sdks/python/apache_beam/coders/stream.pxd index 4e01a893bfaae..ade9b722c6ea2 100644 --- a/sdks/python/apache_beam/coders/stream.pxd +++ b/sdks/python/apache_beam/coders/stream.pxd @@ -53,7 +53,7 @@ cdef class InputStream(object): cdef bytes all cdef char* allc - cpdef size_t size(self) except? -1 + cpdef ssize_t size(self) except? -1 cpdef bytes read(self, size_t len) cpdef long read_byte(self) except? -1 cpdef libc.stdint.int64_t read_var_int64(self) except? -1 diff --git a/sdks/python/apache_beam/coders/stream.pyx b/sdks/python/apache_beam/coders/stream.pyx index 8d97681692201..7c9521a863794 100644 --- a/sdks/python/apache_beam/coders/stream.pyx +++ b/sdks/python/apache_beam/coders/stream.pyx @@ -167,7 +167,7 @@ cdef class InputStream(object): # unsigned char here. return ( self.allc[self.pos - 1]) - cpdef size_t size(self) except? -1: + cpdef ssize_t size(self) except? -1: return len(self.all) - self.pos cpdef bytes read_all(self, bint nested=False): diff --git a/sdks/python/apache_beam/runners/pipeline_context.py b/sdks/python/apache_beam/runners/pipeline_context.py index f4de42ac82af1..42d7f5df14a86 100644 --- a/sdks/python/apache_beam/runners/pipeline_context.py +++ b/sdks/python/apache_beam/runners/pipeline_context.py @@ -40,7 +40,7 @@ def __init__(self, context, obj_type, proto_map=None): self._obj_type = obj_type self._obj_to_id = {} self._id_to_obj = {} - self._id_to_proto = proto_map if proto_map else {} + self._id_to_proto = dict(proto_map) if proto_map else {} self._counter = 0 def _unique_ref(self, obj=None, label=None): @@ -66,6 +66,12 @@ def get_by_id(self, id): self._id_to_proto[id], self._pipeline_context) return self._id_to_obj[id] + def __getitem__(self, id): + return self.get_by_id(id) + + def __contains__(self, id): + return id in self._id_to_proto + class PipelineContext(object): """For internal use only; no backwards-compatibility guarantees. diff --git a/sdks/python/apache_beam/runners/portability/fn_api_runner.py b/sdks/python/apache_beam/runners/portability/fn_api_runner.py index f88fe53309beb..3222bcb9ba469 100644 --- a/sdks/python/apache_beam/runners/portability/fn_api_runner.py +++ b/sdks/python/apache_beam/runners/portability/fn_api_runner.py @@ -19,6 +19,7 @@ """ import base64 import collections +import copy import logging import Queue as queue import threading @@ -28,21 +29,26 @@ import grpc import apache_beam as beam # pylint: disable=ungrouped-imports +from apache_beam.coders import registry from apache_beam.coders import WindowedValueCoder from apache_beam.coders.coder_impl import create_InputStream from apache_beam.coders.coder_impl import create_OutputStream from apache_beam.internal import pickler from apache_beam.io import iobase -from apache_beam.transforms.window import GlobalWindows +from apache_beam.metrics.execution import MetricsEnvironment from apache_beam.portability.api import beam_fn_api_pb2 from apache_beam.portability.api import beam_runner_api_pb2 from apache_beam.runners import pipeline_context from apache_beam.runners.portability import maptask_executor_runner +from apache_beam.runners.runner import PipelineState from apache_beam.runners.worker import bundle_processor from apache_beam.runners.worker import data_plane from apache_beam.runners.worker import operation_specs from apache_beam.runners.worker import sdk_worker +from apache_beam.transforms.window import GlobalWindows from apache_beam.utils import proto_utils +from apache_beam.utils import urns + # This module is experimental. No backwards-compatibility guarantees. @@ -113,6 +119,30 @@ def process(self, source): beam.transforms.core.Windowing(GlobalWindows()))) +class _GroupingBuffer(object): + """Used to accumulate groupded (shuffled) results.""" + def __init__(self, pre_grouped_coder, post_grouped_coder): + self._key_coder = pre_grouped_coder.value_coder().key_coder() + self._pre_grouped_coder = pre_grouped_coder + self._post_grouped_coder = post_grouped_coder + self._table = collections.defaultdict(list) + + def append(self, elements_data): + input_stream = create_InputStream(elements_data) + while input_stream.size() > 0: + key, value = self._pre_grouped_coder.get_impl().decode_from_stream( + input_stream, True).value + self._table[self._key_coder.encode(key)].append(value) + + def __iter__(self): + output_stream = create_OutputStream() + for encoded_key, values in self._table.items(): + key = self._key_coder.decode(encoded_key) + self._post_grouped_coder.get_impl().encode_to_stream( + GlobalWindows.windowed_value((key, values)), output_stream, True) + return iter([output_stream.get()]) + + class FnApiRunner(maptask_executor_runner.MapTaskExecutorRunner): def __init__(self): @@ -126,6 +156,520 @@ def _next_uid(self): self._last_uid += 1 return str(self._last_uid) + def run(self, pipeline): + MetricsEnvironment.set_metrics_supported(self.has_metrics_support()) + if pipeline._verify_runner_api_compatible(): + return self.run_via_runner_api(pipeline.to_runner_api()) + else: + return super(FnApiRunner, self).run(pipeline) + + def run_via_runner_api(self, pipeline_proto): + return self.run_stages(*self.create_stages(pipeline_proto)) + + def create_stages(self, pipeline_proto): + + # First define a couple of helpers. + + def union(a, b): + # Minimize the number of distinct sets. + if not a or a == b: + return b + elif not b: + return a + else: + return frozenset.union(a, b) + + class Stage(object): + """A set of Transforms that can be sent to the worker for processing.""" + def __init__(self, name, transforms, + downstream_side_inputs=None, must_follow=frozenset()): + self.name = name + self.transforms = transforms + self.downstream_side_inputs = downstream_side_inputs + self.must_follow = must_follow + + def __repr__(self): + must_follow = ', '.join(prev.name for prev in self.must_follow) + return "%s\n %s\n must follow: %s" % ( + self.name, + '\n'.join(["%s:%s" % (transform.unique_name, transform.spec.urn) + for transform in self.transforms]), + must_follow) + + def can_fuse(self, consumer): + def no_overlap(a, b): + return not a.intersection(b) + return ( + not self in consumer.must_follow + and not self.is_flatten() and not consumer.is_flatten() + and no_overlap(self.downstream_side_inputs, consumer.side_inputs())) + + def fuse(self, other): + return Stage( + "(%s)+(%s)" % (self.name, other.name), + self.transforms + other.transforms, + union(self.downstream_side_inputs, other.downstream_side_inputs), + union(self.must_follow, other.must_follow)) + + def is_flatten(self): + return any(transform.spec.urn == urns.FLATTEN_TRANSFORM + for transform in self.transforms) + + def side_inputs(self): + for transform in self.transforms: + if transform.spec.urn == urns.PARDO_TRANSFORM: + payload = proto_utils.unpack_Any( + transform.spec.parameter, beam_runner_api_pb2.ParDoPayload) + for side_input in payload.side_inputs: + yield transform.inputs[side_input] + + def has_as_main_input(self, pcoll): + for transform in self.transforms: + if transform.spec.urn == urns.PARDO_TRANSFORM: + payload = proto_utils.unpack_Any( + transform.spec.parameter, beam_runner_api_pb2.ParDoPayload) + local_side_inputs = payload.side_inputs + else: + local_side_inputs = {} + for local_id, pipeline_id in transform.inputs.items(): + if pcoll == pipeline_id and local_id not in local_side_inputs: + return True + + def deduplicate_read(self): + seen_pcolls = set() + new_transforms = [] + for transform in self.transforms: + if transform.spec.urn == bundle_processor.DATA_INPUT_URN: + pcoll = only_element(transform.outputs.items())[1] + if pcoll in seen_pcolls: + continue + seen_pcolls.add(pcoll) + new_transforms.append(transform) + self.transforms = new_transforms + + # Now define the "optimization" phases. + + def expand_gbk(stages): + """Transforms each GBK into a write followed by a read. + """ + for stage in stages: + assert len(stage.transforms) == 1 + transform = stage.transforms[0] + if transform.spec.urn == urns.GROUP_BY_KEY_ONLY_TRANSFORM: + # This is used later to correlate the read and write. + param = proto_utils.pack_Any( + wrappers_pb2.BytesValue( + value=str("group:%s" % stage.name))) + gbk_write = Stage( + transform.unique_name + '/Write', + [beam_runner_api_pb2.PTransform( + unique_name=transform.unique_name + '/Write', + inputs=transform.inputs, + spec=beam_runner_api_pb2.FunctionSpec( + urn=bundle_processor.DATA_OUTPUT_URN, + parameter=param))], + downstream_side_inputs=frozenset(), + must_follow=stage.must_follow) + yield gbk_write + + yield Stage( + transform.unique_name + '/Read', + [beam_runner_api_pb2.PTransform( + unique_name=transform.unique_name + '/Read', + outputs=transform.outputs, + spec=beam_runner_api_pb2.FunctionSpec( + urn=bundle_processor.DATA_INPUT_URN, + parameter=param))], + downstream_side_inputs=frozenset(), + must_follow=union(frozenset([gbk_write]), stage.must_follow)) + else: + yield stage + + def sink_flattens(stages): + """Sink flattens and remove them from the graph. + + A flatten that cannot be sunk/fused away becomes multiple writes (to the + same logical sink) followed by a read. + """ + # TODO(robertwb): Actually attempt to sink rather than always materialize. + # TODO(robertwb): Possibly fuse this into one of the stages. + pcollections = pipeline_components.pcollections + for stage in stages: + assert len(stage.transforms) == 1 + transform = stage.transforms[0] + if transform.spec.urn == urns.FLATTEN_TRANSFORM: + # This is used later to correlate the read and writes. + param = proto_utils.pack_Any( + wrappers_pb2.BytesValue( + value=str("materialize:%s" % transform.unique_name))) + output_pcoll_id, = transform.outputs.values() + output_coder_id = pcollections[output_pcoll_id].coder_id + flatten_writes = [] + for local_in, pcoll_in in transform.inputs.items(): + + if pcollections[pcoll_in].coder_id != output_coder_id: + # Flatten inputs must all be written with the same coder as is + # used to read them. + pcollections[pcoll_in].coder_id = output_coder_id + transcoded_pcollection = ( + transform.unique_name + '/Transcode/' + local_in + '/out') + yield Stage( + transform.unique_name + '/Transcode/' + local_in, + [beam_runner_api_pb2.PTransform( + unique_name= + transform.unique_name + '/Transcode/' + local_in, + inputs={local_in: pcoll_in}, + outputs={'out': transcoded_pcollection}, + spec=beam_runner_api_pb2.FunctionSpec( + urn=bundle_processor.IDENTITY_DOFN_URN))], + downstream_side_inputs=frozenset(), + must_follow=stage.must_follow) + pcollections[transcoded_pcollection].CopyFrom( + pcollections[pcoll_in]) + pcollections[transcoded_pcollection].coder_id = output_coder_id + else: + transcoded_pcollection = pcoll_in + + flatten_write = Stage( + transform.unique_name + '/Write/' + local_in, + [beam_runner_api_pb2.PTransform( + unique_name=transform.unique_name + '/Write/' + local_in, + inputs={local_in: transcoded_pcollection}, + spec=beam_runner_api_pb2.FunctionSpec( + urn=bundle_processor.DATA_OUTPUT_URN, + parameter=param))], + downstream_side_inputs=frozenset(), + must_follow=stage.must_follow) + flatten_writes.append(flatten_write) + yield flatten_write + + yield Stage( + transform.unique_name + '/Read', + [beam_runner_api_pb2.PTransform( + unique_name=transform.unique_name + '/Read', + outputs=transform.outputs, + spec=beam_runner_api_pb2.FunctionSpec( + urn=bundle_processor.DATA_INPUT_URN, + parameter=param))], + downstream_side_inputs=frozenset(), + must_follow=union(frozenset(flatten_writes), stage.must_follow)) + + else: + yield stage + + def annotate_downstream_side_inputs(stages): + """Annotate each stage with fusion-prohibiting information. + + Each stage is annotated with the (transitive) set of pcollections that + depend on this stage that are also used later in the pipeline as a + side input. + + While theoretically this could result in O(n^2) annotations, the size of + each set is bounded by the number of side inputs (typically much smaller + than the number of total nodes) and the number of *distinct* side-input + sets is also generally small (and shared due to the use of union + defined above). + + This representation is also amenable to simple recomputation on fusion. + """ + consumers = collections.defaultdict(list) + all_side_inputs = set() + for stage in stages: + for transform in stage.transforms: + for input in transform.inputs.values(): + consumers[input].append(stage) + for si in stage.side_inputs(): + all_side_inputs.add(si) + all_side_inputs = frozenset(all_side_inputs) + + downstream_side_inputs_by_stage = {} + + def compute_downstream_side_inputs(stage): + if stage not in downstream_side_inputs_by_stage: + downstream_side_inputs = frozenset() + for transform in stage.transforms: + for output in transform.outputs.values(): + if output in all_side_inputs: + downstream_side_inputs = union(downstream_side_inputs, output) + for consumer in consumers[output]: + downstream_side_inputs = union( + downstream_side_inputs, + compute_downstream_side_inputs(consumer)) + downstream_side_inputs_by_stage[stage] = downstream_side_inputs + return downstream_side_inputs_by_stage[stage] + + for stage in stages: + stage.downstream_side_inputs = compute_downstream_side_inputs(stage) + return stages + + def greedily_fuse(stages): + """Places transforms sharing an edge in the same stage, whenever possible. + """ + producers_by_pcoll = {} + consumers_by_pcoll = collections.defaultdict(list) + + # Used to always reference the correct stage as the producer and + # consumer maps are not updated when stages are fused away. + replacements = {} + + def replacement(s): + old_ss = [] + while s in replacements: + old_ss.append(s) + s = replacements[s] + for old_s in old_ss[:-1]: + replacements[old_s] = s + return s + + def fuse(producer, consumer): + fused = producer.fuse(consumer) + replacements[producer] = fused + replacements[consumer] = fused + + # First record the producers and consumers of each PCollection. + for stage in stages: + for transform in stage.transforms: + for input in transform.inputs.values(): + consumers_by_pcoll[input].append(stage) + for output in transform.outputs.values(): + producers_by_pcoll[output] = stage + + logging.debug('consumers\n%s', consumers_by_pcoll) + logging.debug('producers\n%s', producers_by_pcoll) + + # Now try to fuse away all pcollections. + for pcoll, producer in producers_by_pcoll.items(): + pcoll_as_param = proto_utils.pack_Any( + wrappers_pb2.BytesValue( + value=str("materialize:%s" % pcoll))) + write_pcoll = None + for consumer in consumers_by_pcoll[pcoll]: + producer = replacement(producer) + consumer = replacement(consumer) + # Update consumer.must_follow set, as it's used in can_fuse. + consumer.must_follow = set( + replacement(s) for s in consumer.must_follow) + if producer.can_fuse(consumer): + fuse(producer, consumer) + else: + # If we can't fuse, do a read + write. + if write_pcoll is None: + write_pcoll = Stage( + pcoll + '/Write', + [beam_runner_api_pb2.PTransform( + unique_name=pcoll + '/Write', + inputs={'in': pcoll}, + spec=beam_runner_api_pb2.FunctionSpec( + urn=bundle_processor.DATA_OUTPUT_URN, + parameter=pcoll_as_param))]) + fuse(producer, write_pcoll) + if consumer.has_as_main_input(pcoll): + read_pcoll = Stage( + pcoll + '/Read', + [beam_runner_api_pb2.PTransform( + unique_name=pcoll + '/Read', + outputs={'out': pcoll}, + spec=beam_runner_api_pb2.FunctionSpec( + urn=bundle_processor.DATA_INPUT_URN, + parameter=pcoll_as_param))], + must_follow={write_pcoll}) + fuse(read_pcoll, consumer) + + # Everything that was originally a stage or a replacement, but wasn't + # replaced, should be in the final graph. + final_stages = frozenset(stages).union(replacements.values()).difference( + replacements.keys()) + + for stage in final_stages: + # Update all references to their final values before throwing + # the replacement data away. + stage.must_follow = frozenset(replacement(s) for s in stage.must_follow) + # Two reads of the same stage may have been fused. This is unneeded. + stage.deduplicate_read() + return final_stages + + def sort_stages(stages): + """Order stages suitable for sequential execution. + """ + seen = set() + ordered = [] + + def process(stage): + if stage not in seen: + seen.add(stage) + for prev in stage.must_follow: + process(prev) + ordered.append(stage) + for stage in stages: + process(stage) + return ordered + + # Now actually apply the operations. + + pipeline_components = copy.deepcopy(pipeline_proto.components) + + # Reify coders. + # TODO(BEAM-2717): Remove once Coders are already in proto. + coders = pipeline_context.PipelineContext(pipeline_components).coders + for pcoll in pipeline_components.pcollections.values(): + if pcoll.coder_id not in coders: + window_coder = coders[ + pipeline_components.windowing_strategies[ + pcoll.windowing_strategy_id].window_coder_id] + coder = WindowedValueCoder( + registry.get_coder(pickler.loads(pcoll.coder_id)), + window_coder=window_coder) + pcoll.coder_id = coders.get_id(coder) + coders.populate_map(pipeline_components.coders) + + # Initial set of stages are singleton transforms. + stages = [ + Stage(name, [transform]) + for name, transform in pipeline_proto.components.transforms.items() + if not transform.subtransforms] + + # Apply each phase in order. + for phase in [ + annotate_downstream_side_inputs, expand_gbk, sink_flattens, + greedily_fuse, sort_stages]: + logging.info('%s %s %s', '=' * 20, phase, '=' * 20) + stages = list(phase(stages)) + logging.debug('Stages: %s', [str(s) for s in stages]) + + # Return the (possibly mutated) context and ordered set of stages. + return pipeline_components, stages + + def run_stages(self, pipeline_components, stages, direct=True): + + if direct: + controller = FnApiRunner.DirectController() + else: + controller = FnApiRunner.GrpcController() + + try: + pcoll_buffers = collections.defaultdict(list) + for stage in stages: + self.run_stage(controller, pipeline_components, stage, pcoll_buffers) + finally: + controller.close() + + return maptask_executor_runner.WorkerRunnerResult(PipelineState.DONE) + + def run_stage(self, controller, pipeline_components, stage, pcoll_buffers): + + coders = pipeline_context.PipelineContext(pipeline_components).coders + data_operation_spec = controller.data_operation_spec() + + def extract_endpoints(stage): + # Returns maps of transform names to PCollection identifiers. + # Also mutates IO stages to point to the data data_operation_spec. + data_input = {} + data_side_input = {} + data_output = {} + for transform in stage.transforms: + pcoll_id = proto_utils.unpack_Any( + transform.spec.parameter, wrappers_pb2.BytesValue).value + if transform.spec.urn in (bundle_processor.DATA_INPUT_URN, + bundle_processor.DATA_OUTPUT_URN): + if transform.spec.urn == bundle_processor.DATA_INPUT_URN: + target = transform.unique_name, only_element(transform.outputs) + data_input[target] = pcoll_id + elif transform.spec.urn == bundle_processor.DATA_OUTPUT_URN: + target = transform.unique_name, only_element(transform.inputs) + data_output[target] = pcoll_id + else: + raise NotImplementedError + if data_operation_spec: + transform.spec.parameter.CopyFrom(data_operation_spec) + else: + transform.spec.parameter.Clear() + return data_input, data_side_input, data_output + + logging.info('Running %s', stage.name) + logging.debug(' %s', stage) + data_input, data_side_input, data_output = extract_endpoints(stage) + if data_side_input: + raise NotImplementedError('Side inputs.') + + process_bundle_descriptor = beam_fn_api_pb2.ProcessBundleDescriptor( + id=self._next_uid(), + transforms={transform.unique_name: transform + for transform in stage.transforms}, + pcollections=dict(pipeline_components.pcollections.items()), + coders=dict(pipeline_components.coders.items()), + windowing_strategies=dict( + pipeline_components.windowing_strategies.items()), + environments=dict(pipeline_components.environments.items())) + + process_bundle_registration = beam_fn_api_pb2.InstructionRequest( + instruction_id=self._next_uid(), + register=beam_fn_api_pb2.RegisterRequest( + process_bundle_descriptor=[process_bundle_descriptor])) + + process_bundle = beam_fn_api_pb2.InstructionRequest( + instruction_id=self._next_uid(), + process_bundle=beam_fn_api_pb2.ProcessBundleRequest( + process_bundle_descriptor_reference= + process_bundle_descriptor.id)) + + # Write all the input data to the channel. + for (transform_id, name), pcoll_id in data_input.items(): + data_out = controller.data_plane_handler.output_stream( + process_bundle.instruction_id, beam_fn_api_pb2.Target( + primitive_transform_reference=transform_id, name=name)) + for element_data in pcoll_buffers[pcoll_id]: + data_out.write(element_data) + data_out.close() + + # Register and start running the bundle. + controller.control_handler.push(process_bundle_registration) + controller.control_handler.push(process_bundle) + + # Wait for the bundle to finish. + while True: + result = controller.control_handler.pull() + if result.instruction_id == process_bundle.instruction_id: + if result.error: + raise RuntimeError(result.error) + break + + # Gather all output data. + expected_targets = [ + beam_fn_api_pb2.Target(primitive_transform_reference=transform_id, + name=output_name) + for (transform_id, output_name), _ in data_output.items()] + for output in controller.data_plane_handler.input_elements( + process_bundle.instruction_id, expected_targets): + target_tuple = ( + output.target.primitive_transform_reference, output.target.name) + if target_tuple in data_output: + pcoll_id = data_output[target_tuple] + if pcoll_id.startswith('materialize:'): + # Just store the data chunks for replay. + pcoll_buffers[pcoll_id].append(output.data) + elif pcoll_id.startswith('group:'): + # This is a grouping write, create a grouping buffer if needed. + if pcoll_id not in pcoll_buffers: + original_gbk_transform = pcoll_id.split(':', 1)[1] + transform_proto = pipeline_components.transforms[ + original_gbk_transform] + input_pcoll = only_element(transform_proto.inputs.values()) + output_pcoll = only_element(transform_proto.outputs.values()) + pre_gbk_coder = coders[ + pipeline_components.pcollections[input_pcoll].coder_id] + post_gbk_coder = coders[ + pipeline_components.pcollections[output_pcoll].coder_id] + pcoll_buffers[pcoll_id] = _GroupingBuffer( + pre_gbk_coder, post_gbk_coder) + pcoll_buffers[pcoll_id].append(output.data) + else: + # These should be the only two identifiers we produce for now, + # but special side input writes may go here. + raise NotImplementedError(pcoll_id) + + # This is the "old" way of executing pipelines. + # TODO(robertwb): Remove once runner API supports side inputs. + def _map_task_registration(self, map_task, state_handler, data_operation_spec): input_data, side_input_data, runner_sinks, process_bundle_descriptor = ( @@ -175,10 +719,6 @@ def get_outputs(op_ix): return {tag: pcollection_id(op_ix, out_ix) for out_ix, tag in enumerate(getattr(op, 'output_tags', ['out']))} - def only_element(iterable): - element, = iterable - return element - for op_ix, (stage_name, operation) in enumerate(map_task): transform_id = uniquify(stage_name) @@ -332,6 +872,15 @@ def execute_map_tasks(self, ordered_map_tasks, direct=False): finally: controller.close() + @staticmethod + def _reencode_elements(elements, element_coder): + output_stream = create_OutputStream() + for element in elements: + element_coder.get_impl().encode_to_stream(element, output_stream, True) + return output_stream.get() + + # These classes are used to interact with the worker. + class SimpleState(object): # TODO(robertwb): Inherit from GRPC servicer. def __init__(self): @@ -429,9 +978,7 @@ def close(self): self.control_server.stop(5).wait() self.data_server.stop(5).wait() - @staticmethod - def _reencode_elements(elements, element_coder): - output_stream = create_OutputStream() - for element in elements: - element_coder.get_impl().encode_to_stream(element, output_stream, True) - return output_stream.get() + +def only_element(iterable): + element, = iterable + return element diff --git a/sdks/python/apache_beam/runners/portability/fn_api_runner_test.py b/sdks/python/apache_beam/runners/portability/fn_api_runner_test.py index 163e98029467c..ba219547e6cfe 100644 --- a/sdks/python/apache_beam/runners/portability/fn_api_runner_test.py +++ b/sdks/python/apache_beam/runners/portability/fn_api_runner_test.py @@ -51,7 +51,7 @@ def test_pardo_unfusable_side_inputs(self): def test_assert_that(self): # TODO: figure out a way for fn_api_runner to parse and raise the # underlying exception. - with self.assertRaisesRegexp(RuntimeError, 'BeamAssertException'): + with self.assertRaisesRegexp(Exception, 'Failed assert'): with self.create_pipeline() as p: assert_that(p | beam.Create(['a', 'b']), equal_to(['a'])) diff --git a/sdks/python/apache_beam/runners/worker/bundle_processor.py b/sdks/python/apache_beam/runners/worker/bundle_processor.py index 2669bfce947a2..9474eda4725f5 100644 --- a/sdks/python/apache_beam/runners/worker/bundle_processor.py +++ b/sdks/python/apache_beam/runners/worker/bundle_processor.py @@ -28,17 +28,20 @@ from google.protobuf import wrappers_pb2 +import apache_beam as beam from apache_beam.coders import coder_impl from apache_beam.coders import WindowedValueCoder from apache_beam.internal import pickler from apache_beam.io import iobase from apache_beam.portability.api import beam_fn_api_pb2 +from apache_beam.portability.api import beam_runner_api_pb2 from apache_beam.runners.dataflow.native_io import iobase as native_iobase from apache_beam.runners import pipeline_context from apache_beam.runners.worker import operation_specs from apache_beam.runners.worker import operations from apache_beam.utils import counters from apache_beam.utils import proto_utils +from apache_beam.utils import urns # This module is experimental. No backwards-compatibility guarantees. @@ -374,6 +377,24 @@ def create(factory, transform_id, transform_proto, parameter, consumers): consumers) +@BeamTransformFactory.register_urn( + urns.READ_TRANSFORM, beam_runner_api_pb2.ReadPayload) +def create(factory, transform_id, transform_proto, parameter, consumers): + # The Dataflow runner harness strips the base64 encoding. + source = iobase.SourceBase.from_runner_api(parameter.source, factory.context) + spec = operation_specs.WorkerRead( + iobase.SourceBundle(1.0, source, None, None), + [WindowedValueCoder(source.default_output_coder())]) + return factory.augment_oldstyle_op( + operations.ReadOperation( + transform_proto.unique_name, + spec, + factory.counter_factory, + factory.state_sampler), + transform_proto.unique_name, + consumers) + + @BeamTransformFactory.register_urn(PYTHON_DOFN_URN, wrappers_pb2.BytesValue) def create(factory, transform_id, transform_proto, parameter, consumers): dofn_data = pickler.loads(parameter.value) @@ -383,7 +404,32 @@ def create(factory, transform_id, transform_proto, parameter, consumers): else: # No side input data. serialized_fn, side_input_data = parameter.value, [] + return _create_pardo_operation( + factory, transform_id, transform_proto, consumers, + serialized_fn, side_input_data) + + +@BeamTransformFactory.register_urn( + urns.PARDO_TRANSFORM, beam_runner_api_pb2.ParDoPayload) +def create(factory, transform_id, transform_proto, parameter, consumers): + assert parameter.do_fn.spec.urn == urns.PICKLED_DO_FN_INFO + serialized_fn = proto_utils.unpack_Any( + parameter.do_fn.spec.parameter, wrappers_pb2.BytesValue).value + dofn_data = pickler.loads(serialized_fn) + if len(dofn_data) == 2: + # Has side input data. + serialized_fn, side_input_data = dofn_data + else: + # No side input data. + side_input_data = [] + return _create_pardo_operation( + factory, transform_id, transform_proto, consumers, + serialized_fn, side_input_data) + +def _create_pardo_operation( + factory, transform_id, transform_proto, consumers, + serialized_fn, side_input_data): def create_side_input(tag, coder): # TODO(robertwb): Extract windows (and keys) out of element data. # TODO(robertwb): Extract state key from ParDoPayload. @@ -395,10 +441,27 @@ def create_side_input(tag, coder): key=side_input_tag(transform_id, tag)), coder=coder)) output_tags = list(transform_proto.outputs.keys()) + + # Hack to match out prefix injected by dataflow runner. + def mutate_tag(tag): + if 'None' in output_tags: + if tag == 'None': + return 'out' + else: + return 'out_' + tag + else: + return tag + dofn_data = pickler.loads(serialized_fn) + if not dofn_data[-1]: + # Windowing not set. + pcoll_id, = transform_proto.inputs.values() + windowing = factory.context.windowing_strategies.get_by_id( + factory.descriptor.pcollections[pcoll_id].windowing_strategy_id) + serialized_fn = pickler.dumps(dofn_data[:-1] + (windowing,)) output_coders = factory.get_output_coders(transform_proto) spec = operation_specs.WorkerDoFn( serialized_fn=serialized_fn, - output_tags=output_tags, + output_tags=[mutate_tag(tag) for tag in output_tags], input=None, side_inputs=[ create_side_input(tag, coder) for tag, coder in side_input_data], @@ -414,12 +477,52 @@ def create_side_input(tag, coder): output_tags) +def _create_simple_pardo_operation( + factory, transform_id, transform_proto, consumers, dofn): + serialized_fn = pickler.dumps((dofn, (), {}, [], None)) + side_input_data = [] + return _create_pardo_operation( + factory, transform_id, transform_proto, consumers, + serialized_fn, side_input_data) + + +@BeamTransformFactory.register_urn( + urns.GROUP_ALSO_BY_WINDOW_TRANSFORM, wrappers_pb2.BytesValue) +def create(factory, transform_id, transform_proto, parameter, consumers): + # Perhaps this hack can go away once all apply overloads are gone. + from apache_beam.transforms.core import _GroupAlsoByWindowDoFn + return _create_simple_pardo_operation( + factory, transform_id, transform_proto, consumers, + _GroupAlsoByWindowDoFn( + factory.context.windowing_strategies.get_by_id(parameter.value))) + + +@BeamTransformFactory.register_urn( + urns.WINDOW_INTO_TRANSFORM, beam_runner_api_pb2.WindowingStrategy) +def create(factory, transform_id, transform_proto, parameter, consumers): + class WindowIntoDoFn(beam.DoFn): + def __init__(self, windowing): + self.windowing = windowing + + def process(self, element, timestamp=beam.DoFn.TimestampParam): + new_windows = self.windowing.windowfn.assign( + WindowFn.AssignContext(timestamp, element=element)) + yield WindowedValue(element, timestamp, new_windows) + from apache_beam.transforms.core import Windowing + from apache_beam.transforms.window import WindowFn, WindowedValue + windowing = Windowing.from_runner_api(parameter, factory.context) + return _create_simple_pardo_operation( + factory, transform_id, transform_proto, consumers, + WindowIntoDoFn(windowing)) + + @BeamTransformFactory.register_urn(IDENTITY_DOFN_URN, None) def create(factory, transform_id, transform_proto, unused_parameter, consumers): return factory.augment_oldstyle_op( operations.FlattenOperation( transform_proto.unique_name, - None, + operation_specs.WorkerFlatten( + None, [factory.get_only_output_coder(transform_proto)]), factory.counter_factory, factory.state_sampler), transform_proto.unique_name, diff --git a/sdks/python/apache_beam/transforms/core.py b/sdks/python/apache_beam/transforms/core.py index cff6dbec86014..3f92ce95a3bad 100644 --- a/sdks/python/apache_beam/transforms/core.py +++ b/sdks/python/apache_beam/transforms/core.py @@ -1461,6 +1461,7 @@ def from_runner_api_parameter(proto, context): # (Right now only WindowFn is used, but we need this to reconstitute the # WindowInto transform, and in the future will need it at runtime to # support meta-data driven triggers.) + # TODO(robertwb): Use a reference rather than embedding? beam_runner_api_pb2.WindowingStrategy, WindowInto.from_runner_api_parameter) @@ -1500,7 +1501,10 @@ def _extract_input_pvalues(self, pvalueish): def expand(self, pcolls): for pcoll in pcolls: self._check_pcollection(pcoll) - return pvalue.PCollection(self.pipeline) + result = pvalue.PCollection(self.pipeline) + result.element_type = typehints.Union[ + tuple(pcoll.element_type for pcoll in pcolls)] + return result def get_windowing(self, inputs): if not inputs: From fc00ce90060ccd99b69ff7225e3e35d259d3c7ad Mon Sep 17 00:00:00 2001 From: Mark Liu Date: Tue, 1 Aug 2017 17:46:05 -0700 Subject: [PATCH 242/346] [BEAM-1584] Add clean up in bigquery integration test --- .../cookbook/bigquery_tornadoes_it_test.py | 14 +++- sdks/python/apache_beam/io/gcp/tests/utils.py | 63 +++++++++++++++++ .../apache_beam/io/gcp/tests/utils_test.py | 70 +++++++++++++++++++ 3 files changed, 144 insertions(+), 3 deletions(-) create mode 100644 sdks/python/apache_beam/io/gcp/tests/utils.py create mode 100644 sdks/python/apache_beam/io/gcp/tests/utils_test.py diff --git a/sdks/python/apache_beam/examples/cookbook/bigquery_tornadoes_it_test.py b/sdks/python/apache_beam/examples/cookbook/bigquery_tornadoes_it_test.py index 5d2ee7c2cef8a..05ee3c53f48f4 100644 --- a/sdks/python/apache_beam/examples/cookbook/bigquery_tornadoes_it_test.py +++ b/sdks/python/apache_beam/examples/cookbook/bigquery_tornadoes_it_test.py @@ -26,6 +26,7 @@ from apache_beam.examples.cookbook import bigquery_tornadoes from apache_beam.io.gcp.tests.bigquery_matcher import BigqueryMatcher +from apache_beam.io.gcp.tests import utils from apache_beam.testing.pipeline_verifiers import PipelineStateMatcher from apache_beam.testing.test_pipeline import TestPipeline @@ -44,17 +45,24 @@ def test_bigquery_tornadoes_it(self): test_pipeline = TestPipeline(is_integration_test=True) # Set extra options to the pipeline for test purpose - output_table = ('BigQueryTornadoesIT' - '.monthly_tornadoes_%s' % int(round(time.time() * 1000))) + project = test_pipeline.get_option('project') + + dataset = 'BigQueryTornadoesIT' + table = 'monthly_tornadoes_%s' % int(round(time.time() * 1000)) + output_table = '.'.join([dataset, table]) query = 'SELECT month, tornado_count FROM [%s]' % output_table + pipeline_verifiers = [PipelineStateMatcher(), BigqueryMatcher( - project=test_pipeline.get_option('project'), + project=project, query=query, checksum=self.DEFAULT_CHECKSUM)] extra_opts = {'output': output_table, 'on_success_matcher': all_of(*pipeline_verifiers)} + # Register cleanup before pipeline execution. + self.addCleanup(utils.delete_bq_table, project, dataset, table) + # Get pipeline options from command argument: --test-pipeline-options, # and start pipeline job by calling pipeline main function. bigquery_tornadoes.run( diff --git a/sdks/python/apache_beam/io/gcp/tests/utils.py b/sdks/python/apache_beam/io/gcp/tests/utils.py new file mode 100644 index 0000000000000..40eb975ca7eb0 --- /dev/null +++ b/sdks/python/apache_beam/io/gcp/tests/utils.py @@ -0,0 +1,63 @@ +# +# 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. +# + + +"""Utility methods for testing on GCP.""" + +import logging + +from apache_beam.utils import retry + +# Protect against environments where bigquery library is not available. +try: + from google.cloud import bigquery +except ImportError: + bigquery = None + + +class GcpTestIOError(retry.PermanentException): + """Basic GCP IO error for testing. Function that raises this error should + not be retried.""" + pass + + +@retry.with_exponential_backoff( + num_retries=3, + retry_filter=retry.retry_on_server_errors_filter) +def delete_bq_table(project, dataset, table): + """Delete a Biqquery table. + + Args: + project: Name of the project. + dataset: Name of the dataset where table is. + table: Name of the table. + """ + logging.info('Clean up a Bigquery table with project: %s, dataset: %s, ' + 'table: %s.', project, dataset, table) + bq_dataset = bigquery.Client(project=project).dataset(dataset) + if not bq_dataset.exists(): + raise GcpTestIOError('Failed to cleanup. Bigquery dataset %s doesn\'t' + 'exist in project %s.' % dataset, project) + bq_table = bq_dataset.table(table) + if not bq_table.exists(): + raise GcpTestIOError('Failed to cleanup. Biqeury table %s doesn\'t ' + 'exist in project %s, dataset %s.' % + table, project, dataset) + bq_table.delete() + if bq_table.exists(): + raise RuntimeError('Failed to cleanup. Bigquery table %s still exists ' + 'after cleanup.' % table) diff --git a/sdks/python/apache_beam/io/gcp/tests/utils_test.py b/sdks/python/apache_beam/io/gcp/tests/utils_test.py new file mode 100644 index 0000000000000..270750a252176 --- /dev/null +++ b/sdks/python/apache_beam/io/gcp/tests/utils_test.py @@ -0,0 +1,70 @@ +# +# 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. +# + +"""Unittest for GCP testing utils.""" + +import logging +import unittest +from mock import Mock, patch + +from apache_beam.io.gcp.tests import utils +from apache_beam.testing.test_utils import patch_retry + +# Protect against environments where bigquery library is not available. +try: + from google.cloud import bigquery +except ImportError: + bigquery = None + + +@unittest.skipIf(bigquery is None, 'Bigquery dependencies are not installed.') +class UtilsTest(unittest.TestCase): + + def setUp(self): + self._mock_result = Mock() + patch_retry(self, utils) + + @patch('google.cloud.bigquery.Table.delete') + @patch('google.cloud.bigquery.Table.exists', side_effect=[True, False]) + @patch('google.cloud.bigquery.Dataset.exists', return_value=True) + def test_delete_bq_table_succeeds(self, *_): + utils.delete_bq_table('unused_project', + 'unused_dataset', + 'unused_table') + + @patch('google.cloud.bigquery.Table.delete', side_effect=Exception) + @patch('google.cloud.bigquery.Table.exists', return_value=True) + @patch('google.cloud.bigquery.Dataset.exists', return_vaue=True) + def test_delete_bq_table_fails_with_server_error(self, *_): + with self.assertRaises(Exception): + utils.delete_bq_table('unused_project', + 'unused_dataset', + 'unused_table') + + @patch('google.cloud.bigquery.Table.delete') + @patch('google.cloud.bigquery.Table.exists', return_value=[True, True]) + @patch('google.cloud.bigquery.Dataset.exists', return_vaue=True) + def test_delete_bq_table_fails_with_delete_error(self, *_): + with self.assertRaises(RuntimeError): + utils.delete_bq_table('unused_project', + 'unused_dataset', + 'unused_table') + + +if __name__ == '__main__': + logging.getLogger().setLevel(logging.INFO) + unittest.main() From b10c1c34e03e7e77add8a4cc7a98c60914becda6 Mon Sep 17 00:00:00 2001 From: Robert Bradshaw Date: Fri, 4 Aug 2017 11:43:12 -0700 Subject: [PATCH 243/346] [BEAM-2729] Allow GBK of union-typed PCollections. --- .../runners/dataflow/dataflow_runner.py | 25 ++++++++++++++----- .../runners/dataflow/dataflow_runner_test.py | 22 ++++++++++++++++ 2 files changed, 41 insertions(+), 6 deletions(-) diff --git a/sdks/python/apache_beam/runners/dataflow/dataflow_runner.py b/sdks/python/apache_beam/runners/dataflow/dataflow_runner.py index 0df18825536ee..880901e671472 100644 --- a/sdks/python/apache_beam/runners/dataflow/dataflow_runner.py +++ b/sdks/python/apache_beam/runners/dataflow/dataflow_runner.py @@ -188,18 +188,31 @@ def visit_transform(self, transform_node): if not input_type: input_type = typehints.Any - if not isinstance(input_type, typehints.TupleHint.TupleConstraint): - if isinstance(input_type, typehints.AnyTypeConstraint): + def coerce_to_kv_type(element_type): + if isinstance(element_type, typehints.TupleHint.TupleConstraint): + if len(element_type.tuple_types) == 2: + return element_type + else: + raise ValueError( + "Tuple input to GroupByKey must be have two components. " + "Found %s for %s" % (element_type, pcoll)) + elif isinstance(input_type, typehints.AnyTypeConstraint): # `Any` type needs to be replaced with a KV[Any, Any] to # force a KV coder as the main output coder for the pcollection # preceding a GroupByKey. - pcoll.element_type = typehints.KV[typehints.Any, typehints.Any] + return typehints.KV[typehints.Any, typehints.Any] + elif isinstance(element_type, typehints.UnionConstraint): + union_types = [ + coerce_to_kv_type(t) for t in element_type.union_types] + return typehints.KV[ + typehints.Union[tuple(t.tuple_types[0] for t in union_types)], + typehints.Union[tuple(t.tuple_types[1] for t in union_types)]] else: - # TODO: Handle other valid types, - # e.g. Union[KV[str, int], KV[str, float]] + # TODO: Possibly handle other valid types. raise ValueError( "Input to GroupByKey must be of Tuple or Any type. " - "Found %s for %s" % (input_type, pcoll)) + "Found %s for %s" % (element_type, pcoll)) + pcoll.element_type = coerce_to_kv_type(input_type) return GroupByKeyInputVisitor() diff --git a/sdks/python/apache_beam/runners/dataflow/dataflow_runner_test.py b/sdks/python/apache_beam/runners/dataflow/dataflow_runner_test.py index a9b8fdb2a24c3..80414d6883daf 100644 --- a/sdks/python/apache_beam/runners/dataflow/dataflow_runner_test.py +++ b/sdks/python/apache_beam/runners/dataflow/dataflow_runner_test.py @@ -256,6 +256,28 @@ def _test_flatten_input_visitor(self, input_type, output_type, num_inputs): for _ in range(num_inputs): self.assertEqual(inputs[0].element_type, output_type) + def test_gbk_then_flatten_input_visitor(self): + p = TestPipeline( + runner=DataflowRunner(), + options=PipelineOptions(self.default_properties)) + none_str_pc = p | 'c1' >> beam.Create({None: 'a'}) + none_int_pc = p | 'c2' >> beam.Create({None: 3}) + flat = (none_str_pc, none_int_pc) | beam.Flatten() + _ = flat | beam.GroupByKey() + + # This may change if type inference changes, but we assert it here + # to make sure the check below is not vacuous. + self.assertNotIsInstance(flat.element_type, typehints.TupleConstraint) + + p.visit(DataflowRunner.group_by_key_input_visitor()) + p.visit(DataflowRunner.flatten_input_visitor()) + + # The dataflow runner requires gbk input to be tuples *and* flatten + # inputs to be equal to their outputs. Assert both hold. + self.assertIsInstance(flat.element_type, typehints.TupleConstraint) + self.assertEqual(flat.element_type, none_str_pc.element_type) + self.assertEqual(flat.element_type, none_int_pc.element_type) + def test_serialize_windowing_strategy(self): # This just tests the basic path; more complete tests # are in window_test.py. From 5e43b2388652f38a37ab3378a63ae88e6ad53ee3 Mon Sep 17 00:00:00 2001 From: Eugene Kirpichov Date: Thu, 3 Aug 2017 14:42:07 -0700 Subject: [PATCH 244/346] Adds coders for boolean, ResourceId and Metadata --- .../apache/beam/sdk/coders/BooleanCoder.java | 59 +++++++++++++++++ .../apache/beam/sdk/coders/CoderRegistry.java | 10 +++ .../apache/beam/sdk/io/fs/MetadataCoder.java | 63 +++++++++++++++++++ .../beam/sdk/io/fs/ResourceIdCoder.java | 56 +++++++++++++++++ .../org/apache/beam/sdk/transforms/Watch.java | 7 ++- 5 files changed, 192 insertions(+), 3 deletions(-) create mode 100644 sdks/java/core/src/main/java/org/apache/beam/sdk/coders/BooleanCoder.java create mode 100644 sdks/java/core/src/main/java/org/apache/beam/sdk/io/fs/MetadataCoder.java create mode 100644 sdks/java/core/src/main/java/org/apache/beam/sdk/io/fs/ResourceIdCoder.java diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/BooleanCoder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/BooleanCoder.java new file mode 100644 index 0000000000000..e7f7543ee4b52 --- /dev/null +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/BooleanCoder.java @@ -0,0 +1,59 @@ +/* + * 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.coders; + +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; + +/** A {@link Coder} for {@link Boolean}. */ +public class BooleanCoder extends AtomicCoder { + private static final ByteCoder BYTE_CODER = ByteCoder.of(); + + private static final BooleanCoder INSTANCE = new BooleanCoder(); + + /** Returns the singleton instance of {@link BooleanCoder}. */ + public static BooleanCoder of() { + return INSTANCE; + } + + @Override + public void encode(Boolean value, OutputStream os) throws IOException { + BYTE_CODER.encode(value ? (byte) 1 : 0, os); + } + + @Override + public Boolean decode(InputStream is) throws IOException { + return BYTE_CODER.decode(is) == 1; + } + + @Override + public boolean consistentWithEquals() { + return true; + } + + @Override + public boolean isRegisterByteSizeObserverCheap(Boolean value) { + return true; + } + + @Override + protected long getEncodedElementByteSize(Boolean value) throws Exception { + return 1; + } +} 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 48389b1abbfbf..c335bda7e0fe8 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 @@ -43,6 +43,10 @@ import javax.annotation.Nullable; import org.apache.beam.sdk.annotations.Internal; import org.apache.beam.sdk.coders.CannotProvideCoderException.ReasonCode; +import org.apache.beam.sdk.io.fs.MatchResult.Metadata; +import org.apache.beam.sdk.io.fs.MetadataCoder; +import org.apache.beam.sdk.io.fs.ResourceId; +import org.apache.beam.sdk.io.fs.ResourceIdCoder; import org.apache.beam.sdk.transforms.SerializableFunction; import org.apache.beam.sdk.transforms.windowing.IntervalWindow; import org.apache.beam.sdk.util.CoderUtils; @@ -89,6 +93,8 @@ private static class CommonTypes extends CoderProvider { private CommonTypes() { ImmutableMap.Builder, CoderProvider> builder = ImmutableMap.builder(); + builder.put(Boolean.class, + CoderProviders.fromStaticMethods(Boolean.class, BooleanCoder.class)); builder.put(Byte.class, CoderProviders.fromStaticMethods(Byte.class, ByteCoder.class)); builder.put(BitSet.class, @@ -109,6 +115,10 @@ private CommonTypes() { CoderProviders.fromStaticMethods(Long.class, VarLongCoder.class)); builder.put(Map.class, CoderProviders.fromStaticMethods(Map.class, MapCoder.class)); + builder.put(Metadata.class, + CoderProviders.fromStaticMethods(Metadata.class, MetadataCoder.class)); + builder.put(ResourceId.class, + CoderProviders.fromStaticMethods(ResourceId.class, ResourceIdCoder.class)); builder.put(Set.class, CoderProviders.fromStaticMethods(Set.class, SetCoder.class)); builder.put(String.class, diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/fs/MetadataCoder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/fs/MetadataCoder.java new file mode 100644 index 0000000000000..5c9c4d7637231 --- /dev/null +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/fs/MetadataCoder.java @@ -0,0 +1,63 @@ +/* + * 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.fs; + +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import org.apache.beam.sdk.coders.AtomicCoder; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.coders.VarIntCoder; +import org.apache.beam.sdk.coders.VarLongCoder; +import org.apache.beam.sdk.io.fs.MatchResult.Metadata; + +/** A {@link Coder} for {@link Metadata}. */ +public class MetadataCoder extends AtomicCoder { + private static final ResourceIdCoder RESOURCE_ID_CODER = ResourceIdCoder.of(); + private static final VarIntCoder INT_CODER = VarIntCoder.of(); + private static final VarLongCoder LONG_CODER = VarLongCoder.of(); + + /** Creates a {@link MetadataCoder}. */ + public static MetadataCoder of() { + return new MetadataCoder(); + } + + @Override + public void encode(Metadata value, OutputStream os) throws IOException { + RESOURCE_ID_CODER.encode(value.resourceId(), os); + INT_CODER.encode(value.isReadSeekEfficient() ? 1 : 0, os); + LONG_CODER.encode(value.sizeBytes(), os); + } + + @Override + public Metadata decode(InputStream is) throws IOException { + ResourceId resourceId = RESOURCE_ID_CODER.decode(is); + boolean isReadSeekEfficient = INT_CODER.decode(is) == 1; + long sizeBytes = LONG_CODER.decode(is); + return Metadata.builder() + .setResourceId(resourceId) + .setIsReadSeekEfficient(isReadSeekEfficient) + .setSizeBytes(sizeBytes) + .build(); + } + + @Override + public boolean consistentWithEquals() { + return true; + } +} diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/fs/ResourceIdCoder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/fs/ResourceIdCoder.java new file mode 100644 index 0000000000000..d7649c04512bc --- /dev/null +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/fs/ResourceIdCoder.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.sdk.io.fs; + +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import org.apache.beam.sdk.coders.AtomicCoder; +import org.apache.beam.sdk.coders.BooleanCoder; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.coders.StringUtf8Coder; +import org.apache.beam.sdk.io.FileSystems; + +/** A {@link Coder} for {@link ResourceId}. */ +public class ResourceIdCoder extends AtomicCoder { + private static final Coder STRING_CODER = StringUtf8Coder.of(); + private static final Coder BOOL_CODER = BooleanCoder.of(); + + /** Creates a {@link ResourceIdCoder}. */ + public static ResourceIdCoder of() { + return new ResourceIdCoder(); + } + + @Override + public void encode(ResourceId value, OutputStream os) throws IOException { + STRING_CODER.encode(value.toString(), os); + BOOL_CODER.encode(value.isDirectory(), os); + } + + @Override + public ResourceId decode(InputStream is) throws IOException { + String spec = STRING_CODER.decode(is); + boolean isDirectory = BOOL_CODER.decode(is); + return FileSystems.matchNewResource(spec, isDirectory); + } + + @Override + public boolean consistentWithEquals() { + return true; + } +} diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Watch.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Watch.java index b21eb62226609..fc6f18d6b5640 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Watch.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Watch.java @@ -47,6 +47,7 @@ import javax.annotation.Nullable; import org.apache.beam.sdk.annotations.Experimental; import org.apache.beam.sdk.coders.AtomicCoder; +import org.apache.beam.sdk.coders.BooleanCoder; import org.apache.beam.sdk.coders.CannotProvideCoderException; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.DurationCoder; @@ -958,7 +959,7 @@ public static GrowthStateCoder(outputCoder, terminationStateCoder); } - private static final Coder INT_CODER = VarIntCoder.of(); + private static final Coder BOOLEAN_CODER = BooleanCoder.of(); private static final Coder INSTANT_CODER = NullableCoder.of(InstantCoder.of()); private static final Coder HASH_CODE_CODER = HashCode128Coder.of(); @@ -980,7 +981,7 @@ public void encode(GrowthState value, OutputStream o throws IOException { completedCoder.encode(value.completed, os); pendingCoder.encode(value.pending, os); - INT_CODER.encode(value.isOutputComplete ? 1 : 0, os); + BOOLEAN_CODER.encode(value.isOutputComplete, os); terminationStateCoder.encode(value.terminationState, os); INSTANT_CODER.encode(value.pollWatermark, os); } @@ -989,7 +990,7 @@ public void encode(GrowthState value, OutputStream o public GrowthState decode(InputStream is) throws IOException { Map completed = completedCoder.decode(is); List> pending = pendingCoder.decode(is); - boolean isOutputComplete = (INT_CODER.decode(is) == 1); + boolean isOutputComplete = BOOLEAN_CODER.decode(is); TerminationStateT terminationState = terminationStateCoder.decode(is); Instant pollWatermark = INSTANT_CODER.decode(is); return new GrowthState<>( From db9aede289f8546bb30113353f07aa75daa83eba Mon Sep 17 00:00:00 2001 From: Eugene Kirpichov Date: Thu, 3 Aug 2017 14:43:48 -0700 Subject: [PATCH 245/346] Introduces EmptyMatchTreatment parameter to FileSystems.match() --- .../apache/beam/sdk/io/FileBasedSource.java | 52 ++++++++++--------- .../org/apache/beam/sdk/io/FileSystems.java | 46 ++++++++++++++++ .../org/apache/beam/sdk/io/TextSource.java | 7 ++- .../beam/sdk/io/fs/EmptyMatchTreatment.java | 46 ++++++++++++++++ .../apache/beam/sdk/io/fs/MatchResult.java | 5 +- .../beam/sdk/io/FileBasedSourceTest.java | 51 ++++++++++++++++++ 6 files changed, 180 insertions(+), 27 deletions(-) create mode 100644 sdks/java/core/src/main/java/org/apache/beam/sdk/io/fs/EmptyMatchTreatment.java 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 d4413c90765e8..7f865deb4639d 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 @@ -23,19 +23,17 @@ import static com.google.common.base.Verify.verify; import com.google.common.collect.ImmutableList; -import com.google.common.collect.Iterables; import java.io.IOException; import java.nio.channels.ReadableByteChannel; import java.nio.channels.SeekableByteChannel; import java.util.ArrayList; -import java.util.Collections; import java.util.List; import java.util.ListIterator; import java.util.NoSuchElementException; import javax.annotation.Nullable; +import org.apache.beam.sdk.io.fs.EmptyMatchTreatment; import org.apache.beam.sdk.io.fs.MatchResult; import org.apache.beam.sdk.io.fs.MatchResult.Metadata; -import org.apache.beam.sdk.io.fs.MatchResult.Status; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.options.ValueProvider; import org.apache.beam.sdk.options.ValueProvider.StaticValueProvider; @@ -68,6 +66,7 @@ public abstract class FileBasedSource extends OffsetBasedSource { private static final Logger LOG = LoggerFactory.getLogger(FileBasedSource.class); private final ValueProvider fileOrPatternSpec; + private final EmptyMatchTreatment emptyMatchTreatment; @Nullable private MatchResult.Metadata singleFileMetadata; private final Mode mode; @@ -80,14 +79,27 @@ public enum Mode { } /** - * Create a {@code FileBaseSource} based on a file or a file pattern specification. + * Create a {@code FileBaseSource} based on a file or a file pattern specification, with the given + * strategy for treating filepatterns that do not match any files. */ - protected FileBasedSource(ValueProvider fileOrPatternSpec, long minBundleSize) { + protected FileBasedSource( + ValueProvider fileOrPatternSpec, + EmptyMatchTreatment emptyMatchTreatment, + long minBundleSize) { super(0, Long.MAX_VALUE, minBundleSize); - mode = Mode.FILEPATTERN; + this.mode = Mode.FILEPATTERN; + this.emptyMatchTreatment = emptyMatchTreatment; this.fileOrPatternSpec = fileOrPatternSpec; } + /** + * Like {@link #FileBasedSource(ValueProvider, EmptyMatchTreatment, long)}, but uses the default + * value of {@link EmptyMatchTreatment#DISALLOW}. + */ + protected FileBasedSource(ValueProvider fileOrPatternSpec, long minBundleSize) { + this(fileOrPatternSpec, EmptyMatchTreatment.DISALLOW, minBundleSize); + } + /** * Create a {@code FileBasedSource} based on a single file. This constructor must be used when * creating a new {@code FileBasedSource} for a subrange of a single file. @@ -110,6 +122,9 @@ protected FileBasedSource( mode = Mode.SINGLE_FILE_OR_SUBRANGE; this.singleFileMetadata = checkNotNull(fileMetadata, "fileMetadata"); this.fileOrPatternSpec = StaticValueProvider.of(fileMetadata.resourceId().toString()); + + // This field will be unused in this mode. + this.emptyMatchTreatment = null; } /** @@ -204,14 +219,7 @@ public final long getEstimatedSizeBytes(PipelineOptions options) throws IOExcept if (mode == Mode.FILEPATTERN) { long totalSize = 0; - List inputs = FileSystems.match(Collections.singletonList(fileOrPattern)); - MatchResult result = Iterables.getOnlyElement(inputs); - checkArgument( - result.status() == Status.OK, - "Error matching the pattern or glob %s: status %s", - fileOrPattern, - result.status()); - List allMatches = result.metadata(); + List allMatches = FileSystems.match(fileOrPattern, emptyMatchTreatment).metadata(); for (Metadata metadata : allMatches) { totalSize += metadata.sizeBytes(); } @@ -254,9 +262,8 @@ public final List> split( if (mode == Mode.FILEPATTERN) { long startTime = System.currentTimeMillis(); - List expandedFiles = FileBasedSource.expandFilePattern(fileOrPattern); - checkArgument(!expandedFiles.isEmpty(), - "Unable to find any files matching %s", fileOrPattern); + List expandedFiles = + FileSystems.match(fileOrPattern, emptyMatchTreatment).metadata(); List> splitResults = new ArrayList<>(expandedFiles.size()); for (Metadata metadata : expandedFiles) { FileBasedSource split = createForSubrangeOfFile(metadata, 0, metadata.sizeBytes()); @@ -327,7 +334,9 @@ public final BoundedReader createReader(PipelineOptions options) throws IOExc if (mode == Mode.FILEPATTERN) { long startTime = System.currentTimeMillis(); - List fileMetadata = FileBasedSource.expandFilePattern(fileOrPattern); + List fileMetadata = + FileSystems.match(fileOrPattern, emptyMatchTreatment).metadata(); + LOG.info("Matched {} files for pattern {}", fileMetadata.size(), fileOrPattern); List> fileReaders = new ArrayList<>(); for (Metadata metadata : fileMetadata) { long endOffset = metadata.sizeBytes(); @@ -389,13 +398,6 @@ public final long getMaxEndOffset(PipelineOptions options) throws IOException { return metadata.sizeBytes(); } - private static List expandFilePattern(String fileOrPatternSpec) throws IOException { - MatchResult matches = - Iterables.getOnlyElement(FileSystems.match(Collections.singletonList(fileOrPatternSpec))); - LOG.info("Matched {} files for pattern {}", matches.metadata().size(), fileOrPatternSpec); - return ImmutableList.copyOf(matches.metadata()); - } - /** * A {@link Source.Reader reader} that implements code common to readers of * {@code FileBasedSource}s. 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 index bd4668f970d32..96394b83678a4 100644 --- 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 @@ -54,6 +54,7 @@ import org.apache.beam.sdk.annotations.Internal; import org.apache.beam.sdk.io.fs.CreateOptions; import org.apache.beam.sdk.io.fs.CreateOptions.StandardCreateOptions; +import org.apache.beam.sdk.io.fs.EmptyMatchTreatment; import org.apache.beam.sdk.io.fs.MatchResult; import org.apache.beam.sdk.io.fs.MatchResult.Metadata; import org.apache.beam.sdk.io.fs.MatchResult.Status; @@ -72,6 +73,8 @@ public class FileSystems { public static final String DEFAULT_SCHEME = "file"; private static final Pattern FILE_SCHEME_PATTERN = Pattern.compile("(?[a-zA-Z][-a-zA-Z0-9+.]*):.*"); + private static final Pattern GLOB_PATTERN = + Pattern.compile("[*?{}]"); private static final AtomicReference> SCHEME_TO_FILESYSTEM = new AtomicReference>( @@ -79,6 +82,11 @@ public class FileSystems { /********************************** METHODS FOR CLIENT **********************************/ + /** Checks whether the given spec contains a glob wildcard character. */ + public static boolean hasGlobWildcard(String spec) { + return GLOB_PATTERN.matcher(spec).find(); + } + /** * This is the entry point to convert user-provided specs to {@link ResourceId ResourceIds}. * Callers should use {@link #match} to resolve users specs ambiguities before @@ -102,6 +110,9 @@ public class FileSystems { *

              In case the spec schemes don't match any known {@link FileSystem} implementations, * FileSystems will attempt to use {@link LocalFileSystem} to resolve a path. * + *

              Specs that do not match any resources are treated according to + * {@link EmptyMatchTreatment#DISALLOW}. + * * @return {@code List} in the same order of the input specs. * * @throws IllegalArgumentException if specs are invalid -- empty or have different schemes. @@ -114,6 +125,17 @@ public static List match(List specs) throws IOException { return getFileSystemInternal(getOnlyScheme(specs)).match(specs); } + /** Like {@link #match(List)}, but with a configurable {@link EmptyMatchTreatment}. */ + public static List match(List specs, EmptyMatchTreatment emptyMatchTreatment) + throws IOException { + List matches = getFileSystemInternal(getOnlyScheme(specs)).match(specs); + List res = Lists.newArrayListWithExpectedSize(matches.size()); + for (int i = 0; i < matches.size(); i++) { + res.add(maybeAdjustEmptyMatchResult(specs.get(i), matches.get(i), emptyMatchTreatment)); + } + return res; + } + /** * Like {@link #match(List)}, but for a single resource specification. @@ -130,6 +152,30 @@ public static MatchResult match(String spec) throws IOException { matches); return matches.get(0); } + + /** Like {@link #match(String)}, but with a configurable {@link EmptyMatchTreatment}. */ + public static MatchResult match(String spec, EmptyMatchTreatment emptyMatchTreatment) + throws IOException { + MatchResult res = match(spec); + return maybeAdjustEmptyMatchResult(spec, res, emptyMatchTreatment); + } + + private static MatchResult maybeAdjustEmptyMatchResult( + String spec, MatchResult res, EmptyMatchTreatment emptyMatchTreatment) + throws IOException { + if (res.status() != Status.NOT_FOUND) { + return res; + } + boolean notFoundAllowed = + emptyMatchTreatment == EmptyMatchTreatment.ALLOW + || (FileSystems.hasGlobWildcard(spec) + && emptyMatchTreatment == EmptyMatchTreatment.ALLOW_IF_WILDCARD); + if (notFoundAllowed) { + return MatchResult.create(Status.OK, Collections.emptyList()); + } + return res; + } + /** * Returns the {@link Metadata} for a single file resource. Expects a resource specification * {@code spec} that matches a single result. diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/TextSource.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/TextSource.java index 86c73d5d32606..29188dc88638f 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/TextSource.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/TextSource.java @@ -28,6 +28,7 @@ import java.util.NoSuchElementException; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.StringUtf8Coder; +import org.apache.beam.sdk.io.fs.EmptyMatchTreatment; import org.apache.beam.sdk.io.fs.MatchResult; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.options.ValueProvider; @@ -48,7 +49,11 @@ @VisibleForTesting class TextSource extends FileBasedSource { TextSource(ValueProvider fileSpec) { - super(fileSpec, 1L); + this(fileSpec, EmptyMatchTreatment.DISALLOW); + } + + TextSource(ValueProvider fileSpec, EmptyMatchTreatment emptyMatchTreatment) { + super(fileSpec, emptyMatchTreatment, 1L); } private TextSource(MatchResult.Metadata metadata, long start, long end) { diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/fs/EmptyMatchTreatment.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/fs/EmptyMatchTreatment.java new file mode 100644 index 0000000000000..8e12993720311 --- /dev/null +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/fs/EmptyMatchTreatment.java @@ -0,0 +1,46 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.sdk.io.fs; + +import org.apache.beam.sdk.io.fs.MatchResult.Status; + +/** + * Options for allowing or disallowing filepatterns that match no resources in {@link + * org.apache.beam.sdk.io.FileSystems#match}. + */ +public enum EmptyMatchTreatment { + /** + * Filepatterns matching no resources are allowed. For such a filepattern, {@link + * MatchResult#status} will be {@link Status#OK} and {@link MatchResult#metadata} will return an + * empty list. + */ + ALLOW, + + /** + * Filepatterns matching no resources are disallowed. For such a filepattern, {@link + * MatchResult#status} will be {@link Status#NOT_FOUND} and {@link MatchResult#metadata} will + * throw a {@link java.io.FileNotFoundException}. + */ + DISALLOW, + + /** + * Filepatterns matching no resources are allowed if the filepattern contains a glob wildcard + * character, and disallowed otherwise (i.e. if the filepattern specifies a single file). + */ + ALLOW_IF_WILDCARD +} diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/fs/MatchResult.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/fs/MatchResult.java index 642c04972eb14..aa80b9605a051 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/fs/MatchResult.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/fs/MatchResult.java @@ -21,6 +21,7 @@ import java.io.IOException; import java.io.Serializable; import java.util.List; +import org.apache.beam.sdk.io.FileSystems; /** * The result of {@link org.apache.beam.sdk.io.FileSystem#match}. @@ -78,7 +79,9 @@ public static MatchResult unknown() { public abstract Status status(); /** - * {@link Metadata} of matched files. + * {@link Metadata} of matched files. Note that if {@link #status()} is {@link Status#NOT_FOUND}, + * this may either throw a {@link java.io.FileNotFoundException} or return an empty list, + * depending on the {@link EmptyMatchTreatment} used in the {@link FileSystems#match} call. */ public abstract List metadata() throws IOException; 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 1bdb915d9d00d..ea9e06bda6d28 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 @@ -47,6 +47,7 @@ import org.apache.beam.sdk.coders.StringUtf8Coder; import org.apache.beam.sdk.io.FileBasedSource.FileBasedReader; import org.apache.beam.sdk.io.Source.Reader; +import org.apache.beam.sdk.io.fs.EmptyMatchTreatment; import org.apache.beam.sdk.io.fs.MatchResult.Metadata; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.options.PipelineOptionsFactory; @@ -93,6 +94,15 @@ public TestFileBasedSource(String fileOrPattern, long minBundleSize, String spli this.splitHeader = splitHeader; } + public TestFileBasedSource( + String fileOrPattern, + EmptyMatchTreatment emptyMatchTreatment, + long minBundleSize, + String splitHeader) { + super(StaticValueProvider.of(fileOrPattern), emptyMatchTreatment, minBundleSize); + this.splitHeader = splitHeader; + } + public TestFileBasedSource( Metadata fileOrPattern, long minBundleSize, @@ -370,6 +380,47 @@ public void testFullyReadFilePattern() throws IOException { assertThat(expectedResults, containsInAnyOrder(readFromSource(source, options).toArray())); } + @Test + public void testEmptyFilepatternTreatmentDefaultDisallow() throws IOException { + PipelineOptions options = PipelineOptionsFactory.create(); + TestFileBasedSource source = + new TestFileBasedSource(new File(tempFolder.getRoot(), "doesNotExist").getPath(), 64, null); + thrown.expect(FileNotFoundException.class); + readFromSource(source, options); + } + + @Test + public void testEmptyFilepatternTreatmentAllow() throws IOException { + PipelineOptions options = PipelineOptionsFactory.create(); + TestFileBasedSource source = + new TestFileBasedSource( + new File(tempFolder.getRoot(), "doesNotExist").getPath(), + EmptyMatchTreatment.ALLOW, + 64, + null); + TestFileBasedSource sourceWithWildcard = + new TestFileBasedSource( + new File(tempFolder.getRoot(), "doesNotExist*").getPath(), + EmptyMatchTreatment.ALLOW_IF_WILDCARD, + 64, + null); + assertEquals(0, readFromSource(source, options).size()); + assertEquals(0, readFromSource(sourceWithWildcard, options).size()); + } + + @Test + public void testEmptyFilepatternTreatmentAllowIfWildcard() throws IOException { + PipelineOptions options = PipelineOptionsFactory.create(); + TestFileBasedSource source = + new TestFileBasedSource( + new File(tempFolder.getRoot(), "doesNotExist").getPath(), + EmptyMatchTreatment.ALLOW_IF_WILDCARD, + 64, + null); + thrown.expect(FileNotFoundException.class); + readFromSource(source, options); + } + @Test public void testCloseUnstartedFilePatternReader() throws IOException { PipelineOptions options = PipelineOptionsFactory.create(); From fe002c221602a543b99afd6db910a7a60b259fa4 Mon Sep 17 00:00:00 2001 From: Eugene Kirpichov Date: Thu, 3 Aug 2017 14:44:35 -0700 Subject: [PATCH 246/346] [BEAM-2512] Introduces TextIO.watchForNewFiles() and Match Part of http://s.apache.org/textio-sdf, based on http://s.apache.org/beam-watch-transform. The Match transform can be useful for users who want to write their own file-based connectors, or for advanced use cases such as: watch for new subdirectories to appear in a directory (using Match), and then start watching each subdirectory for new files and reading them (using TextIO.watchForNewFiles()). Additionally, finally makes it configurable whether TextIO.read/readAll() allow filepatterns matching no files. Normal reads disallow empty filepatterns (to preserve old behavior), readAll() allows them if the filepattern contains a wildcard (which seems a reasonable default behavior that read() should have had from the beginning, but we can't change it), and watchForNewFiles() allows them unconditionally (because files might appear later). --- .../beam/sdk/annotations/Experimental.java | 5 +- .../java/org/apache/beam/sdk/io/AvroIO.java | 2 + .../java/org/apache/beam/sdk/io/Match.java | 156 ++++++++++++++++++ .../sdk/io/ReadAllViaFileBasedSource.java | 46 +++--- .../java/org/apache/beam/sdk/io/TextIO.java | 156 +++++++++++++++--- .../org/apache/beam/sdk/transforms/DoFn.java | 11 +- .../org/apache/beam/sdk/transforms/Watch.java | 16 +- .../apache/beam/sdk/io/TextIOReadTest.java | 54 +++++- 8 files changed, 384 insertions(+), 62 deletions(-) create mode 100644 sdks/java/core/src/main/java/org/apache/beam/sdk/io/Match.java 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 8224ebbe75d3c..80c4613c5b0fa 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 @@ -72,8 +72,9 @@ enum Kind { OUTPUT_TIME, /** - * Splittable DoFn. - * Do not use: API is unstable and runner support is incomplete. + * Splittable DoFn. See capability matrix + * for runner support. */ SPLITTABLE_DO_FN, 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 cd5857c902589..653b80699dd81 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 @@ -478,6 +478,7 @@ ReadAll withDesiredBundleSizeBytes(long desiredBundleSizeBytes) { public PCollection expand(PCollection input) { checkNotNull(getSchema(), "schema"); return input + .apply(Match.filepatterns()) .apply( "Read all via FileBasedSource", new ReadAllViaFileBasedSource<>( @@ -632,6 +633,7 @@ public FileBasedSource apply(String input) { } }; return input + .apply(Match.filepatterns()) .apply( "Parse all via FileBasedSource", new ReadAllViaFileBasedSource<>( diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/Match.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/Match.java new file mode 100644 index 0000000000000..bb44face1eba4 --- /dev/null +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/Match.java @@ -0,0 +1,156 @@ +/* + * 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.value.AutoValue; +import javax.annotation.Nullable; +import org.apache.beam.sdk.annotations.Experimental; +import org.apache.beam.sdk.io.fs.EmptyMatchTreatment; +import org.apache.beam.sdk.io.fs.MatchResult; +import org.apache.beam.sdk.io.fs.MatchResult.Metadata; +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.Values; +import org.apache.beam.sdk.transforms.Watch; +import org.apache.beam.sdk.transforms.Watch.Growth.PollResult; +import org.apache.beam.sdk.transforms.Watch.Growth.TerminationCondition; +import org.apache.beam.sdk.values.PCollection; +import org.joda.time.Duration; +import org.joda.time.Instant; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Matches each filepattern in a collection of filepatterns using {@link FileSystems#match}, and + * produces a collection of matched resources (both files and directories) as {@link Metadata}. + * Resources are not deduplicated between filepatterns, i.e. if the same resource matches multiple + * filepatterns, it will be produced multiple times. + * + *

              By default, this transform matches each filepattern once and produces a bounded {@link + * PCollection}. To continuously watch each filepattern for new matches, use {@link + * Filepatterns#continuously(Duration, TerminationCondition)} - this will produce an unbounded + * {@link PCollection}. + * + *

              By default, filepatterns matching no resources are treated according to {@link + * EmptyMatchTreatment#ALLOW_IF_WILDCARD}. To configure this behavior, use {@link + * Filepatterns#withEmptyMatchTreatment}. + */ +public class Match { + private static final Logger LOG = LoggerFactory.getLogger(Match.class); + + /** See {@link Match}. */ + public static Filepatterns filepatterns() { + return new AutoValue_Match_Filepatterns.Builder() + .setEmptyMatchTreatment(EmptyMatchTreatment.ALLOW_IF_WILDCARD) + .build(); + } + + /** Implementation of {@link #filepatterns}. */ + @AutoValue + public abstract static class Filepatterns + extends PTransform, PCollection> { + abstract EmptyMatchTreatment getEmptyMatchTreatment(); + + @Nullable + abstract Duration getWatchInterval(); + + @Nullable + abstract TerminationCondition getWatchTerminationCondition(); + + abstract Builder toBuilder(); + + @AutoValue.Builder + abstract static class Builder { + abstract Builder setEmptyMatchTreatment(EmptyMatchTreatment treatment); + + abstract Builder setWatchInterval(Duration watchInterval); + + abstract Builder setWatchTerminationCondition(TerminationCondition condition); + + abstract Filepatterns build(); + } + + /** + * Sets whether or not filepatterns matching no files are allowed. When using {@link + * #continuously}, they are always allowed, and this parameter is ignored. + */ + public Filepatterns withEmptyMatchTreatment(EmptyMatchTreatment treatment) { + return toBuilder().setEmptyMatchTreatment(treatment).build(); + } + + /** + * Continuously watches for new resources matching the filepattern, repeatedly matching it at + * the given interval, until the given termination condition is reached. The returned {@link + * PCollection} is unbounded. + * + *

              This works only in runners supporting {@link Experimental.Kind#SPLITTABLE_DO_FN}. + * + * @see TerminationCondition + */ + @Experimental(Experimental.Kind.SPLITTABLE_DO_FN) + public Filepatterns continuously( + Duration pollInterval, TerminationCondition terminationCondition) { + return toBuilder() + .setWatchInterval(pollInterval) + .setWatchTerminationCondition(terminationCondition) + .build(); + } + + @Override + public PCollection expand(PCollection input) { + if (getWatchInterval() == null) { + return input.apply("Match filepatterns", ParDo.of(new MatchFn(getEmptyMatchTreatment()))); + } else { + return input + .apply( + "Continuously match filepatterns", + Watch.growthOf(new MatchPollFn()) + .withPollInterval(getWatchInterval()) + .withTerminationPerInput(getWatchTerminationCondition())) + .apply(Values.create()); + } + } + + private static class MatchFn extends DoFn { + private final EmptyMatchTreatment emptyMatchTreatment; + + public MatchFn(EmptyMatchTreatment emptyMatchTreatment) { + this.emptyMatchTreatment = emptyMatchTreatment; + } + + @ProcessElement + public void process(ProcessContext c) throws Exception { + String filepattern = c.element(); + MatchResult match = FileSystems.match(filepattern, emptyMatchTreatment); + LOG.info("Matched {} files for pattern {}", match.metadata().size(), filepattern); + for (Metadata metadata : match.metadata()) { + c.output(metadata); + } + } + } + + private static class MatchPollFn implements Watch.Growth.PollFn { + @Override + public PollResult apply(String input, Instant timestamp) throws Exception { + return PollResult.incomplete( + Instant.now(), FileSystems.match(input, EmptyMatchTreatment.ALLOW).metadata()); + } + } + } +} diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/ReadAllViaFileBasedSource.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/ReadAllViaFileBasedSource.java index 66aa41e316da6..990f508d676cb 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/ReadAllViaFileBasedSource.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/ReadAllViaFileBasedSource.java @@ -21,7 +21,8 @@ import java.io.IOException; import java.util.concurrent.ThreadLocalRandom; -import org.apache.beam.sdk.io.fs.MatchResult; +import org.apache.beam.sdk.io.fs.MatchResult.Metadata; +import org.apache.beam.sdk.io.fs.ResourceId; import org.apache.beam.sdk.io.range.OffsetRange; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.PTransform; @@ -33,10 +34,14 @@ import org.apache.beam.sdk.values.PCollection; /** - * Reads each filepattern in the input {@link PCollection} using given parameters for splitting - * files into offset ranges and for creating a {@link FileBasedSource} for a file. + * Reads each file in the input {@link PCollection} of {@link Metadata} using given parameters for + * splitting files into offset ranges and for creating a {@link FileBasedSource} for a file. The + * input {@link PCollection} must not contain {@link ResourceId#isDirectory directories}. + * + *

              To obtain the collection of {@link Metadata} from a filepattern, use {@link + * Match#filepatterns()}. */ -class ReadAllViaFileBasedSource extends PTransform, PCollection> { +class ReadAllViaFileBasedSource extends PTransform, PCollection> { private final SerializableFunction isSplittable; private final long desiredBundleSizeBytes; private final SerializableFunction> createSource; @@ -51,13 +56,12 @@ public ReadAllViaFileBasedSource( } @Override - public PCollection expand(PCollection input) { + public PCollection expand(PCollection input) { return input - .apply("Expand glob", ParDo.of(new ExpandGlobFn())) .apply( "Split into ranges", ParDo.of(new SplitIntoRangesFn(isSplittable, desiredBundleSizeBytes))) - .apply("Reshuffle", new ReshuffleWithUniqueKey>()) + .apply("Reshuffle", new ReshuffleWithUniqueKey>()) .apply("Read ranges", ParDo.of(new ReadFileRangesFn(createSource))); } @@ -86,23 +90,7 @@ public void process(ProcessContext c) { } } - private static class ExpandGlobFn extends DoFn { - @ProcessElement - public void process(ProcessContext c) throws Exception { - MatchResult match = FileSystems.match(c.element()); - checkArgument( - match.status().equals(MatchResult.Status.OK), - "Failed to match filepattern %s: %s", - c.element(), - match.status()); - for (MatchResult.Metadata metadata : match.metadata()) { - c.output(metadata); - } - } - } - - private static class SplitIntoRangesFn - extends DoFn> { + private static class SplitIntoRangesFn extends DoFn> { private final SerializableFunction isSplittable; private final long desiredBundleSizeBytes; @@ -114,7 +102,11 @@ private SplitIntoRangesFn( @ProcessElement public void process(ProcessContext c) { - MatchResult.Metadata metadata = c.element(); + Metadata metadata = c.element(); + checkArgument( + !metadata.resourceId().isDirectory(), + "Resource %s is a directory", + metadata.resourceId()); if (!metadata.isReadSeekEfficient() || !isSplittable.apply(metadata.resourceId().toString())) { c.output(KV.of(metadata, new OffsetRange(0, metadata.sizeBytes()))); @@ -127,7 +119,7 @@ public void process(ProcessContext c) { } } - private static class ReadFileRangesFn extends DoFn, T> { + private static class ReadFileRangesFn extends DoFn, T> { private final SerializableFunction> createSource; private ReadFileRangesFn(SerializableFunction> createSource) { @@ -136,7 +128,7 @@ private ReadFileRangesFn(SerializableFunction> create @ProcessElement public void process(ProcessContext c) throws IOException { - MatchResult.Metadata metadata = c.element().getKey(); + Metadata metadata = c.element().getKey(); OffsetRange range = c.element().getValue(); FileBasedSource source = createSource.apply(metadata.toString()); try (BoundedSource.BoundedReader reader = 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 9a14ad9d12b95..612f5c5917083 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 @@ -36,6 +36,7 @@ import org.apache.beam.sdk.io.FileBasedSink.DynamicDestinations; import org.apache.beam.sdk.io.FileBasedSink.FilenamePolicy; import org.apache.beam.sdk.io.FileBasedSink.WritableByteChannelFactory; +import org.apache.beam.sdk.io.fs.EmptyMatchTreatment; import org.apache.beam.sdk.io.fs.ResourceId; import org.apache.beam.sdk.options.ValueProvider; import org.apache.beam.sdk.options.ValueProvider.NestedValueProvider; @@ -44,10 +45,12 @@ import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.SerializableFunction; import org.apache.beam.sdk.transforms.SerializableFunctions; +import org.apache.beam.sdk.transforms.Watch.Growth.TerminationCondition; import org.apache.beam.sdk.transforms.display.DisplayData; import org.apache.beam.sdk.values.PBegin; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PDone; +import org.joda.time.Duration; /** * {@link PTransform}s for reading and writing text files. @@ -57,9 +60,16 @@ * file(s) to be read. Alternatively, if the filenames to be read are themselves in a {@link * PCollection}, apply {@link TextIO#readAll()}. * - *

              {@link TextIO.Read} returns a {@link PCollection} of {@link String Strings}, each - * corresponding to one line of an input UTF-8 text file (split into lines delimited by '\n', '\r', - * or '\r\n'). + *

              {@link #read} returns a {@link PCollection} of {@link String Strings}, each corresponding to + * one line of an input UTF-8 text file (split into lines delimited by '\n', '\r', or '\r\n'). + * + *

              By default, the filepatterns are expanded only once. {@link Read#watchForNewFiles} and {@link + * ReadAll#watchForNewFiles} allow streaming of new files matching the filepattern(s). + * + *

              By default, {@link #read} prohibits filepatterns that match no files, and {@link #readAll} + * allows them in case the filepattern contains a glob wildcard character. Use {@link + * TextIO.Read#withEmptyMatchTreatment} and {@link TextIO.ReadAll#withEmptyMatchTreatment} to + * configure this behavior. * *

              Example 1: reading a file or filepattern. * @@ -88,6 +98,20 @@ * PCollection lines = filenames.apply(TextIO.readAll()); * } * + *

              Example 3: streaming new files matching a filepattern. + * + *

              {@code
              + * Pipeline p = ...;
              + *
              + * PCollection lines = p.apply(TextIO.read()
              + *     .from("/local/path/to/files/*")
              + *     .watchForNewFiles(
              + *       // Check for new files every minute
              + *       Duration.standardMinutes(1),
              + *       // Stop watching the filepattern if no new files appear within an hour
              + *       afterTimeSinceNewOutput(Duration.standardHours(1))));
              + * }
              + * *

              To write a {@link PCollection} to one or more text files, use {@code TextIO.write()}, using * {@link TextIO.Write#to(String)} to specify the output prefix of the files to write. * @@ -153,6 +177,7 @@ public static Read read() { return new AutoValue_TextIO_Read.Builder() .setCompressionType(CompressionType.AUTO) .setHintMatchesManyFiles(false) + .setEmptyMatchTreatment(EmptyMatchTreatment.DISALLOW) .build(); } @@ -173,6 +198,7 @@ public static ReadAll readAll() { // but is not so large as to exhaust a typical runner's maximum amount of output per // ProcessElement call. .setDesiredBundleSizeBytes(64 * 1024 * 1024L) + .setEmptyMatchTreatment(EmptyMatchTreatment.ALLOW_IF_WILDCARD) .build(); } @@ -219,7 +245,15 @@ public static TypedWrite writeCustomType() { public abstract static class Read extends PTransform> { @Nullable abstract ValueProvider getFilepattern(); abstract CompressionType getCompressionType(); + + @Nullable + abstract Duration getWatchForNewFilesInterval(); + + @Nullable + abstract TerminationCondition getWatchForNewFilesTerminationCondition(); + abstract boolean getHintMatchesManyFiles(); + abstract EmptyMatchTreatment getEmptyMatchTreatment(); abstract Builder toBuilder(); @@ -227,7 +261,10 @@ public abstract static class Read extends PTransform abstract static class Builder { abstract Builder setFilepattern(ValueProvider filepattern); abstract Builder setCompressionType(CompressionType compressionType); + abstract Builder setWatchForNewFilesInterval(Duration watchForNewFilesInterval); + abstract Builder setWatchForNewFilesTerminationCondition(TerminationCondition condition); abstract Builder setHintMatchesManyFiles(boolean hintManyFiles); + abstract Builder setEmptyMatchTreatment(EmptyMatchTreatment treatment); abstract Read build(); } @@ -257,8 +294,7 @@ public Read from(ValueProvider filepattern) { } /** - * Returns a new transform for reading from text files that's like this one but - * reads from input sources using the specified compression type. + * Reads from input sources using the specified compression type. * *

              If no compression type is specified, the default is {@link TextIO.CompressionType#AUTO}. */ @@ -266,6 +302,23 @@ public Read withCompressionType(TextIO.CompressionType compressionType) { return toBuilder().setCompressionType(compressionType).build(); } + /** + * Continuously watches for new files matching the filepattern, polling it at the given + * interval, until the given termination condition is reached. The returned {@link PCollection} + * is unbounded. + * + *

              This works only in runners supporting {@link Kind#SPLITTABLE_DO_FN}. + * + * @see TerminationCondition + */ + @Experimental(Kind.SPLITTABLE_DO_FN) + public Read watchForNewFiles(Duration pollInterval, TerminationCondition terminationCondition) { + return toBuilder() + .setWatchForNewFilesInterval(pollInterval) + .setWatchForNewFilesTerminationCondition(terminationCondition) + .build(); + } + /** * Hints that the filepattern specified in {@link #from(String)} matches a very large number of * files. @@ -279,20 +332,40 @@ public Read withHintMatchesManyFiles() { return toBuilder().setHintMatchesManyFiles(true).build(); } + /** + * Configures whether or not a filepattern matching no files is allowed. When using {@link + * #watchForNewFiles}, it is always allowed and this parameter is ignored. + */ + public Read withEmptyMatchTreatment(EmptyMatchTreatment treatment) { + return toBuilder().setEmptyMatchTreatment(treatment).build(); + } + @Override public PCollection expand(PBegin input) { checkNotNull(getFilepattern(), "need to set the filepattern of a TextIO.Read transform"); - return getHintMatchesManyFiles() - ? input - .apply( - "Create filepattern", Create.ofProvider(getFilepattern(), StringUtf8Coder.of())) - .apply(readAll().withCompressionType(getCompressionType())) - : input.apply("Read", org.apache.beam.sdk.io.Read.from(getSource())); + if (getWatchForNewFilesInterval() == null && !getHintMatchesManyFiles()) { + return input.apply("Read", org.apache.beam.sdk.io.Read.from(getSource())); + } + // All other cases go through ReadAll. + ReadAll readAll = + readAll() + .withCompressionType(getCompressionType()) + .withEmptyMatchTreatment(getEmptyMatchTreatment()); + if (getWatchForNewFilesInterval() != null) { + readAll = + readAll.watchForNewFiles( + getWatchForNewFilesInterval(), getWatchForNewFilesTerminationCondition()); + } + return input + .apply("Create filepattern", Create.ofProvider(getFilepattern(), StringUtf8Coder.of())) + .apply("Via ReadAll", readAll); } // Helper to create a source specific to the requested compression type. protected FileBasedSource getSource() { - return wrapWithCompression(new TextSource(getFilepattern()), getCompressionType()); + return wrapWithCompression( + new TextSource(getFilepattern(), getEmptyMatchTreatment()), + getCompressionType()); } private static FileBasedSource wrapWithCompression( @@ -330,10 +403,17 @@ public void populateDisplayData(DisplayData.Builder builder) { String filepatternDisplay = getFilepattern().isAccessible() ? getFilepattern().get() : getFilepattern().toString(); builder - .add(DisplayData.item("compressionType", getCompressionType().toString()) - .withLabel("Compression Type")) - .addIfNotNull(DisplayData.item("filePattern", filepatternDisplay) - .withLabel("File Pattern")); + .add( + DisplayData.item("compressionType", getCompressionType().toString()) + .withLabel("Compression Type")) + .addIfNotNull( + DisplayData.item("filePattern", filepatternDisplay).withLabel("File Pattern")) + .add( + DisplayData.item("emptyMatchTreatment", getEmptyMatchTreatment().toString()) + .withLabel("Treatment of filepatterns that match no files")) + .addIfNotNull( + DisplayData.item("watchForNewFilesInterval", getWatchForNewFilesInterval()) + .withLabel("Interval to watch for new files")); } } @@ -344,6 +424,14 @@ public void populateDisplayData(DisplayData.Builder builder) { public abstract static class ReadAll extends PTransform, PCollection> { abstract CompressionType getCompressionType(); + + @Nullable + abstract Duration getWatchForNewFilesInterval(); + + @Nullable + abstract TerminationCondition getWatchForNewFilesTerminationCondition(); + + abstract EmptyMatchTreatment getEmptyMatchTreatment(); abstract long getDesiredBundleSizeBytes(); abstract Builder toBuilder(); @@ -351,6 +439,10 @@ public abstract static class ReadAll @AutoValue.Builder abstract static class Builder { abstract Builder setCompressionType(CompressionType compressionType); + abstract Builder setWatchForNewFilesInterval(Duration watchForNewFilesInterval); + abstract Builder setWatchForNewFilesTerminationCondition( + TerminationCondition condition); + abstract Builder setEmptyMatchTreatment(EmptyMatchTreatment treatment); abstract Builder setDesiredBundleSizeBytes(long desiredBundleSizeBytes); abstract ReadAll build(); @@ -361,6 +453,21 @@ public ReadAll withCompressionType(CompressionType compressionType) { return toBuilder().setCompressionType(compressionType).build(); } + /** Same as {@link Read#withEmptyMatchTreatment}. */ + public ReadAll withEmptyMatchTreatment(EmptyMatchTreatment treatment) { + return toBuilder().setEmptyMatchTreatment(treatment).build(); + } + + /** Same as {@link Read#watchForNewFiles(Duration, TerminationCondition)}. */ + @Experimental(Kind.SPLITTABLE_DO_FN) + public ReadAll watchForNewFiles( + Duration pollInterval, TerminationCondition terminationCondition) { + return toBuilder() + .setWatchForNewFilesInterval(pollInterval) + .setWatchForNewFilesTerminationCondition(terminationCondition) + .build(); + } + @VisibleForTesting ReadAll withDesiredBundleSizeBytes(long desiredBundleSizeBytes) { return toBuilder().setDesiredBundleSizeBytes(desiredBundleSizeBytes).build(); @@ -368,13 +475,21 @@ ReadAll withDesiredBundleSizeBytes(long desiredBundleSizeBytes) { @Override public PCollection expand(PCollection input) { + Match.Filepatterns matchFilepatterns = + Match.filepatterns().withEmptyMatchTreatment(getEmptyMatchTreatment()); + if (getWatchForNewFilesInterval() != null) { + matchFilepatterns = + matchFilepatterns.continuously( + getWatchForNewFilesInterval(), getWatchForNewFilesTerminationCondition()); + } return input + .apply(matchFilepatterns) .apply( "Read all via FileBasedSource", new ReadAllViaFileBasedSource<>( new IsSplittableFn(getCompressionType()), getDesiredBundleSizeBytes(), - new CreateTextSourceFn(getCompressionType()))) + new CreateTextSourceFn(getCompressionType(), getEmptyMatchTreatment()))) .setCoder(StringUtf8Coder.of()); } @@ -390,15 +505,18 @@ public void populateDisplayData(DisplayData.Builder builder) { private static class CreateTextSourceFn implements SerializableFunction> { private final CompressionType compressionType; + private final EmptyMatchTreatment emptyMatchTreatment; - private CreateTextSourceFn(CompressionType compressionType) { + private CreateTextSourceFn( + CompressionType compressionType, EmptyMatchTreatment emptyMatchTreatment) { this.compressionType = compressionType; + this.emptyMatchTreatment = emptyMatchTreatment; } @Override public FileBasedSource apply(String input) { return Read.wrapWithCompression( - new TextSource(StaticValueProvider.of(input)), compressionType); + new TextSource(StaticValueProvider.of(input), emptyMatchTreatment), compressionType); } } 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 37c626398bc62..3e023db679d1a 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 @@ -524,12 +524,15 @@ public interface OutputReceiver { *

            6. It must return {@code void}. * * - *

              Splittable DoFn's (WARNING: work in progress, do not use)

              + *

              Splittable DoFn's

              * *

              A {@link DoFn} is splittable if its {@link ProcessElement} method has a parameter * whose type is a subtype of {@link RestrictionTracker}. This is an advanced feature and an - * overwhelming majority of users will never need to write a splittable {@link DoFn}. Right now - * the implementation of this feature is in progress and it's not ready for any use. + * overwhelming majority of users will never need to write a splittable {@link DoFn}. + * + *

              Not all runners support Splittable DoFn. See the + * capability + * matrix. * *

              See the proposal for an overview of the * involved concepts (splittable DoFn, restriction, restriction tracker). @@ -558,8 +561,6 @@ public interface OutputReceiver { * * *

              A non-splittable {@link DoFn} must not define any of these methods. - * - *

              More documentation will be added when the feature becomes ready for general usage. */ @Documented @Retention(RetentionPolicy.RUNTIME) diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Watch.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Watch.java index fc6f18d6b5640..9da240843a7cc 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Watch.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Watch.java @@ -38,7 +38,6 @@ import java.io.InputStream; import java.io.OutputStream; import java.io.Serializable; -import java.lang.reflect.TypeVariable; import java.util.Arrays; import java.util.Collections; import java.util.LinkedList; @@ -64,6 +63,8 @@ import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.TimestampedValue; import org.apache.beam.sdk.values.TypeDescriptor; +import org.apache.beam.sdk.values.TypeDescriptors; +import org.apache.beam.sdk.values.TypeDescriptors.TypeVariableExtractor; import org.joda.time.Duration; import org.joda.time.Instant; import org.joda.time.ReadableDuration; @@ -554,14 +555,13 @@ public PCollection> expand(PCollection input) { if (outputCoder == null) { // If a coder was not specified explicitly, infer it from the OutputT type parameter // of the PollFn. - TypeDescriptor superDescriptor = - TypeDescriptor.of(getPollFn().getClass()).getSupertype(PollFn.class); - TypeVariable typeVariable = superDescriptor.getTypeParameter("OutputT"); - @SuppressWarnings("unchecked") - TypeDescriptor descriptor = - (TypeDescriptor) superDescriptor.resolveType(typeVariable); + TypeDescriptor outputT = + TypeDescriptors.extractFromTypeParameters( + getPollFn(), + PollFn.class, + new TypeVariableExtractor, OutputT>() {}); try { - outputCoder = input.getPipeline().getCoderRegistry().getCoder(descriptor); + outputCoder = input.getPipeline().getCoderRegistry().getCoder(outputT); } catch (CannotProvideCoderException e) { throw new RuntimeException( "Unable to infer coder for OutputT. Specify it explicitly using withOutputCoder()."); diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/TextIOReadTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/TextIOReadTest.java index 8ad60304389ee..aa6090dadbb8e 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/TextIOReadTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/TextIOReadTest.java @@ -25,6 +25,7 @@ import static org.apache.beam.sdk.io.TextIO.CompressionType.GZIP; import static org.apache.beam.sdk.io.TextIO.CompressionType.UNCOMPRESSED; import static org.apache.beam.sdk.io.TextIO.CompressionType.ZIP; +import static org.apache.beam.sdk.transforms.Watch.Growth.afterTimeSinceNewOutput; import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasDisplayItem; import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasValue; import static org.hamcrest.Matchers.containsInAnyOrder; @@ -63,6 +64,7 @@ import org.apache.beam.sdk.coders.StringUtf8Coder; import org.apache.beam.sdk.io.BoundedSource.BoundedReader; import org.apache.beam.sdk.io.TextIO.CompressionType; +import org.apache.beam.sdk.io.fs.EmptyMatchTreatment; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.options.ValueProvider; @@ -70,6 +72,7 @@ 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.testing.UsesSplittableParDo; import org.apache.beam.sdk.testing.ValidatesRunner; import org.apache.beam.sdk.transforms.Create; import org.apache.beam.sdk.transforms.display.DisplayData; @@ -78,6 +81,7 @@ import org.apache.beam.sdk.values.PCollection; import org.apache.commons.compress.compressors.bzip2.BZip2CompressorOutputStream; import org.apache.commons.compress.compressors.deflate.DeflateCompressorOutputStream; +import org.joda.time.Duration; import org.junit.AfterClass; import org.junit.BeforeClass; import org.junit.Rule; @@ -787,7 +791,8 @@ public void testSplittingSourceWithMixedDelimitersAndNonEmptyBytesAtEnd() throws private TextSource prepareSource(byte[] data) throws IOException { Path path = Files.createTempFile(tempFolder, "tempfile", "ext"); Files.write(path, data); - return new TextSource(ValueProvider.StaticValueProvider.of(path.toString())); + return new TextSource( + ValueProvider.StaticValueProvider.of(path.toString()), EmptyMatchTreatment.DISALLOW); } @Test @@ -872,4 +877,51 @@ public void testReadAll() throws IOException { PAssert.that(lines).containsInAnyOrder(Iterables.concat(TINY, TINY, LARGE, LARGE)); p.run(); } + + @Test + @Category({NeedsRunner.class, UsesSplittableParDo.class}) + public void testReadWatchForNewFiles() throws IOException, InterruptedException { + final Path basePath = tempFolder.resolve("readWatch"); + basePath.toFile().mkdir(); + PCollection lines = + p.apply( + TextIO.read() + .from(basePath.resolve("*").toString()) + // Make sure that compression type propagates into readAll() + .withCompressionType(ZIP) + .watchForNewFiles( + Duration.millis(100), afterTimeSinceNewOutput(Duration.standardSeconds(3)))); + + Thread writer = + new Thread() { + @Override + public void run() { + try { + Thread.sleep(1000); + writeToFile( + Arrays.asList("a.1", "a.2"), + basePath.resolve("fileA").toString(), + CompressionType.ZIP); + Thread.sleep(300); + writeToFile( + Arrays.asList("b.1", "b.2"), + basePath.resolve("fileB").toString(), + CompressionType.ZIP); + Thread.sleep(300); + writeToFile( + Arrays.asList("c.1", "c.2"), + basePath.resolve("fileC").toString(), + CompressionType.ZIP); + } catch (IOException | InterruptedException e) { + throw new RuntimeException(e); + } + } + }; + writer.start(); + + PAssert.that(lines).containsInAnyOrder("a.1", "a.2", "b.1", "b.2", "c.1", "c.2"); + p.run(); + + writer.join(); + } } From 97b12d53f956dfffd7594ae5b9433ac9df9c793a Mon Sep 17 00:00:00 2001 From: Eugene Kirpichov Date: Fri, 4 Aug 2017 16:46:42 -0700 Subject: [PATCH 247/346] [BEAM-2734] Unbreaks some Dataflow ValidatesRunner tests --- .../UnboundedReadFromBoundedSource.java | 11 ++++++++--- .../main/java/org/apache/beam/sdk/io/Source.java | 14 +++++++++++++- 2 files changed, 21 insertions(+), 4 deletions(-) diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/UnboundedReadFromBoundedSource.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/UnboundedReadFromBoundedSource.java index 55f95194cbc4d..24eb384792686 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/UnboundedReadFromBoundedSource.java +++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/UnboundedReadFromBoundedSource.java @@ -90,6 +90,11 @@ public PCollection expand(PBegin input) { Read.from(new BoundedToUnboundedSourceAdapter<>(source))); } + @Override + protected Coder getDefaultOutputCoder() { + return source.getDefaultOutputCoder(); + } + @Override public String getKindString() { return String.format("Read(%s)", NameUtils.approximateSimpleName(source)); @@ -161,14 +166,14 @@ public Reader createReader(PipelineOptions options, Checkpoint checkpoint) } @Override - public Coder getOutputCoder() { - return boundedSource.getOutputCoder(); + public Coder getDefaultOutputCoder() { + return boundedSource.getDefaultOutputCoder(); } @SuppressWarnings({"rawtypes", "unchecked"}) @Override public Coder> getCheckpointMarkCoder() { - return new CheckpointCoder<>(boundedSource.getOutputCoder()); + return new CheckpointCoder<>(boundedSource.getDefaultOutputCoder()); } @VisibleForTesting diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/Source.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/Source.java index 32a72705c346f..872c135ad0724 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/Source.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/Source.java @@ -64,11 +64,23 @@ public abstract class Source implements Serializable, HasDisplayData { /** @deprecated Override {@link #getOutputCoder()} instead. */ @Deprecated public Coder getDefaultOutputCoder() { - throw new UnsupportedOperationException("Source needs to override getOutputCoder()"); + // If the subclass doesn't override getDefaultOutputCoder(), hopefully it overrides the proper + // version - getOutputCoder(). Check that it does, before calling the method (if subclass + // doesn't override it, we'll call the default implementation and get infinite recursion). + try { + if (getClass().getMethod("getOutputCoder").getDeclaringClass().equals(Source.class)) { + throw new UnsupportedOperationException( + getClass() + " needs to override getOutputCoder()."); + } + } catch (NoSuchMethodException e) { + throw new RuntimeException(e); + } + return getOutputCoder(); } /** Returns the {@code Coder} to use for the data read from this source. */ public Coder getOutputCoder() { + // Call the old method for compatibility. return getDefaultOutputCoder(); } From f48bb4be1d3bb23f3cc978c4c25cf43842639296 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Jean-Baptiste=20Onofr=C3=A9?= Date: Mon, 24 Jul 2017 17:53:15 +0200 Subject: [PATCH 248/346] [BEAM-1274] Add SSL mutual authentication support --- .../sdk/io/elasticsearch/ElasticsearchIO.java | 69 ++++++++++++++++++- 1 file changed, 67 insertions(+), 2 deletions(-) diff --git a/sdks/java/io/elasticsearch/src/main/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIO.java b/sdks/java/io/elasticsearch/src/main/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIO.java index 50468887120bb..2cd3bcd0baed5 100644 --- a/sdks/java/io/elasticsearch/src/main/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIO.java +++ b/sdks/java/io/elasticsearch/src/main/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIO.java @@ -25,10 +25,14 @@ import com.google.auto.value.AutoValue; import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Strings; + +import java.io.File; +import java.io.FileInputStream; import java.io.IOException; +import java.io.InputStream; import java.io.Serializable; -import java.net.MalformedURLException; import java.net.URL; +import java.security.KeyStore; import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; @@ -39,6 +43,8 @@ import java.util.Map; import java.util.NoSuchElementException; import javax.annotation.Nullable; +import javax.net.ssl.SSLContext; + import org.apache.beam.sdk.annotations.Experimental; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.StringUtf8Coder; @@ -60,7 +66,9 @@ import org.apache.http.impl.client.BasicCredentialsProvider; import org.apache.http.impl.nio.client.HttpAsyncClientBuilder; import org.apache.http.message.BasicHeader; +import org.apache.http.nio.conn.ssl.SSLIOSessionStrategy; import org.apache.http.nio.entity.NStringEntity; +import org.apache.http.ssl.SSLContexts; import org.elasticsearch.client.Response; import org.elasticsearch.client.RestClient; import org.elasticsearch.client.RestClientBuilder; @@ -155,6 +163,12 @@ public abstract static class ConnectionConfiguration implements Serializable { @Nullable abstract String getPassword(); + @Nullable + abstract String getKeystorePath(); + + @Nullable + abstract String getKeystorePassword(); + abstract String getIndex(); abstract String getType(); @@ -169,6 +183,10 @@ abstract static class Builder { abstract Builder setPassword(String password); + abstract Builder setKeystorePath(String keystorePath); + + abstract Builder setKeystorePassword(String password); + abstract Builder setIndex(String index); abstract Builder setType(String type); @@ -239,6 +257,32 @@ public ConnectionConfiguration withPassword(String password) { return builder().setPassword(password).build(); } + /** + * If Elasticsearch uses SSL with mutual authentication (via shield), + * provide the keystore containing the client key. + * + * @param keystorePath the location of the keystore containing the client key. + * @return the {@link ConnectionConfiguration} object with keystore path set. + */ + public ConnectionConfiguration withKeystorePath(String keystorePath) { + checkArgument(keystorePath != null, "ConnectionConfiguration.create()" + + ".withKeystorePath(keystorePath) called with null keystorePath"); + return builder().setKeystorePath(keystorePath).build(); + } + + /** + * If Elasticsearch uses SSL with mutual authentication (via shield), + * provide the password to open the client keystore. + * + * @param keystorePassword the password of the client keystore. + * @return the {@link ConnectionConfiguration} object with keystore password set. + */ + public ConnectionConfiguration withKeystorePassword(String keystorePassword) { + checkArgument(keystorePassword != null, "ConnectionConfiguration.create()" + + ".withKeystorePassword(keystorePassword) called with null keystorePassword"); + return builder().setKeystorePassword(keystorePassword).build(); + } + private void populateDisplayData(DisplayData.Builder builder) { builder.add(DisplayData.item("address", getAddresses().toString())); builder.add(DisplayData.item("index", getIndex())); @@ -246,7 +290,7 @@ private void populateDisplayData(DisplayData.Builder builder) { builder.addIfNotNull(DisplayData.item("username", getUsername())); } - RestClient createClient() throws MalformedURLException { + RestClient createClient() throws IOException { HttpHost[] hosts = new HttpHost[getAddresses().size()]; int i = 0; for (String address : getAddresses()) { @@ -267,6 +311,27 @@ public HttpAsyncClientBuilder customizeHttpClient( } }); } + if (getKeystorePath() != null) { + try { + KeyStore keyStore = KeyStore.getInstance("jks"); + try (InputStream is = new FileInputStream(new File(getKeystorePath()))) { + keyStore.load(is, getKeystorePassword().toCharArray()); + } + final SSLContext sslContext = SSLContexts.custom() + .loadTrustMaterial(keyStore, null).build(); + final SSLIOSessionStrategy sessionStrategy = new SSLIOSessionStrategy(sslContext); + restClientBuilder.setHttpClientConfigCallback( + new RestClientBuilder.HttpClientConfigCallback() { + @Override + public HttpAsyncClientBuilder customizeHttpClient( + HttpAsyncClientBuilder httpClientBuilder) { + return httpClientBuilder.setSSLContext(sslContext).setSSLStrategy(sessionStrategy); + } + }); + } catch (Exception e) { + throw new IOException("Can't load the client certificate from the keystore", e); + } + } return restClientBuilder.build(); } } From 02f11d3db98f33475ff1152d33e36161d59fd400 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Jean-Baptiste=20Onofr=C3=A9?= Date: Mon, 7 Aug 2017 07:49:36 +0200 Subject: [PATCH 249/346] =?UTF-8?q?[BEAM-1274]=C2=A0Add=20SSL/TLS=20in=20t?= =?UTF-8?q?he=20comments,=20add=20the=20self=20signed=20policy=20support?= =?UTF-8?q?=20for=20the=20SSL=20context?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../beam/sdk/io/elasticsearch/ElasticsearchIO.java | 14 +++++++++----- 1 file changed, 9 insertions(+), 5 deletions(-) diff --git a/sdks/java/io/elasticsearch/src/main/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIO.java b/sdks/java/io/elasticsearch/src/main/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIO.java index 2cd3bcd0baed5..e6a6a9ff4d9b9 100644 --- a/sdks/java/io/elasticsearch/src/main/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIO.java +++ b/sdks/java/io/elasticsearch/src/main/java/org/apache/beam/sdk/io/elasticsearch/ElasticsearchIO.java @@ -62,6 +62,7 @@ import org.apache.http.auth.AuthScope; import org.apache.http.auth.UsernamePasswordCredentials; import org.apache.http.client.CredentialsProvider; +import org.apache.http.conn.ssl.TrustSelfSignedStrategy; import org.apache.http.entity.ContentType; import org.apache.http.impl.client.BasicCredentialsProvider; import org.apache.http.impl.nio.client.HttpAsyncClientBuilder; @@ -258,7 +259,7 @@ public ConnectionConfiguration withPassword(String password) { } /** - * If Elasticsearch uses SSL with mutual authentication (via shield), + * If Elasticsearch uses SSL/TLS with mutual authentication (via shield), * provide the keystore containing the client key. * * @param keystorePath the location of the keystore containing the client key. @@ -267,15 +268,17 @@ public ConnectionConfiguration withPassword(String password) { public ConnectionConfiguration withKeystorePath(String keystorePath) { checkArgument(keystorePath != null, "ConnectionConfiguration.create()" + ".withKeystorePath(keystorePath) called with null keystorePath"); + checkArgument(!keystorePath.isEmpty(), "ConnectionConfiguration.create()" + + ".withKeystorePath(keystorePath) called with empty keystorePath"); return builder().setKeystorePath(keystorePath).build(); } /** - * If Elasticsearch uses SSL with mutual authentication (via shield), + * If Elasticsearch uses SSL/TLS with mutual authentication (via shield), * provide the password to open the client keystore. * * @param keystorePassword the password of the client keystore. - * @return the {@link ConnectionConfiguration} object with keystore password set. + * @return the {@link ConnectionConfiguration} object with keystore passwo:rd set. */ public ConnectionConfiguration withKeystorePassword(String keystorePassword) { checkArgument(keystorePassword != null, "ConnectionConfiguration.create()" @@ -288,6 +291,7 @@ private void populateDisplayData(DisplayData.Builder builder) { builder.add(DisplayData.item("index", getIndex())); builder.add(DisplayData.item("type", getType())); builder.addIfNotNull(DisplayData.item("username", getUsername())); + builder.addIfNotNull(DisplayData.item("keystore.path", getKeystorePath())); } RestClient createClient() throws IOException { @@ -311,14 +315,14 @@ public HttpAsyncClientBuilder customizeHttpClient( } }); } - if (getKeystorePath() != null) { + if (getKeystorePath() != null && !getKeystorePath().isEmpty()) { try { KeyStore keyStore = KeyStore.getInstance("jks"); try (InputStream is = new FileInputStream(new File(getKeystorePath()))) { keyStore.load(is, getKeystorePassword().toCharArray()); } final SSLContext sslContext = SSLContexts.custom() - .loadTrustMaterial(keyStore, null).build(); + .loadTrustMaterial(keyStore, new TrustSelfSignedStrategy()).build(); final SSLIOSessionStrategy sessionStrategy = new SSLIOSessionStrategy(sslContext); restClientBuilder.setHttpClientConfigCallback( new RestClientBuilder.HttpClientConfigCallback() { From 545946a97366cfb91069ee2bd4a3cf0f06e0b266 Mon Sep 17 00:00:00 2001 From: Thomas Groh Date: Mon, 7 Aug 2017 09:27:46 -0700 Subject: [PATCH 250/346] Suppress Attempted Metrics tests in Dataflow Batch Batch does not support attempted metrics, only committed and tentative. --- 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 1181b79876382..46352fbc52e99 100644 --- a/runners/google-cloud-dataflow-java/pom.xml +++ b/runners/google-cloud-dataflow-java/pom.xml @@ -245,6 +245,7 @@ --> org.apache.beam.sdk.testing.LargeKeys$Above10MB, + org.apache.beam.sdk.testing.UsesAttemptedMetrics, org.apache.beam.sdk.testing.UsesDistributionMetrics, org.apache.beam.sdk.testing.UsesGaugeMetrics, org.apache.beam.sdk.testing.UsesSetState, From 7dbe906d463ac823c76e56239897f456cb8d53a1 Mon Sep 17 00:00:00 2001 From: Thomas Groh Date: Fri, 4 Aug 2017 09:51:02 -0700 Subject: [PATCH 251/346] Add GRPC to Compilation of the Runner API Generates the JobService GRPC classes. --- sdks/common/runner-api/pom.xml | 23 +++++++++++++++++++++++ 1 file changed, 23 insertions(+) diff --git a/sdks/common/runner-api/pom.xml b/sdks/common/runner-api/pom.xml index 8bc4123cb541d..e138ca8416acb 100644 --- a/sdks/common/runner-api/pom.xml +++ b/sdks/common/runner-api/pom.xml @@ -65,11 +65,14 @@ protobuf-maven-plugin com.google.protobuf:protoc:${protobuf.version}:exe:${os.detected.classifier} + grpc-java + io.grpc:protoc-gen-grpc-java:${grpc.version}:exe:${os.detected.classifier} compile + compile-custom @@ -82,5 +85,25 @@ com.google.protobuf protobuf-java + + + com.google.guava + guava + + + + io.grpc + grpc-core + + + + io.grpc + grpc-protobuf + + + + io.grpc + grpc-stub + From 6c81a3aa77e8cf6f1f7499e60f1c7d1ce1dfda46 Mon Sep 17 00:00:00 2001 From: jasonkuster Date: Mon, 7 Aug 2017 14:13:57 -0700 Subject: [PATCH 252/346] Delete list job creation -- creates in wrong place. --- .../jenkins/job_beam_ListView_Create.groovy | 42 ------------------- 1 file changed, 42 deletions(-) delete mode 100644 .test-infra/jenkins/job_beam_ListView_Create.groovy diff --git a/.test-infra/jenkins/job_beam_ListView_Create.groovy b/.test-infra/jenkins/job_beam_ListView_Create.groovy deleted file mode 100644 index 9c304da46c2b0..0000000000000 --- a/.test-infra/jenkins/job_beam_ListView_Create.groovy +++ /dev/null @@ -1,42 +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. - */ - -// This creates a standard List View for Beam. It filters out executors not associated with Beam -// and pulls in all jobs which start with 'beam_'. -listView('Beam') { - description('Jenkins builds for the Beam project.') - filterBuildQueue() - filterExecutors() - jobs { - regex(/beam_.+/) - } - jobFilters { - all() - } - columns { - status() - weather() - name() - lastSuccess() - lastFailure() - lastDuration() - lastBuildConsole() - buildButton() - } -} - From 2fa4fdecd8ef06534e369e527b15ae8193823c8b Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Tue, 8 Aug 2017 09:12:29 -0700 Subject: [PATCH 253/346] Fix compile break --- .../gearpump/translators/CreateStreamingGearpumpView.java | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/CreateStreamingGearpumpView.java b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/CreateStreamingGearpumpView.java index 60577b338a671..3ebe5c842ef82 100644 --- a/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/CreateStreamingGearpumpView.java +++ b/runners/gearpump/src/main/java/org/apache/beam/runners/gearpump/translators/CreateStreamingGearpumpView.java @@ -124,8 +124,7 @@ public static CreateGearpumpPCollectionView of( @Override public PCollection> expand(PCollection> input) { return PCollection.>createPrimitiveOutputInternal( - input.getPipeline(), input.getWindowingStrategy(), input.isBounded()) - .setCoder(input.getCoder()); + input.getPipeline(), input.getWindowingStrategy(), input.isBounded(), input.getCoder()); } public PCollectionView getView() { From d00ff9e215092af2666459b742f62c6b0bb4bff9 Mon Sep 17 00:00:00 2001 From: Cao Manh Dat Date: Sat, 22 Jul 2017 14:38:07 +0700 Subject: [PATCH 254/346] [BEAM-2657] Create Solr IO --- pom.xml | 8 +- .../sdk/io/common/IOTestPipelineOptions.java | 6 + sdks/java/io/pom.xml | 1 + sdks/java/io/solr/pom.xml | 147 ++++ .../sdk/io/solr/AuthorizedSolrClient.java | 91 +++ .../beam/sdk/io/solr/JavaBinCodecCoder.java | 98 +++ .../org/apache/beam/sdk/io/solr/SolrIO.java | 717 ++++++++++++++++++ .../apache/beam/sdk/io/solr/package-info.java | 20 + .../sdk/io/solr/JavaBinCodecCoderTest.java | 81 ++ .../apache/beam/sdk/io/solr/SolrIOTest.java | 269 +++++++ .../beam/sdk/io/solr/SolrIOTestUtils.java | 132 ++++ .../resources/cloud-minimal/conf/schema.xml | 29 + .../cloud-minimal/conf/solrconfig.xml | 48 ++ sdks/java/javadoc/pom.xml | 5 + 14 files changed, 1651 insertions(+), 1 deletion(-) create mode 100644 sdks/java/io/solr/pom.xml create mode 100644 sdks/java/io/solr/src/main/java/org/apache/beam/sdk/io/solr/AuthorizedSolrClient.java create mode 100644 sdks/java/io/solr/src/main/java/org/apache/beam/sdk/io/solr/JavaBinCodecCoder.java create mode 100644 sdks/java/io/solr/src/main/java/org/apache/beam/sdk/io/solr/SolrIO.java create mode 100644 sdks/java/io/solr/src/main/java/org/apache/beam/sdk/io/solr/package-info.java create mode 100644 sdks/java/io/solr/src/test/java/org/apache/beam/sdk/io/solr/JavaBinCodecCoderTest.java create mode 100644 sdks/java/io/solr/src/test/java/org/apache/beam/sdk/io/solr/SolrIOTest.java create mode 100644 sdks/java/io/solr/src/test/java/org/apache/beam/sdk/io/solr/SolrIOTestUtils.java create mode 100644 sdks/java/io/solr/src/test/resources/cloud-minimal/conf/schema.xml create mode 100644 sdks/java/io/solr/src/test/resources/cloud-minimal/conf/solrconfig.xml diff --git a/pom.xml b/pom.xml index 80ab6e276331d..1bdaa97f5259b 100644 --- a/pom.xml +++ b/pom.xml @@ -524,7 +524,13 @@ ${project.version} - + + org.apache.beam + beam-sdks-java-io-solr + ${project.version} + + + org.apache.beam beam-sdks-java-io-hadoop-input-format ${project.version} diff --git a/sdks/java/io/common/src/test/java/org/apache/beam/sdk/io/common/IOTestPipelineOptions.java b/sdks/java/io/common/src/test/java/org/apache/beam/sdk/io/common/IOTestPipelineOptions.java index 25ab9298ea2fc..256c94d8019f6 100644 --- a/sdks/java/io/common/src/test/java/org/apache/beam/sdk/io/common/IOTestPipelineOptions.java +++ b/sdks/java/io/common/src/test/java/org/apache/beam/sdk/io/common/IOTestPipelineOptions.java @@ -71,6 +71,12 @@ public interface IOTestPipelineOptions extends TestPipelineOptions { Integer getElasticsearchHttpPort(); void setElasticsearchHttpPort(Integer value); + /* Solr */ + @Description("Address of Zookeeper server for Solr") + @Default.String("zookeeper-server") + String getSolrZookeeperServer(); + void setSolrZookeeperServer(String value); + /* Cassandra */ @Description("Host for Cassandra server (host name/ip address)") @Default.String("cassandra-host") diff --git a/sdks/java/io/pom.xml b/sdks/java/io/pom.xml index 4e02aa8286f1f..c291e5d1d3cf4 100644 --- a/sdks/java/io/pom.xml +++ b/sdks/java/io/pom.xml @@ -56,6 +56,7 @@ kinesis mongodb mqtt + solr xml diff --git a/sdks/java/io/solr/pom.xml b/sdks/java/io/solr/pom.xml new file mode 100644 index 0000000000000..a757a5721fbf9 --- /dev/null +++ b/sdks/java/io/solr/pom.xml @@ -0,0 +1,147 @@ + + + + + beam-sdks-java-io-parent + org.apache.beam + 2.2.0-SNAPSHOT + ../pom.xml + + 4.0.0 + + beam-sdks-java-io-solr + Apache Beam :: SDKs :: Java :: IO :: Solr + IO to read and write from/to Solr. + + + + org.apache.beam + beam-sdks-java-core + + + + com.google.guava + guava + + + + org.apache.solr + solr-solrj + 5.5.4 + + + + com.google.code.findbugs + jsr305 + + + + org.apache.commons + commons-compress + + + + + com.google.auto.value + auto-value + provided + + + + com.google.auto.service + auto-service + true + + + + org.apache.httpcomponents + httpclient + 4.4.1 + provided + + + + + org.hamcrest + hamcrest-core + test + + + + org.hamcrest + hamcrest-all + test + + + + junit + junit + test + + + + org.apache.beam + beam-sdks-java-io-common + test + tests + + + + org.apache.beam + beam-runners-direct-java + test + + + + org.apache.solr + solr-test-framework + 5.5.4 + test + + + + org.apache.solr + solr-core + 5.5.4 + test + + + + org.slf4j + slf4j-api + test + + + + com.carrotsearch.randomizedtesting + randomizedtesting-runner + 2.3.2 + test + + + + org.slf4j + slf4j-log4j12 + ${slf4j.version} + test + + + + \ No newline at end of file diff --git a/sdks/java/io/solr/src/main/java/org/apache/beam/sdk/io/solr/AuthorizedSolrClient.java b/sdks/java/io/solr/src/main/java/org/apache/beam/sdk/io/solr/AuthorizedSolrClient.java new file mode 100644 index 0000000000000..44d7b88d9fde4 --- /dev/null +++ b/sdks/java/io/solr/src/main/java/org/apache/beam/sdk/io/solr/AuthorizedSolrClient.java @@ -0,0 +1,91 @@ +/* + * 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.solr; + +import static com.google.common.base.Preconditions.checkArgument; + +import java.io.Closeable; +import java.io.IOException; + +import org.apache.beam.sdk.io.solr.SolrIO.ConnectionConfiguration; +import org.apache.solr.client.solrj.SolrClient; +import org.apache.solr.client.solrj.SolrRequest; +import org.apache.solr.client.solrj.SolrResponse; +import org.apache.solr.client.solrj.SolrServerException; +import org.apache.solr.client.solrj.impl.CloudSolrClient; +import org.apache.solr.client.solrj.request.CollectionAdminRequest; +import org.apache.solr.client.solrj.request.CoreAdminRequest; +import org.apache.solr.client.solrj.request.QueryRequest; +import org.apache.solr.client.solrj.response.CoreAdminResponse; +import org.apache.solr.client.solrj.response.QueryResponse; +import org.apache.solr.common.cloud.ClusterState; +import org.apache.solr.common.params.SolrParams; + +/** + * Client for interact with Solr. + * @param type of SolrClient + */ +class AuthorizedSolrClient implements Closeable { + private final ClientT solrClient; + private final String username; + private final String password; + + AuthorizedSolrClient(ClientT solrClient, ConnectionConfiguration configuration) { + checkArgument( + solrClient != null, + "solrClient can not be null"); + checkArgument( + configuration != null, + "configuration can not be null"); + this.solrClient = solrClient; + this.username = configuration.getUsername(); + this.password = configuration.getPassword(); + } + + QueryResponse query(String collection, SolrParams solrParams) + throws IOException, SolrServerException { + QueryRequest query = new QueryRequest(solrParams); + return process(collection, query); + } + + ResponseT process(String collection, + SolrRequest request) throws IOException, SolrServerException { + request.setBasicAuthCredentials(username, password); + return request.process(solrClient, collection); + } + + CoreAdminResponse process(CoreAdminRequest request) + throws IOException, SolrServerException { + return process(null, request); + } + + SolrResponse process(CollectionAdminRequest request) + throws IOException, SolrServerException { + return process(null, request); + } + + static ClusterState getClusterState( + AuthorizedSolrClient authorizedSolrClient) { + authorizedSolrClient.solrClient.connect(); + return authorizedSolrClient.solrClient.getZkStateReader().getClusterState(); + } + + @Override public void close() throws IOException { + solrClient.close(); + } +} diff --git a/sdks/java/io/solr/src/main/java/org/apache/beam/sdk/io/solr/JavaBinCodecCoder.java b/sdks/java/io/solr/src/main/java/org/apache/beam/sdk/io/solr/JavaBinCodecCoder.java new file mode 100644 index 0000000000000..aef3c4b4a5ed0 --- /dev/null +++ b/sdks/java/io/solr/src/main/java/org/apache/beam/sdk/io/solr/JavaBinCodecCoder.java @@ -0,0 +1,98 @@ +/* + * 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.solr; + +import com.google.auto.service.AutoService; +import java.io.ByteArrayOutputStream; +import java.io.DataInputStream; +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.util.Arrays; +import java.util.List; +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.CoderProvider; +import org.apache.beam.sdk.coders.CoderProviderRegistrar; +import org.apache.beam.sdk.coders.CoderProviders; +import org.apache.beam.sdk.util.VarInt; +import org.apache.beam.sdk.values.TypeDescriptor; +import org.apache.commons.compress.utils.BoundedInputStream; +import org.apache.solr.common.SolrDocument; +import org.apache.solr.common.SolrInputDocument; +import org.apache.solr.common.util.JavaBinCodec; + +/** A {@link Coder} that encodes using {@link JavaBinCodec}. */ +class JavaBinCodecCoder extends AtomicCoder { + private final Class clazz; + + private JavaBinCodecCoder(Class clazz) { + this.clazz = clazz; + } + + public static JavaBinCodecCoder of(Class clazz) { + return new JavaBinCodecCoder<>(clazz); + } + + @Override + public void encode(T value, OutputStream outStream) throws IOException { + if (value == null) { + throw new CoderException("cannot encode a null SolrDocument"); + } + + ByteArrayOutputStream baos = new ByteArrayOutputStream(); + JavaBinCodec codec = new JavaBinCodec(); + codec.marshal(value, baos); + + byte[] bytes = baos.toByteArray(); + VarInt.encode(bytes.length, outStream); + outStream.write(bytes); + } + + @Override + public T decode(InputStream inStream) throws IOException { + DataInputStream in = new DataInputStream(inStream); + + int len = VarInt.decodeInt(in); + if (len < 0) { + throw new CoderException("Invalid encoded SolrDocument length: " + len); + } + + JavaBinCodec codec = new JavaBinCodec(); + return (T) codec.unmarshal(new BoundedInputStream(in, len)); + } + + @Override + public TypeDescriptor getEncodedTypeDescriptor() { + return TypeDescriptor.of(clazz); + } + + @AutoService(CoderProviderRegistrar.class) + public static class Provider implements CoderProviderRegistrar { + @Override + public List getCoderProviders() { + return Arrays.asList( + CoderProviders.forCoder( + TypeDescriptor.of(SolrDocument.class), JavaBinCodecCoder.of(SolrDocument.class)), + CoderProviders.forCoder( + TypeDescriptor.of(SolrInputDocument.class), + JavaBinCodecCoder.of(SolrInputDocument.class))); + } + } +} diff --git a/sdks/java/io/solr/src/main/java/org/apache/beam/sdk/io/solr/SolrIO.java b/sdks/java/io/solr/src/main/java/org/apache/beam/sdk/io/solr/SolrIO.java new file mode 100644 index 0000000000000..c137eea37fa28 --- /dev/null +++ b/sdks/java/io/solr/src/main/java/org/apache/beam/sdk/io/solr/SolrIO.java @@ -0,0 +1,717 @@ +/* + * 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.solr; + +import static com.google.common.base.Preconditions.checkArgument; +import static com.google.common.base.Preconditions.checkState; + +import com.google.auto.value.AutoValue; +import com.google.common.annotations.VisibleForTesting; +import com.google.common.util.concurrent.MoreExecutors; +import com.google.common.util.concurrent.ThreadFactoryBuilder; + +import java.io.IOException; +import java.io.Serializable; +import java.net.MalformedURLException; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.Iterator; +import java.util.List; +import java.util.NoSuchElementException; +import java.util.concurrent.Callable; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.Future; +import javax.annotation.Nullable; + +import org.apache.beam.sdk.annotations.Experimental; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.io.BoundedSource; +import org.apache.beam.sdk.options.PipelineOptions; +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.display.DisplayData; +import org.apache.beam.sdk.values.PBegin; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.PDone; +import org.apache.http.client.HttpClient; +import org.apache.solr.client.solrj.SolrQuery; +import org.apache.solr.client.solrj.SolrServerException; +import org.apache.solr.client.solrj.impl.CloudSolrClient; +import org.apache.solr.client.solrj.impl.HttpClientUtil; +import org.apache.solr.client.solrj.impl.HttpSolrClient; +import org.apache.solr.client.solrj.request.CoreAdminRequest; +import org.apache.solr.client.solrj.request.UpdateRequest; +import org.apache.solr.client.solrj.request.schema.SchemaRequest; +import org.apache.solr.client.solrj.response.CoreAdminResponse; +import org.apache.solr.client.solrj.response.QueryResponse; +import org.apache.solr.client.solrj.response.schema.SchemaResponse; +import org.apache.solr.common.SolrDocument; +import org.apache.solr.common.SolrInputDocument; +import org.apache.solr.common.cloud.ClusterState; +import org.apache.solr.common.cloud.DocCollection; +import org.apache.solr.common.cloud.Replica; +import org.apache.solr.common.cloud.Slice; +import org.apache.solr.common.cloud.ZkStateReader; +import org.apache.solr.common.params.CoreAdminParams; +import org.apache.solr.common.params.CursorMarkParams; +import org.apache.solr.common.params.ModifiableSolrParams; +import org.apache.solr.common.util.NamedList; + +/** + * Transforms for reading and writing data from/to Solr. + * + *

              Reading from Solr

              + * + *

              {@link SolrIO#read SolrIO.read()} returns a bounded {@link PCollection + * PCollection<SolrDocument>} representing Solr documents. + * + *

              To configure the {@link SolrIO#read}, you have to provide a connection configuration + * containing the Zookeeper address of the Solr cluster, and the collection name. The following + * example illustrates options for configuring the source: + * + *

              {@code
              + * SolrIO.ConnectionConfiguration conn = SolrIO.ConnectionConfiguration.create("127.0.0.1:9983");
              + * // Optionally: .withBasicCredentials(username, password)
              + *
              + * PCollection docs = p.apply(
              + *     SolrIO.read().from("my-collection").withConnectionConfiguration(conn));
              + *
              + * }
              + * + *

              You can specify a query on the {@code read()} using {@code withQuery()}. + * + *

              Writing to Solr

              + * + *

              To write documents to Solr, use {@link SolrIO#write SolrIO.write()}, which writes Solr + * documents from a {@link PCollection PCollection<SolrInputDocument>} (which can be bounded + * or unbounded). + * + *

              To configure {@link SolrIO#write SolrIO.write()}, similar to the read, you have to provide a + * connection configuration, and a collection name. For instance: + * + *

              {@code
              + * PCollection inputDocs = ...;
              + * inputDocs.apply(SolrIO.write().to("my-collection").withConnectionConfiguration(conn));
              + *
              + * }
              + */ +@Experimental(Experimental.Kind.SOURCE_SINK) +public class SolrIO { + + public static Read read() { + // 1000 for batch size is good enough in many cases, + // ex: if document size is large, around 10KB, the response's size will be around 10MB + // if document seize is small, around 1KB, the response's size will be around 1MB + return new AutoValue_SolrIO_Read.Builder().setBatchSize(1000).setQuery("*:*").build(); + } + + public static Write write() { + // 1000 for batch size is good enough in many cases, + // ex: if document size is large, around 10KB, the request's size will be around 10MB + // if document seize is small, around 1KB, the request's size will be around 1MB + return new AutoValue_SolrIO_Write.Builder().setMaxBatchSize(1000).build(); + } + + private SolrIO() {} + + /** A POJO describing a connection configuration to Solr. */ + @AutoValue + public abstract static class ConnectionConfiguration implements Serializable { + + abstract String getZkHost(); + + @Nullable + abstract String getUsername(); + + @Nullable + abstract String getPassword(); + + abstract Builder builder(); + + @AutoValue.Builder + abstract static class Builder { + abstract Builder setZkHost(String zkHost); + + abstract Builder setUsername(String username); + + abstract Builder setPassword(String password); + + abstract ConnectionConfiguration build(); + } + + /** + * Creates a new Solr connection configuration. + * + * @param zkHost host of zookeeper + * @return the connection configuration object + */ + public static ConnectionConfiguration create(String zkHost) { + checkArgument(zkHost != null, "zkHost can not be null"); + return new AutoValue_SolrIO_ConnectionConfiguration.Builder().setZkHost(zkHost).build(); + } + + /** If Solr basic authentication is enabled, provide the username and password. */ + public ConnectionConfiguration withBasicCredentials(String username, String password) { + checkArgument(username != null, "username can not be null"); + checkArgument(!username.isEmpty(), "username can not be empty"); + checkArgument(password != null, "password can not be null"); + checkArgument(!password.isEmpty(), "password can not be empty"); + return builder().setUsername(username).setPassword(password).build(); + } + + private void populateDisplayData(DisplayData.Builder builder) { + builder.add(DisplayData.item("zkHost", getZkHost())); + builder.addIfNotNull(DisplayData.item("username", getUsername())); + } + + private HttpClient createHttpClient() { + // This is bug in Solr, if we don't create a customize HttpClient, + // UpdateRequest with commit flag will throw an authentication error. + ModifiableSolrParams params = new ModifiableSolrParams(); + params.set(HttpClientUtil.PROP_BASIC_AUTH_USER, getUsername()); + params.set(HttpClientUtil.PROP_BASIC_AUTH_PASS, getPassword()); + return HttpClientUtil.createClient(params); + } + + AuthorizedSolrClient createClient() throws MalformedURLException { + CloudSolrClient solrClient = new CloudSolrClient(getZkHost(), createHttpClient()); + return new AuthorizedSolrClient<>(solrClient, this); + } + + AuthorizedSolrClient createClient(String shardUrl) { + HttpSolrClient solrClient = new HttpSolrClient(shardUrl, createHttpClient()); + return new AuthorizedSolrClient<>(solrClient, this); + } + } + + /** A {@link PTransform} reading data from Solr. */ + @AutoValue + public abstract static class Read extends PTransform> { + private static final long MAX_BATCH_SIZE = 10000L; + + @Nullable + abstract ConnectionConfiguration getConnectionConfiguration(); + + @Nullable + abstract String getCollection(); + + abstract String getQuery(); + + abstract int getBatchSize(); + + abstract Builder builder(); + + @AutoValue.Builder + abstract static class Builder { + abstract Builder setConnectionConfiguration(ConnectionConfiguration connectionConfiguration); + + abstract Builder setQuery(String query); + + abstract Builder setBatchSize(int batchSize); + + abstract Builder setCollection(String collection); + + abstract Read build(); + } + + /** Provide the Solr connection configuration object. */ + public Read withConnectionConfiguration(ConnectionConfiguration connectionConfiguration) { + checkArgument(connectionConfiguration != null, "connectionConfiguration can not be null"); + return builder().setConnectionConfiguration(connectionConfiguration).build(); + } + + /** + * Provide name of collection while reading from Solr. + * + * @param collection the collection toward which the requests will be issued + */ + public Read from(String collection) { + checkArgument(collection != null, "collection can not be null"); + return builder().setCollection(collection).build(); + } + + /** + * Provide a query used while reading from Solr. + * + * @param query the query. See Solr + * Query + */ + public Read withQuery(String query) { + checkArgument(query != null, "query can not be null"); + checkArgument(!query.isEmpty(), "query can not be empty"); + return builder().setQuery(query).build(); + } + + /** + * Provide a size for the cursor read. See cursor API + * Default is 100. Maximum is 10 000. If documents are small, increasing batch size might + * improve read performance. If documents are big, you might need to decrease batchSize + * + * @param batchSize number of documents read in each scroll read + */ + @VisibleForTesting + Read withBatchSize(int batchSize) { + // TODO remove this configuration, we can figure out the best number + // by tuning batchSize when pipelines run. + checkArgument( + batchSize > 0 && batchSize < MAX_BATCH_SIZE, + "Valid values for batchSize are 1 (inclusize) to %s (exclusive), but was: %s ", + MAX_BATCH_SIZE, + batchSize); + return builder().setBatchSize(batchSize).build(); + } + + @Override + public PCollection expand(PBegin input) { + return input.apply(org.apache.beam.sdk.io.Read.from(new BoundedSolrSource(this, null))); + } + + @Override + public void validate(PipelineOptions options) { + checkState( + getConnectionConfiguration() != null, + "Need to set connection configuration using withConnectionConfiguration()"); + checkState(getCollection() != null, "Need to set collection name using to()"); + } + + @Override + public void populateDisplayData(DisplayData.Builder builder) { + super.populateDisplayData(builder); + builder.addIfNotNull(DisplayData.item("query", getQuery())); + getConnectionConfiguration().populateDisplayData(builder); + } + } + + /** A POJO describing a replica of Solr. */ + @AutoValue + abstract static class ReplicaInfo implements Serializable { + public abstract String coreName(); + + public abstract String coreUrl(); + + public abstract String baseUrl(); + + static ReplicaInfo create(Replica replica) { + return new AutoValue_SolrIO_ReplicaInfo( + replica.getStr(ZkStateReader.CORE_NAME_PROP), + replica.getCoreUrl(), + replica.getStr(ZkStateReader.BASE_URL_PROP)); + } + } + + /** A {@link BoundedSource} reading from Solr. */ + @VisibleForTesting + static class BoundedSolrSource extends BoundedSource { + + private final SolrIO.Read spec; + // replica is the info of the shard where the source will read the documents + @Nullable private final ReplicaInfo replica; + + BoundedSolrSource(Read spec, @Nullable Replica replica) { + this.spec = spec; + this.replica = replica == null ? null : ReplicaInfo.create(replica); + } + + @Override + public List> split( + long desiredBundleSizeBytes, PipelineOptions options) throws Exception { + ConnectionConfiguration connectionConfig = spec.getConnectionConfiguration(); + List sources = new ArrayList<>(); + try (AuthorizedSolrClient client = connectionConfig.createClient()) { + String collection = spec.getCollection(); + final ClusterState clusterState = AuthorizedSolrClient.getClusterState(client); + DocCollection docCollection = clusterState.getCollection(collection); + for (Slice slice : docCollection.getSlices()) { + ArrayList replicas = new ArrayList<>(slice.getReplicas()); + Collections.shuffle(replicas); + // Load balancing by randomly picking an active replica + Replica randomActiveReplica = null; + for (Replica replica : replicas) { + // We need to check both state of the replica and live nodes + // to make sure that the replica is alive + if (replica.getState() == Replica.State.ACTIVE + && clusterState.getLiveNodes().contains(replica.getNodeName())) { + randomActiveReplica = replica; + break; + } + } + // TODO in case of this replica goes inactive while the pipeline runs. + // We should pick another active replica of this shard. + checkState( + randomActiveReplica != null, + "Can not found an active replica for slice %s", + slice.getName()); + sources.add(new BoundedSolrSource(spec, randomActiveReplica)); + } + } + return sources; + } + + @Override + public long getEstimatedSizeBytes(PipelineOptions options) throws IOException { + if (replica != null) { + return getEstimatedSizeOfShard(replica); + } else { + return getEstimatedSizeOfCollection(); + } + } + + private long getEstimatedSizeOfShard(ReplicaInfo replica) throws IOException { + try (AuthorizedSolrClient solrClient = + spec.getConnectionConfiguration().createClient(replica.baseUrl())) { + CoreAdminRequest req = new CoreAdminRequest(); + req.setAction(CoreAdminParams.CoreAdminAction.STATUS); + req.setIndexInfoNeeded(true); + CoreAdminResponse response; + try { + response = solrClient.process(req); + } catch (SolrServerException e) { + throw new IOException("Can not get core status from " + replica, e); + } + NamedList coreStatus = response.getCoreStatus(replica.coreName()); + NamedList indexStats = (NamedList) coreStatus.get("index"); + return (long) indexStats.get("sizeInBytes"); + } + } + + private long getEstimatedSizeOfCollection() throws IOException { + long sizeInBytes = 0; + ConnectionConfiguration config = spec.getConnectionConfiguration(); + try (AuthorizedSolrClient solrClient = config.createClient()) { + DocCollection docCollection = + AuthorizedSolrClient.getClusterState(solrClient).getCollection(spec.getCollection()); + if (docCollection.getSlices().isEmpty()) { + return 0; + } + + ArrayList slices = new ArrayList<>(docCollection.getSlices()); + Collections.shuffle(slices); + ExecutorService executor = + Executors.newCachedThreadPool( + new ThreadFactoryBuilder() + .setThreadFactory(MoreExecutors.platformThreadFactory()) + .setDaemon(true) + .setNameFormat("solrio-size-of-collection-estimation") + .build()); + try { + ArrayList> futures = new ArrayList<>(); + for (int i = 0; i < 100 && i < slices.size(); i++) { + Slice slice = slices.get(i); + final Replica replica = slice.getLeader(); + Future future = + executor.submit( + new Callable() { + @Override + public Long call() throws Exception { + return getEstimatedSizeOfShard(ReplicaInfo.create(replica)); + } + }); + futures.add(future); + } + for (Future future : futures) { + try { + sizeInBytes += future.get(); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new IOException(e); + } catch (ExecutionException e) { + throw new IOException("Can not estimate size of shard", e.getCause()); + } + } + } finally { + executor.shutdownNow(); + } + + if (slices.size() <= 100) { + return sizeInBytes; + } + return (sizeInBytes / 100) * slices.size(); + } + } + + @Override + public void populateDisplayData(DisplayData.Builder builder) { + spec.populateDisplayData(builder); + if (replica != null) { + builder.addIfNotNull(DisplayData.item("shardUrl", replica.coreUrl())); + } + } + + @Override + public BoundedReader createReader(PipelineOptions options) throws IOException { + return new BoundedSolrReader(this); + } + + @Override + public void validate() { + spec.validate(null); + } + + @Override + public Coder getOutputCoder() { + return JavaBinCodecCoder.of(SolrDocument.class); + } + } + + private static class BoundedSolrReader extends BoundedSource.BoundedReader { + + private final BoundedSolrSource source; + + private AuthorizedSolrClient solrClient; + private SolrDocument current; + private String cursorMark; + private Iterator batchIterator; + private boolean done; + private String uniqueKey; + + private BoundedSolrReader(BoundedSolrSource source) { + this.source = source; + this.cursorMark = CursorMarkParams.CURSOR_MARK_START; + } + + @Override + public boolean start() throws IOException { + if (source.replica != null) { + solrClient = + source.spec.getConnectionConfiguration().createClient(source.replica.baseUrl()); + } else { + solrClient = source.spec.getConnectionConfiguration().createClient(); + } + SchemaRequest.UniqueKey uniqueKeyRequest = new SchemaRequest.UniqueKey(); + try { + String collection = source.spec.getCollection(); + SchemaResponse.UniqueKeyResponse uniqueKeyResponse = + (SchemaResponse.UniqueKeyResponse) solrClient.process(collection, uniqueKeyRequest); + uniqueKey = uniqueKeyResponse.getUniqueKey(); + } catch (SolrServerException e) { + throw new IOException("Can not get unique key from solr", e); + } + return advance(); + } + + private SolrQuery getQueryParams(BoundedSolrSource source) { + String query = source.spec.getQuery(); + if (query == null) { + query = "*:*"; + } + SolrQuery solrQuery = new SolrQuery(query); + solrQuery.set(CursorMarkParams.CURSOR_MARK_PARAM, cursorMark); + solrQuery.setRows(source.spec.getBatchSize()); + solrQuery.addSort(uniqueKey, SolrQuery.ORDER.asc); + if (source.replica != null) { + solrQuery.setDistrib(false); + } + return solrQuery; + } + + private void updateCursorMark(QueryResponse response) { + if (cursorMark.equals(response.getNextCursorMark())) { + done = true; + } + cursorMark = response.getNextCursorMark(); + } + + @Override + public boolean advance() throws IOException { + if (batchIterator != null && batchIterator.hasNext()) { + current = batchIterator.next(); + return true; + } else { + SolrQuery solrQuery = getQueryParams(source); + try { + QueryResponse response; + if (source.replica != null) { + response = solrClient.query(source.replica.coreName(), solrQuery); + } else { + response = solrClient.query(source.spec.getCollection(), solrQuery); + } + updateCursorMark(response); + return readNextBatchAndReturnFirstDocument(response); + } catch (SolrServerException e) { + throw new IOException(e); + } + } + } + + private boolean readNextBatchAndReturnFirstDocument(QueryResponse response) { + if (done) { + current = null; + batchIterator = null; + return false; + } + + batchIterator = response.getResults().iterator(); + current = batchIterator.next(); + return true; + } + + @Override + public SolrDocument getCurrent() throws NoSuchElementException { + if (current == null) { + throw new NoSuchElementException(); + } + return current; + } + + @Override + public void close() throws IOException { + solrClient.close(); + } + + @Override + public BoundedSource getCurrentSource() { + return source; + } + } + + /** A {@link PTransform} writing data to Solr. */ + @AutoValue + public abstract static class Write extends PTransform, PDone> { + + @Nullable + abstract ConnectionConfiguration getConnectionConfiguration(); + + @Nullable + abstract String getCollection(); + + abstract int getMaxBatchSize(); + + abstract Builder builder(); + + @AutoValue.Builder + abstract static class Builder { + abstract Builder setConnectionConfiguration(ConnectionConfiguration connectionConfiguration); + + abstract Builder setCollection(String collection); + + abstract Builder setMaxBatchSize(int maxBatchSize); + + abstract Write build(); + } + + /** Provide the Solr connection configuration object. */ + public Write withConnectionConfiguration(ConnectionConfiguration connectionConfiguration) { + checkArgument(connectionConfiguration != null, "connectionConfiguration can not be null"); + return builder().setConnectionConfiguration(connectionConfiguration).build(); + } + + /** + * Provide name of collection while reading from Solr. + * + * @param collection the collection toward which the requests will be issued + */ + public Write to(String collection) { + checkArgument(collection != null, "collection can not be null"); + return builder().setCollection(collection).build(); + } + + /** + * Provide a maximum size in number of documents for the batch. Depending on the execution + * engine, size of bundles may vary, this sets the maximum size. Change this if you need to have + * smaller batch. + * + * @param batchSize maximum batch size in number of documents + */ + @VisibleForTesting + Write withMaxBatchSize(int batchSize) { + // TODO remove this configuration, we can figure out the best number + // by tuning batchSize when pipelines run. + checkArgument(batchSize > 0, "batchSize must be larger than 0, but was: %s", batchSize); + return builder().setMaxBatchSize(batchSize).build(); + } + + @Override + public void validate(PipelineOptions options) { + checkState( + getConnectionConfiguration() != null, + "Need to set connection configuration via withConnectionConfiguration()"); + checkState(getCollection() != null, "Need to set collection name via to()"); + } + + @Override + public PDone expand(PCollection input) { + input.apply(ParDo.of(new WriteFn(this))); + return PDone.in(input.getPipeline()); + } + + @VisibleForTesting + static class WriteFn extends DoFn { + + private final Write spec; + + private transient AuthorizedSolrClient solrClient; + private Collection batch; + + WriteFn(Write spec) { + this.spec = spec; + } + + @Setup + public void createClient() throws Exception { + solrClient = spec.getConnectionConfiguration().createClient(); + } + + @StartBundle + public void startBundle(StartBundleContext context) throws Exception { + batch = new ArrayList<>(); + } + + @ProcessElement + public void processElement(ProcessContext context) throws Exception { + SolrInputDocument document = context.element(); + batch.add(document); + if (batch.size() >= spec.getMaxBatchSize()) { + flushBatch(); + } + } + + @FinishBundle + public void finishBundle(FinishBundleContext context) throws Exception { + flushBatch(); + } + + private void flushBatch() throws IOException { + if (batch.isEmpty()) { + return; + } + try { + UpdateRequest updateRequest = new UpdateRequest(); + updateRequest.add(batch); + solrClient.process(spec.getCollection(), updateRequest); + } catch (SolrServerException e) { + throw new IOException("Error writing to Solr", e); + } finally { + batch.clear(); + } + } + + @Teardown + public void closeClient() throws Exception { + if (solrClient != null) { + solrClient.close(); + } + } + } + } +} diff --git a/sdks/java/io/solr/src/main/java/org/apache/beam/sdk/io/solr/package-info.java b/sdks/java/io/solr/src/main/java/org/apache/beam/sdk/io/solr/package-info.java new file mode 100644 index 0000000000000..83867ed0f4430 --- /dev/null +++ b/sdks/java/io/solr/src/main/java/org/apache/beam/sdk/io/solr/package-info.java @@ -0,0 +1,20 @@ +/* + * 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. + */ + +/** Transforms for reading and writing from/to Solr. */ +package org.apache.beam.sdk.io.solr; diff --git a/sdks/java/io/solr/src/test/java/org/apache/beam/sdk/io/solr/JavaBinCodecCoderTest.java b/sdks/java/io/solr/src/test/java/org/apache/beam/sdk/io/solr/JavaBinCodecCoderTest.java new file mode 100644 index 0000000000000..1fb435d1e20a0 --- /dev/null +++ b/sdks/java/io/solr/src/test/java/org/apache/beam/sdk/io/solr/JavaBinCodecCoderTest.java @@ -0,0 +1,81 @@ +/* + * 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.solr; + +import static org.hamcrest.Matchers.equalTo; +import static org.junit.Assert.assertThat; + +import java.util.ArrayList; +import java.util.List; + +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.coders.CoderException; +import org.apache.beam.sdk.testing.CoderProperties; +import org.apache.beam.sdk.util.CoderUtils; +import org.apache.beam.sdk.values.TypeDescriptor; +import org.apache.solr.common.SolrDocument; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.ExpectedException; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +/** Test case for {@link JavaBinCodecCoder}. */ +@RunWith(JUnit4.class) +public class JavaBinCodecCoderTest { + private static final Coder TEST_CODER = JavaBinCodecCoder.of(SolrDocument.class); + private static final List TEST_VALUES = new ArrayList<>(); + + static { + SolrDocument doc = new SolrDocument(); + doc.put("id", "1"); + doc.put("content", "wheel on the bus"); + doc.put("_version_", 1573597324260671488L); + TEST_VALUES.add(doc); + + doc = new SolrDocument(); + doc.put("id", "2"); + doc.put("content", "goes round and round"); + doc.put("_version_", 1573597324260671489L); + TEST_VALUES.add(doc); + } + + @Test + public void testDecodeEncodeEqual() throws Exception { + for (SolrDocument value : TEST_VALUES) { + CoderProperties.coderDecodeEncodeContentsInSameOrder(TEST_CODER, value); + CoderProperties.structuralValueDecodeEncodeEqual(TEST_CODER, value); + } + } + + @Rule public ExpectedException thrown = ExpectedException.none(); + + @Test + public void encodeNullThrowsCoderException() throws Exception { + thrown.expect(CoderException.class); + thrown.expectMessage("cannot encode a null SolrDocument"); + + CoderUtils.encodeToBase64(TEST_CODER, null); + } + + @Test + public void testEncodedTypeDescriptor() throws Exception { + assertThat( + TEST_CODER.getEncodedTypeDescriptor(), equalTo(TypeDescriptor.of(SolrDocument.class))); + } +} diff --git a/sdks/java/io/solr/src/test/java/org/apache/beam/sdk/io/solr/SolrIOTest.java b/sdks/java/io/solr/src/test/java/org/apache/beam/sdk/io/solr/SolrIOTest.java new file mode 100644 index 0000000000000..4358ce41c60fb --- /dev/null +++ b/sdks/java/io/solr/src/test/java/org/apache/beam/sdk/io/solr/SolrIOTest.java @@ -0,0 +1,269 @@ +/* + * 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.solr; + +import static org.apache.beam.sdk.testing.SourceTestUtils.readFromSource; +import static org.hamcrest.Matchers.greaterThan; +import static org.hamcrest.Matchers.lessThan; + +import com.carrotsearch.randomizedtesting.annotations.ThreadLeakScope; +import com.google.common.io.BaseEncoding; + +import java.io.IOException; +import java.nio.charset.Charset; +import java.util.List; + +import org.apache.beam.sdk.io.BoundedSource; +import org.apache.beam.sdk.options.PipelineOptions; +import org.apache.beam.sdk.options.PipelineOptionsFactory; +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.Create; +import org.apache.beam.sdk.transforms.DoFnTester; +import org.apache.beam.sdk.values.PCollection; +import org.apache.solr.SolrTestCaseJ4; +import org.apache.solr.client.solrj.SolrQuery; +import org.apache.solr.client.solrj.impl.CloudSolrClient; +import org.apache.solr.client.solrj.response.QueryResponse; +import org.apache.solr.cloud.SolrCloudTestCase; +import org.apache.solr.common.SolrDocument; +import org.apache.solr.common.SolrException; +import org.apache.solr.common.SolrInputDocument; +import org.apache.solr.common.cloud.ZkStateReader; +import org.apache.solr.security.Sha256AuthenticationProvider; +import org.junit.AfterClass; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.ExpectedException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** A test of {@link SolrIO} on an independent Solr instance. */ +@ThreadLeakScope(value = ThreadLeakScope.Scope.NONE) +@SolrTestCaseJ4.SuppressSSL +public class SolrIOTest extends SolrCloudTestCase { + private static final Logger LOG = LoggerFactory.getLogger(SolrIOTest.class); + + private static final String SOLR_COLLECTION = "beam"; + private static final int NUM_SHARDS = 3; + private static final long NUM_DOCS = 400L; + private static final int NUM_SCIENTISTS = 10; + private static final int BATCH_SIZE = 200; + + private static AuthorizedSolrClient solrClient; + private static SolrIO.ConnectionConfiguration connectionConfiguration; + + @Rule public TestPipeline pipeline = TestPipeline.create(); + + @BeforeClass + public static void beforeClass() throws Exception { + // setup credential for solr user, + // See https://cwiki.apache.org/confluence/display/solr/Basic+Authentication+Plugin + String password = "SolrRocks"; + // salt's size can be arbitrary + byte[] salt = new byte[random().nextInt(30) + 1]; + random().nextBytes(salt); + String base64Salt = BaseEncoding.base64().encode(salt); + String sha56 = Sha256AuthenticationProvider.sha256(password, base64Salt); + String credential = sha56 + " " + base64Salt; + String securityJson = + "{" + + "'authentication':{" + + " 'blockUnknown': true," + + " 'class':'solr.BasicAuthPlugin'," + + " 'credentials':{'solr':'" + + credential + + "'}}" + + "}"; + + configureCluster(3).addConfig("conf", getFile("cloud-minimal/conf").toPath()).configure(); + ZkStateReader zkStateReader = cluster.getSolrClient().getZkStateReader(); + zkStateReader + .getZkClient() + .setData("/security.json", securityJson.getBytes(Charset.defaultCharset()), true); + String zkAddress = cluster.getZkServer().getZkAddress(); + connectionConfiguration = + SolrIO.ConnectionConfiguration.create(zkAddress).withBasicCredentials("solr", password); + solrClient = connectionConfiguration.createClient(); + SolrIOTestUtils.createCollection(SOLR_COLLECTION, NUM_SHARDS, 1, solrClient); + } + + @AfterClass + public static void afterClass() throws Exception { + solrClient.close(); + } + + @Before + public void before() throws Exception { + SolrIOTestUtils.clearCollection(SOLR_COLLECTION, solrClient); + } + + @Rule public ExpectedException thrown = ExpectedException.none(); + + public void testBadCredentials() throws IOException { + thrown.expect(SolrException.class); + + String zkAddress = cluster.getZkServer().getZkAddress(); + SolrIO.ConnectionConfiguration connectionConfiguration = + SolrIO.ConnectionConfiguration.create(zkAddress) + .withBasicCredentials("solr", "wrongpassword"); + try (AuthorizedSolrClient solrClient = connectionConfiguration.createClient()) { + SolrIOTestUtils.insertTestDocuments(SOLR_COLLECTION, NUM_DOCS, solrClient); + } + } + + @Test + public void testSizes() throws Exception { + SolrIOTestUtils.insertTestDocuments(SOLR_COLLECTION, NUM_DOCS, solrClient); + + PipelineOptions options = PipelineOptionsFactory.create(); + SolrIO.Read read = + SolrIO.read().withConnectionConfiguration(connectionConfiguration).from(SOLR_COLLECTION); + SolrIO.BoundedSolrSource initialSource = new SolrIO.BoundedSolrSource(read, null); + // can't use equal assert as Solr collections never have same size + // (due to internal Lucene implementation) + long estimatedSize = initialSource.getEstimatedSizeBytes(options); + LOG.info("Estimated size: {}", estimatedSize); + assertThat( + "Wrong estimated size bellow minimum", + estimatedSize, + greaterThan(SolrIOTestUtils.MIN_DOC_SIZE * NUM_DOCS)); + assertThat( + "Wrong estimated size beyond maximum", + estimatedSize, + lessThan(SolrIOTestUtils.MAX_DOC_SIZE * NUM_DOCS)); + } + + @Test + public void testRead() throws Exception { + SolrIOTestUtils.insertTestDocuments(SOLR_COLLECTION, NUM_DOCS, solrClient); + + PCollection output = + pipeline.apply( + SolrIO.read() + .withConnectionConfiguration(connectionConfiguration) + .from(SOLR_COLLECTION) + .withBatchSize(101)); + PAssert.thatSingleton(output.apply("Count", Count.globally())) + .isEqualTo(NUM_DOCS); + pipeline.run(); + } + + @Test + public void testReadWithQuery() throws Exception { + SolrIOTestUtils.insertTestDocuments(SOLR_COLLECTION, NUM_DOCS, solrClient); + + PCollection output = + pipeline.apply( + SolrIO.read() + .withConnectionConfiguration(connectionConfiguration) + .from(SOLR_COLLECTION) + .withQuery("scientist:Franklin")); + PAssert.thatSingleton(output.apply("Count", Count.globally())) + .isEqualTo(NUM_DOCS / NUM_SCIENTISTS); + pipeline.run(); + } + + @Test + public void testWrite() throws Exception { + List data = SolrIOTestUtils.createDocuments(NUM_DOCS); + SolrIO.Write write = + SolrIO.write().withConnectionConfiguration(connectionConfiguration).to(SOLR_COLLECTION); + pipeline.apply(Create.of(data)).apply(write); + pipeline.run(); + + long currentNumDocs = SolrIOTestUtils.commitAndGetCurrentNumDocs(SOLR_COLLECTION, solrClient); + assertEquals(NUM_DOCS, currentNumDocs); + + QueryResponse response = solrClient.query(SOLR_COLLECTION, new SolrQuery("scientist:Lovelace")); + assertEquals(NUM_DOCS / NUM_SCIENTISTS, response.getResults().getNumFound()); + } + + @Test + public void testWriteWithMaxBatchSize() throws Exception { + SolrIO.Write write = + SolrIO.write() + .withConnectionConfiguration(connectionConfiguration) + .to(SOLR_COLLECTION) + .withMaxBatchSize(BATCH_SIZE); + // write bundles size is the runner decision, we cannot force a bundle size, + // so we test the Writer as a DoFn outside of a runner. + try (DoFnTester fnTester = + DoFnTester.of(new SolrIO.Write.WriteFn(write))) { + List input = SolrIOTestUtils.createDocuments(NUM_DOCS); + long numDocsProcessed = 0; + long numDocsInserted = 0; + for (SolrInputDocument document : input) { + fnTester.processElement(document); + numDocsProcessed++; + // test every 100 docs to avoid overloading Solr + if ((numDocsProcessed % 100) == 0) { + // force the index to upgrade after inserting for the inserted docs + // to be searchable immediately + long currentNumDocs = + SolrIOTestUtils.commitAndGetCurrentNumDocs(SOLR_COLLECTION, solrClient); + if ((numDocsProcessed % BATCH_SIZE) == 0) { + /* bundle end */ + assertEquals( + "we are at the end of a bundle, we should have inserted all processed documents", + numDocsProcessed, + currentNumDocs); + numDocsInserted = currentNumDocs; + } else { + /* not bundle end */ + assertEquals( + "we are not at the end of a bundle, we should have inserted no more documents", + numDocsInserted, + currentNumDocs); + } + } + } + } + } + + @Test + public void testSplit() throws Exception { + SolrIOTestUtils.insertTestDocuments(SOLR_COLLECTION, NUM_DOCS, solrClient); + + PipelineOptions options = PipelineOptionsFactory.create(); + SolrIO.Read read = + SolrIO.read().withConnectionConfiguration(connectionConfiguration).from(SOLR_COLLECTION); + SolrIO.BoundedSolrSource initialSource = new SolrIO.BoundedSolrSource(read, null); + //desiredBundleSize is ignored for now + int desiredBundleSizeBytes = 0; + List> splits = + initialSource.split(desiredBundleSizeBytes, options); + SourceTestUtils.assertSourcesEqualReferenceSource(initialSource, splits, options); + + int expectedNumSplits = NUM_SHARDS; + assertEquals(expectedNumSplits, splits.size()); + int nonEmptySplits = 0; + for (BoundedSource subSource : splits) { + if (readFromSource(subSource, options).size() > 0) { + nonEmptySplits += 1; + } + } + // docs are hashed by id to shards, in this test, NUM_DOCS >> NUM_SHARDS + // therefore, can not exist an empty shard. + assertEquals("Wrong number of empty splits", expectedNumSplits, nonEmptySplits); + } +} diff --git a/sdks/java/io/solr/src/test/java/org/apache/beam/sdk/io/solr/SolrIOTestUtils.java b/sdks/java/io/solr/src/test/java/org/apache/beam/sdk/io/solr/SolrIOTestUtils.java new file mode 100644 index 0000000000000..808cd0f4bb29e --- /dev/null +++ b/sdks/java/io/solr/src/test/java/org/apache/beam/sdk/io/solr/SolrIOTestUtils.java @@ -0,0 +1,132 @@ +/* + * 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.solr; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import org.apache.solr.client.solrj.SolrQuery; +import org.apache.solr.client.solrj.SolrServerException; +import org.apache.solr.client.solrj.request.CollectionAdminRequest; +import org.apache.solr.client.solrj.request.UpdateRequest; +import org.apache.solr.common.SolrInputDocument; + +/** Test utilities to use with {@link SolrIO}. */ +public class SolrIOTestUtils { + public static final long MIN_DOC_SIZE = 40L; + public static final long MAX_DOC_SIZE = 90L; + + static void createCollection( + String collection, int numShards, int replicationFactor, AuthorizedSolrClient client) + throws Exception { + CollectionAdminRequest.Create create = + new CollectionAdminRequest.Create() + .setCollectionName(collection) + .setNumShards(numShards) + .setReplicationFactor(replicationFactor) + .setMaxShardsPerNode(2); + client.process(create); + } + + /** Inserts the given number of test documents into Solr. */ + static void insertTestDocuments(String collection, long numDocs, AuthorizedSolrClient client) + throws IOException { + List data = createDocuments(numDocs); + try { + UpdateRequest updateRequest = new UpdateRequest(); + updateRequest.setAction(UpdateRequest.ACTION.COMMIT, true, true); + updateRequest.add(data); + client.process(collection, updateRequest); + } catch (SolrServerException e) { + throw new IOException("Failed to insert test documents to collection", e); + } + } + + /** Delete given collection. */ + static void deleteCollection(String collection, AuthorizedSolrClient client) throws IOException { + try { + CollectionAdminRequest.Delete delete = + new CollectionAdminRequest.Delete().setCollectionName(collection); + client.process(delete); + } catch (SolrServerException e) { + throw new IOException(e); + } + } + + /** Clear given collection. */ + static void clearCollection(String collection, AuthorizedSolrClient client) throws IOException { + try { + UpdateRequest updateRequest = new UpdateRequest(); + updateRequest.setAction(UpdateRequest.ACTION.COMMIT, true, true); + updateRequest.deleteByQuery("*:*"); + client.process(collection, updateRequest); + } catch (SolrServerException e) { + throw new IOException(e); + } + } + + /** + * Forces a commit of the given collection to make recently inserted documents available for + * search. + * + * @return The number of docs in the index + */ + static long commitAndGetCurrentNumDocs(String collection, AuthorizedSolrClient client) + throws IOException { + SolrQuery solrQuery = new SolrQuery("*:*"); + solrQuery.setRows(0); + try { + UpdateRequest update = new UpdateRequest(); + update.setAction(UpdateRequest.ACTION.COMMIT, true, true); + client.process(collection, update); + + return client.query(collection, new SolrQuery("*:*")).getResults().getNumFound(); + } catch (SolrServerException e) { + throw new IOException(e); + } + } + + /** + * Generates a list of test documents for insertion. + * + * @return the list of json String representing the documents + */ + static List createDocuments(long numDocs) { + String[] scientists = { + "Lovelace", + "Franklin", + "Meitner", + "Hopper", + "Curie", + "Faraday", + "Newton", + "Bohr", + "Galilei", + "Maxwell" + }; + ArrayList data = new ArrayList<>(); + for (int i = 0; i < numDocs; i++) { + int index = i % scientists.length; + SolrInputDocument doc = new SolrInputDocument(); + doc.setField("id", String.valueOf(i)); + doc.setField("scientist", scientists[index]); + data.add(doc); + } + return data; + } +} diff --git a/sdks/java/io/solr/src/test/resources/cloud-minimal/conf/schema.xml b/sdks/java/io/solr/src/test/resources/cloud-minimal/conf/schema.xml new file mode 100644 index 0000000000000..08a1716deb147 --- /dev/null +++ b/sdks/java/io/solr/src/test/resources/cloud-minimal/conf/schema.xml @@ -0,0 +1,29 @@ + + + + + + + + + + + + + id + diff --git a/sdks/java/io/solr/src/test/resources/cloud-minimal/conf/solrconfig.xml b/sdks/java/io/solr/src/test/resources/cloud-minimal/conf/solrconfig.xml new file mode 100644 index 0000000000000..8da7d2847e9bc --- /dev/null +++ b/sdks/java/io/solr/src/test/resources/cloud-minimal/conf/solrconfig.xml @@ -0,0 +1,48 @@ + + + + + + + + + ${solr.data.dir:} + + + + + ${tests.luceneMatchVersion:LATEST} + + + + ${solr.commitwithin.softcommit:true} + + + + + + + explicit + true + text + + + + + diff --git a/sdks/java/javadoc/pom.xml b/sdks/java/javadoc/pom.xml index e1adb7915ea8e..1fb6e410818ca 100644 --- a/sdks/java/javadoc/pom.xml +++ b/sdks/java/javadoc/pom.xml @@ -172,6 +172,11 @@ beam-sdks-java-io-mqtt + + org.apache.beam + beam-sdks-java-io-solr + + com.google.auto.service From 2b9b05049822a22154ac3c2f6b593061f42b54c1 Mon Sep 17 00:00:00 2001 From: Thomas Groh Date: Mon, 5 Jun 2017 11:22:56 -0700 Subject: [PATCH 255/346] Use bytes instead of Any in RunnerApi.FunctionSpec Keep a "any" field, renamed to any_param. Rename `parameter` to `payload` --- .../core/construction/CoderTranslation.java | 16 +-- .../core/construction/CombineTranslation.java | 28 ++---- .../CreatePCollectionViewTranslation.java | 17 +--- .../construction/PTransformTranslation.java | 12 +-- .../core/construction/ParDoTranslation.java | 58 ++++------- .../construction/PipelineTranslation.java | 12 +-- .../core/construction/ReadTranslation.java | 56 +++++------ .../construction/TestStreamTranslation.java | 5 +- .../construction/WindowIntoTranslation.java | 9 +- .../WindowingStrategyTranslation.java | 97 ++++++++----------- .../construction/WriteFilesTranslation.java | 29 +++--- .../CreatePCollectionViewTranslationTest.java | 6 +- .../construction/ParDoTranslationTest.java | 3 +- .../TestStreamTranslationTest.java | 4 +- .../src/main/proto/beam_runner_api.proto | 5 +- .../beam/fn/harness/BeamFnDataReadRunner.java | 4 +- .../fn/harness/BeamFnDataWriteRunner.java | 4 +- .../beam/fn/harness/BoundedSourceRunner.java | 10 +- .../beam/fn/harness/FnApiDoFnRunner.java | 10 +- .../fn/harness/BeamFnDataReadRunnerTest.java | 5 +- .../fn/harness/BeamFnDataWriteRunnerTest.java | 5 +- .../fn/harness/BoundedSourceRunnerTest.java | 18 ++-- .../beam/fn/harness/FnApiDoFnRunnerTest.java | 13 +-- sdks/python/apache_beam/coders/coders.py | 11 +-- .../runners/portability/fn_api_runner.py | 78 +++++++++------ .../runners/worker/bundle_processor.py | 21 ++-- sdks/python/apache_beam/transforms/core.py | 10 +- .../apache_beam/transforms/ptransform.py | 6 +- sdks/python/apache_beam/utils/proto_utils.py | 11 +++ sdks/python/apache_beam/utils/urns.py | 6 +- 30 files changed, 250 insertions(+), 319 deletions(-) diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/CoderTranslation.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/CoderTranslation.java index a6719ff984234..2246f814daf8e 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/CoderTranslation.java +++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/CoderTranslation.java @@ -24,9 +24,7 @@ import com.google.common.collect.BiMap; import com.google.common.collect.ImmutableBiMap; import com.google.common.collect.ImmutableMap; -import com.google.protobuf.Any; import com.google.protobuf.ByteString; -import com.google.protobuf.BytesValue; import java.io.IOException; import java.util.ArrayList; import java.util.LinkedList; @@ -138,13 +136,9 @@ private static RunnerApi.Coder toCustomCoder(Coder coder) throws IOException .setSpec( FunctionSpec.newBuilder() .setUrn(JAVA_SERIALIZED_CODER_URN) - .setParameter( - Any.pack( - BytesValue.newBuilder() - .setValue( - ByteString.copyFrom( - SerializableUtils.serializeToByteArray(coder))) - .build())))) + .setPayload( + ByteString.copyFrom(SerializableUtils.serializeToByteArray(coder))) + .build())) .build(); } @@ -182,9 +176,7 @@ private static Coder fromCustomCoder(RunnerApi.Coder protoCoder) throws IOExc protoCoder .getSpec() .getSpec() - .getParameter() - .unpack(BytesValue.class) - .getValue() + .getPayload() .toByteArray(), "Custom Coder Bytes"); } diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/CombineTranslation.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/CombineTranslation.java index d909ccfb59556..17c48dc540470 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/CombineTranslation.java +++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/CombineTranslation.java @@ -23,9 +23,7 @@ import com.google.auto.service.AutoService; import com.google.common.collect.Iterables; -import com.google.protobuf.Any; import com.google.protobuf.ByteString; -import com.google.protobuf.BytesValue; import java.io.IOException; import java.util.Collections; import java.util.HashMap; @@ -79,7 +77,7 @@ public FunctionSpec translate( CombinePayload payload = toProto(transform, components); return RunnerApi.FunctionSpec.newBuilder() .setUrn(COMBINE_TRANSFORM_URN) - .setParameter(Any.pack(payload)) + .setPayload(payload.toByteString()) .build(); } @@ -138,13 +136,9 @@ public static SdkFunctionSpec toProto(GlobalCombineFn combineFn) { .setSpec( FunctionSpec.newBuilder() .setUrn(JAVA_SERIALIZED_COMBINE_FN_URN) - .setParameter( - Any.pack( - BytesValue.newBuilder() - .setValue( - ByteString.copyFrom( - SerializableUtils.serializeToByteArray(combineFn))) - .build()))) + .setPayload( + ByteString.copyFrom(SerializableUtils.serializeToByteArray(combineFn))) + .build()) .build(); } @@ -171,9 +165,7 @@ public static Coder getAccumulatorCoder( payload .getCombineFn() .getSpec() - .getParameter() - .unpack(BytesValue.class) - .getValue() + .getPayload() .toByteArray(), "CombineFn"); } @@ -190,10 +182,10 @@ private static CombinePayload getCombinePayload(AppliedPTransform trans private static CombinePayload getCombinePayload( AppliedPTransform transform, SdkComponents components) throws IOException { - return PTransformTranslation.toProto( - transform, Collections.>emptyList(), components) - .getSpec() - .getParameter() - .unpack(CombinePayload.class); + return CombinePayload.parseFrom( + PTransformTranslation.toProto( + transform, Collections.>emptyList(), components) + .getSpec() + .getPayload()); } } diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/CreatePCollectionViewTranslation.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/CreatePCollectionViewTranslation.java index c67d688d932de..1027ea2476bf4 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/CreatePCollectionViewTranslation.java +++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/CreatePCollectionViewTranslation.java @@ -21,9 +21,7 @@ import static com.google.common.base.Preconditions.checkArgument; import com.google.auto.service.AutoService; -import com.google.protobuf.Any; import com.google.protobuf.ByteString; -import com.google.protobuf.BytesValue; import java.io.IOException; import java.util.Collections; import java.util.Map; @@ -79,9 +77,7 @@ public static PCollectionView getView( SerializableUtils.deserializeFromByteArray( transformProto .getSpec() - .getParameter() - .unpack(BytesValue.class) - .getValue() + .getPayload() .toByteArray(), PCollectionView.class.getSimpleName()); } @@ -104,14 +100,9 @@ public FunctionSpec translate( SdkComponents components) { return FunctionSpec.newBuilder() .setUrn(getUrn(transform.getTransform())) - .setParameter( - Any.pack( - BytesValue.newBuilder() - .setValue( - ByteString.copyFrom( - SerializableUtils.serializeToByteArray( - transform.getTransform().getView()))) - .build())) + .setPayload( + ByteString.copyFrom( + SerializableUtils.serializeToByteArray(transform.getTransform().getView()))) .build(); } } diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/PTransformTranslation.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/PTransformTranslation.java index b8365c9415107..4bfe17ae6c079 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/PTransformTranslation.java +++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/PTransformTranslation.java @@ -21,7 +21,7 @@ import static com.google.common.base.Preconditions.checkArgument; import com.google.common.collect.ImmutableMap; -import com.google.protobuf.Any; +import com.google.protobuf.ByteString; import java.io.IOException; import java.util.Collections; import java.util.List; @@ -131,9 +131,9 @@ static RunnerApi.PTransform toProto( if (rawPTransform.getUrn() != null) { FunctionSpec.Builder payload = FunctionSpec.newBuilder().setUrn(rawPTransform.getUrn()); - @Nullable Any parameter = rawPTransform.getPayload(); + @Nullable ByteString parameter = rawPTransform.getPayload(); if (parameter != null) { - payload.setParameter(parameter); + payload.setPayload(parameter); } transformBuilder.setSpec(payload); } @@ -224,7 +224,7 @@ public abstract static class RawPTransform< public abstract String getUrn(); @Nullable - public Any getPayload() { + public ByteString getPayload() { return null; } @@ -254,9 +254,9 @@ public FunctionSpec translate( FunctionSpec.Builder transformSpec = FunctionSpec.newBuilder().setUrn(getUrn(transform.getTransform())); - Any payload = transform.getTransform().getPayload(); + ByteString payload = transform.getTransform().getPayload(); if (payload != null) { - transformSpec.setParameter(payload); + transformSpec.setPayload(payload); } // Transforms like Combine may have Coders that need to be added but do not diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ParDoTranslation.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ParDoTranslation.java index 5765c51b26fbe..6ae95e44afd60 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ParDoTranslation.java +++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ParDoTranslation.java @@ -29,9 +29,7 @@ import com.google.common.base.Optional; import com.google.common.collect.Iterables; import com.google.common.collect.Sets; -import com.google.protobuf.Any; import com.google.protobuf.ByteString; -import com.google.protobuf.BytesValue; import com.google.protobuf.InvalidProtocolBufferException; import java.io.IOException; import java.io.Serializable; @@ -122,7 +120,7 @@ public FunctionSpec translate( ParDoPayload payload = toProto(transform.getTransform(), components); return RunnerApi.FunctionSpec.newBuilder() .setUrn(PAR_DO_TRANSFORM_URN) - .setParameter(Any.pack(payload)) + .setPayload(payload.toByteString()) .build(); } @@ -240,7 +238,7 @@ public static TupleTagList getAdditionalOutputTags(AppliedPTransform ap RunnerApi.PTransform protoTransform = PTransformTranslation.toProto(application, SdkComponents.create()); - ParDoPayload payload = protoTransform.getSpec().getParameter().unpack(ParDoPayload.class); + ParDoPayload payload = ParDoPayload.parseFrom(protoTransform.getSpec().getPayload()); TupleTag mainOutputTag = getMainOutputTag(payload); Set outputTags = Sets.difference( @@ -259,7 +257,7 @@ public static List> getSideInputs(AppliedPTransform SdkComponents sdkComponents = SdkComponents.create(); RunnerApi.PTransform parDoProto = PTransformTranslation.toProto(application, sdkComponents); - ParDoPayload payload = parDoProto.getSpec().getParameter().unpack(ParDoPayload.class); + ParDoPayload payload = ParDoPayload.parseFrom(parDoProto.getSpec().getPayload()); List> views = new ArrayList<>(); RehydratedComponents components = @@ -289,7 +287,7 @@ public static RunnerApi.PCollection getMainInput( ptransform.getSpec().getUrn().equals(PAR_DO_TRANSFORM_URN), "Unexpected payload type %s", ptransform.getSpec().getUrn()); - ParDoPayload payload = ptransform.getSpec().getParameter().unpack(ParDoPayload.class); + ParDoPayload payload = ParDoPayload.parseFrom(ptransform.getSpec().getPayload()); String mainInputId = Iterables.getOnlyElement( Sets.difference( @@ -377,7 +375,7 @@ static StateSpec fromProto(RunnerApi.StateSpec stateSpec, RehydratedComponent Combine.CombineFn combineFn = (Combine.CombineFn) SerializableUtils.deserializeFromByteArray( - combineFnSpec.getParameter().unpack(BytesValue.class).toByteArray(), + combineFnSpec.getPayload().toByteArray(), Combine.CombineFn.class.getSimpleName()); // Rawtype coder cast because it is required to be a valid accumulator coder @@ -443,14 +441,10 @@ private static SdkFunctionSpec toProto(DoFn fn, TupleTag tag) { .setSpec( FunctionSpec.newBuilder() .setUrn(CUSTOM_JAVA_DO_FN_URN) - .setParameter( - Any.pack( - BytesValue.newBuilder() - .setValue( - ByteString.copyFrom( - SerializableUtils.serializeToByteArray( - DoFnAndMainOutput.of(fn, tag)))) - .build()))) + .setPayload( + ByteString.copyFrom( + SerializableUtils.serializeToByteArray(DoFnAndMainOutput.of(fn, tag)))) + .build()) .build(); } @@ -458,7 +452,7 @@ private static DoFnAndMainOutput doFnAndMainOutputTagFromProto(SdkFunctionSpec f throws InvalidProtocolBufferException { checkArgument(fnSpec.getSpec().getUrn().equals(CUSTOM_JAVA_DO_FN_URN)); byte[] serializedFn = - fnSpec.getSpec().getParameter().unpack(BytesValue.class).getValue().toByteArray(); + fnSpec.getSpec().getPayload().toByteArray(); return (DoFnAndMainOutput) SerializableUtils.deserializeFromByteArray(serializedFn, "Custom DoFn And Main Output tag"); } @@ -542,22 +536,17 @@ private static SdkFunctionSpec toProto(ViewFn viewFn) { .setSpec( FunctionSpec.newBuilder() .setUrn(CUSTOM_JAVA_VIEW_FN_URN) - .setParameter( - Any.pack( - BytesValue.newBuilder() - .setValue( - ByteString.copyFrom(SerializableUtils.serializeToByteArray(viewFn))) - .build()))) + .setPayload(ByteString.copyFrom(SerializableUtils.serializeToByteArray(viewFn))) + .build()) .build(); } private static ParDoPayload getParDoPayload(AppliedPTransform transform) throws IOException { - return PTransformTranslation.toProto( - transform, Collections.>emptyList(), SdkComponents.create()) - .getSpec() - .getParameter() - .unpack(ParDoPayload.class); + RunnerApi.PTransform parDoPTransform = + PTransformTranslation.toProto( + transform, Collections.>emptyList(), SdkComponents.create()); + return ParDoPayload.parseFrom(parDoPTransform.getSpec().getPayload()); } public static boolean usesStateOrTimers(AppliedPTransform transform) throws IOException { @@ -580,7 +569,7 @@ public static boolean isSplittable(AppliedPTransform transform) throws spec.getUrn()); return (ViewFn) SerializableUtils.deserializeFromByteArray( - spec.getParameter().unpack(BytesValue.class).getValue().toByteArray(), "Custom ViewFn"); + spec.getPayload().toByteArray(), "Custom ViewFn"); } private static SdkFunctionSpec toProto(WindowMappingFn windowMappingFn) { @@ -588,13 +577,9 @@ private static SdkFunctionSpec toProto(WindowMappingFn windowMappingFn) { .setSpec( FunctionSpec.newBuilder() .setUrn(CUSTOM_JAVA_WINDOW_MAPPING_FN_URN) - .setParameter( - Any.pack( - BytesValue.newBuilder() - .setValue( - ByteString.copyFrom( - SerializableUtils.serializeToByteArray(windowMappingFn))) - .build()))) + .setPayload( + ByteString.copyFrom(SerializableUtils.serializeToByteArray(windowMappingFn))) + .build()) .build(); } @@ -608,7 +593,6 @@ private static WindowMappingFn windowMappingFnFromProto(SdkFunctionSpec windo spec.getUrn()); return (WindowMappingFn) SerializableUtils.deserializeFromByteArray( - spec.getParameter().unpack(BytesValue.class).getValue().toByteArray(), - "Custom WinodwMappingFn"); + spec.getPayload().toByteArray(), "Custom WinodwMappingFn"); } } diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/PipelineTranslation.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/PipelineTranslation.java index 9e4839ac9bbfd..d928338f102d0 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/PipelineTranslation.java +++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/PipelineTranslation.java @@ -24,7 +24,7 @@ import com.google.common.base.MoreObjects; import com.google.common.collect.ArrayListMultimap; import com.google.common.collect.ListMultimap; -import com.google.protobuf.Any; +import com.google.protobuf.ByteString; import java.io.IOException; import java.util.ArrayList; import java.util.Collection; @@ -158,7 +158,7 @@ private static void addRehydratedTransform( // TODO: ParDoTranslator should own it - https://issues.apache.org/jira/browse/BEAM-2674 if (transformSpec.getUrn().equals(PTransformTranslation.PAR_DO_TRANSFORM_URN)) { RunnerApi.ParDoPayload payload = - transformSpec.getParameter().unpack(RunnerApi.ParDoPayload.class); + RunnerApi.ParDoPayload.parseFrom(transformSpec.getPayload()); List> views = new ArrayList<>(); for (Map.Entry sideInputEntry : @@ -182,7 +182,7 @@ private static void addRehydratedTransform( List> additionalCoders = Collections.emptyList(); if (transformSpec.getUrn().equals(PTransformTranslation.COMBINE_TRANSFORM_URN)) { RunnerApi.CombinePayload payload = - transformSpec.getParameter().unpack(RunnerApi.CombinePayload.class); + RunnerApi.CombinePayload.parseFrom(transformSpec.getPayload()); additionalCoders = (List) Collections.singletonList( @@ -192,7 +192,7 @@ private static void addRehydratedTransform( RehydratedPTransform transform = RehydratedPTransform.of( transformSpec.getUrn(), - transformSpec.getParameter(), + transformSpec.getPayload(), additionalInputs, additionalCoders); @@ -233,7 +233,7 @@ abstract static class RehydratedPTransform extends RawPTransform, PValue> getAdditionalInputs(); @@ -242,7 +242,7 @@ abstract static class RehydratedPTransform extends RawPTransform, PValue> additionalInputs, List> additionalCoders) { return new AutoValue_PipelineTranslation_RehydratedPTransform( diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ReadTranslation.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ReadTranslation.java index 572384bdd5492..06d1074aa556b 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ReadTranslation.java +++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/ReadTranslation.java @@ -22,9 +22,7 @@ import com.google.auto.service.AutoService; import com.google.common.collect.ImmutableMap; -import com.google.protobuf.Any; import com.google.protobuf.ByteString; -import com.google.protobuf.BytesValue; import com.google.protobuf.InvalidProtocolBufferException; import java.io.IOException; import java.util.Collections; @@ -83,12 +81,8 @@ private static SdkFunctionSpec toProto(BoundedSource source) { .setSpec( FunctionSpec.newBuilder() .setUrn(JAVA_SERIALIZED_BOUNDED_SOURCE) - .setParameter( - Any.pack( - BytesValue.newBuilder() - .setValue( - ByteString.copyFrom(SerializableUtils.serializeToByteArray(source))) - .build()))) + .setPayload(ByteString.copyFrom(SerializableUtils.serializeToByteArray(source))) + .build()) .build(); } @@ -99,9 +93,7 @@ public static BoundedSource boundedSourceFromProto(ReadPayload payload) payload .getSource() .getSpec() - .getParameter() - .unpack(BytesValue.class) - .getValue() + .getPayload() .toByteArray(), "BoundedSource"); } @@ -122,11 +114,13 @@ UnboundedSource unboundedSourceFromTransform( private static ReadPayload getReadPayload( AppliedPTransform, PTransform>> transform) throws IOException { - return PTransformTranslation.toProto( - transform, Collections.>emptyList(), SdkComponents.create()) - .getSpec() - .getParameter() - .unpack(ReadPayload.class); + return ReadPayload.parseFrom( + PTransformTranslation.toProto( + transform, + Collections.>emptyList(), + SdkComponents.create()) + .getSpec() + .getPayload()); } private static SdkFunctionSpec toProto(UnboundedSource source) { @@ -134,12 +128,8 @@ private static SdkFunctionSpec toProto(UnboundedSource source) { .setSpec( FunctionSpec.newBuilder() .setUrn(JAVA_SERIALIZED_UNBOUNDED_SOURCE) - .setParameter( - Any.pack( - BytesValue.newBuilder() - .setValue( - ByteString.copyFrom(SerializableUtils.serializeToByteArray(source))) - .build()))) + .setPayload(ByteString.copyFrom(SerializableUtils.serializeToByteArray(source))) + .build()) .build(); } @@ -150,9 +140,7 @@ private static SdkFunctionSpec toProto(UnboundedSource source) { payload .getSource() .getSpec() - .getParameter() - .unpack(BytesValue.class) - .getValue() + .getPayload() .toByteArray(), "BoundedSource"); } @@ -160,13 +148,13 @@ private static SdkFunctionSpec toProto(UnboundedSource source) { public static PCollection.IsBounded sourceIsBounded(AppliedPTransform transform) { try { return PCollectionTranslation.fromProto( - PTransformTranslation.toProto( - transform, - Collections.>emptyList(), - SdkComponents.create()) - .getSpec() - .getParameter() - .unpack(ReadPayload.class) + ReadPayload.parseFrom( + PTransformTranslation.toProto( + transform, + Collections.>emptyList(), + SdkComponents.create()) + .getSpec() + .getPayload()) .getIsBounded()); } catch (IOException e) { throw new RuntimeException("Internal error determining boundedness of Read", e); @@ -195,7 +183,7 @@ public FunctionSpec translate( ReadPayload payload = toProto(transform.getTransform()); return RunnerApi.FunctionSpec.newBuilder() .setUrn(getUrn(transform.getTransform())) - .setParameter(Any.pack(payload)) + .setPayload(payload.toByteString()) .build(); } } @@ -222,7 +210,7 @@ public FunctionSpec translate( ReadPayload payload = toProto(transform.getTransform()); return RunnerApi.FunctionSpec.newBuilder() .setUrn(getUrn(transform.getTransform())) - .setParameter(Any.pack(payload)) + .setPayload(payload.toByteString()) .build(); } } diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/TestStreamTranslation.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/TestStreamTranslation.java index f23b2eca18c4a..cac7cdc2ff55a 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/TestStreamTranslation.java +++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/TestStreamTranslation.java @@ -22,7 +22,6 @@ import static org.apache.beam.runners.core.construction.PTransformTranslation.TEST_STREAM_TRANSFORM_URN; import com.google.auto.service.AutoService; -import com.google.protobuf.Any; import com.google.protobuf.ByteString; import java.io.IOException; import java.util.ArrayList; @@ -96,7 +95,7 @@ public static TestStream getTestStream( TestStream.class.getSimpleName(), transformProto.getSpec().getUrn()); RunnerApi.TestStreamPayload testStreamPayload = - transformProto.getSpec().getParameter().unpack(RunnerApi.TestStreamPayload.class); + RunnerApi.TestStreamPayload.parseFrom(transformProto.getSpec().getPayload()); return (TestStream) fromProto( @@ -185,7 +184,7 @@ public RunnerApi.FunctionSpec translate( throws IOException { return RunnerApi.FunctionSpec.newBuilder() .setUrn(getUrn(transform.getTransform())) - .setParameter(Any.pack(testStreamToPayload(transform.getTransform(), components))) + .setPayload(testStreamToPayload(transform.getTransform(), components).toByteString()) .build(); } } diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/WindowIntoTranslation.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/WindowIntoTranslation.java index 6aec9082ad6ac..94ef22d7324a8 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/WindowIntoTranslation.java +++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/WindowIntoTranslation.java @@ -21,7 +21,6 @@ import static com.google.common.base.Preconditions.checkArgument; import com.google.auto.service.AutoService; -import com.google.protobuf.Any; import com.google.protobuf.InvalidProtocolBufferException; import java.io.IOException; import java.util.Collections; @@ -54,8 +53,8 @@ public FunctionSpec translate( AppliedPTransform> transform, SdkComponents components) { return FunctionSpec.newBuilder() .setUrn("urn:beam:transform:window:v1") - .setParameter( - Any.pack(WindowIntoTranslation.toProto(transform.getTransform(), components))) + .setPayload( + WindowIntoTranslation.toProto(transform.getTransform(), components).toByteString()) .build(); } } @@ -88,7 +87,7 @@ public static WindowIntoPayload getWindowIntoPayload(AppliedPTransform WindowIntoPayload windowIntoPayload; try { - return transformProto.getSpec().getParameter().unpack(WindowIntoPayload.class); + return WindowIntoPayload.parseFrom(transformProto.getSpec().getPayload()); } catch (InvalidProtocolBufferException exc) { throw new IllegalStateException( String.format( @@ -128,7 +127,7 @@ public FunctionSpec translate( WindowIntoPayload payload = toProto(transform.getTransform(), components); return RunnerApi.FunctionSpec.newBuilder() .setUrn(getUrn(transform.getTransform())) - .setParameter(Any.pack(payload)) + .setPayload(payload.toByteString()) .build(); } } diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/WindowingStrategyTranslation.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/WindowingStrategyTranslation.java index 565b5529c5348..ab50ea25b2417 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/WindowingStrategyTranslation.java +++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/WindowingStrategyTranslation.java @@ -31,6 +31,9 @@ import org.apache.beam.sdk.common.runner.v1.RunnerApi.OutputTime; import org.apache.beam.sdk.common.runner.v1.RunnerApi.SdkFunctionSpec; import org.apache.beam.sdk.common.runner.v1.StandardWindowFns; +import org.apache.beam.sdk.common.runner.v1.StandardWindowFns.FixedWindowsPayload; +import org.apache.beam.sdk.common.runner.v1.StandardWindowFns.SessionsPayload; +import org.apache.beam.sdk.common.runner.v1.StandardWindowFns.SlidingWindowsPayload; import org.apache.beam.sdk.transforms.windowing.FixedWindows; import org.apache.beam.sdk.transforms.windowing.GlobalWindows; import org.apache.beam.sdk.transforms.windowing.Sessions; @@ -199,77 +202,65 @@ public static TimestampCombiner timestampCombinerFromProto(RunnerApi.OutputTime public static SdkFunctionSpec toProto( WindowFn windowFn, @SuppressWarnings("unused") SdkComponents components) { // TODO: Set environment IDs + ByteString serializedFn = ByteString.copyFrom(SerializableUtils.serializeToByteArray(windowFn)); if (USE_OLD_SERIALIZED_JAVA_WINDOWFN_URN) { return SdkFunctionSpec.newBuilder() .setSpec( FunctionSpec.newBuilder() .setUrn(OLD_SERIALIZED_JAVA_WINDOWFN_URN) - .setParameter( - Any.pack( - BytesValue.newBuilder() - .setValue( - ByteString.copyFrom( - SerializableUtils.serializeToByteArray(windowFn))) - .build()))) + .setAnyParam(Any.pack(BytesValue.newBuilder().setValue(serializedFn).build())) + .setPayload(serializedFn) + .build()) .build(); } else if (windowFn instanceof GlobalWindows) { return SdkFunctionSpec.newBuilder() .setSpec(FunctionSpec.newBuilder().setUrn(GLOBAL_WINDOWS_FN)) .build(); } else if (windowFn instanceof FixedWindows) { + FixedWindowsPayload fixedWindowsPayload = + FixedWindowsPayload.newBuilder() + .setSize(Durations.fromMillis(((FixedWindows) windowFn).getSize().getMillis())) + .setOffset(Timestamps.fromMillis(((FixedWindows) windowFn).getOffset().getMillis())) + .build(); return SdkFunctionSpec.newBuilder() .setSpec( FunctionSpec.newBuilder() .setUrn(FIXED_WINDOWS_FN) - .setParameter( - Any.pack( - StandardWindowFns.FixedWindowsPayload.newBuilder() - .setSize(Durations.fromMillis( - ((FixedWindows) windowFn).getSize().getMillis())) - .setOffset(Timestamps.fromMillis( - ((FixedWindows) windowFn).getOffset().getMillis())) - .build()))) + .setAnyParam(Any.pack(fixedWindowsPayload)) + .setPayload(fixedWindowsPayload.toByteString())) .build(); } else if (windowFn instanceof SlidingWindows) { + SlidingWindowsPayload slidingWindowsPayload = SlidingWindowsPayload.newBuilder() + .setSize(Durations.fromMillis(((SlidingWindows) windowFn).getSize().getMillis())) + .setOffset(Timestamps.fromMillis(((SlidingWindows) windowFn).getOffset().getMillis())) + .setPeriod(Durations.fromMillis(((SlidingWindows) windowFn).getPeriod().getMillis())) + .build(); return SdkFunctionSpec.newBuilder() .setSpec( FunctionSpec.newBuilder() .setUrn(SLIDING_WINDOWS_FN) - .setParameter( - Any.pack( - StandardWindowFns.SlidingWindowsPayload.newBuilder() - .setSize(Durations.fromMillis( - ((SlidingWindows) windowFn).getSize().getMillis())) - .setOffset(Timestamps.fromMillis( - ((SlidingWindows) windowFn).getOffset().getMillis())) - .setPeriod(Durations.fromMillis( - ((SlidingWindows) windowFn).getPeriod().getMillis())) - .build()))) + .setAnyParam(Any.pack(slidingWindowsPayload)) + .setPayload(slidingWindowsPayload.toByteString())) .build(); } else if (windowFn instanceof Sessions) { + SessionsPayload sessionsPayload = + SessionsPayload.newBuilder() + .setGapSize(Durations.fromMillis(((Sessions) windowFn).getGapDuration().getMillis())) + .build(); return SdkFunctionSpec.newBuilder() .setSpec( FunctionSpec.newBuilder() .setUrn(SESSION_WINDOWS_FN) - .setParameter( - Any.pack( - StandardWindowFns.SessionsPayload.newBuilder() - .setGapSize(Durations.fromMillis( - ((Sessions) windowFn).getGapDuration().getMillis())) - .build()))) + .setAnyParam(Any.pack(sessionsPayload)) + .setPayload(sessionsPayload.toByteString())) .build(); } else { return SdkFunctionSpec.newBuilder() .setSpec( FunctionSpec.newBuilder() .setUrn(SERIALIZED_JAVA_WINDOWFN_URN) - .setParameter( - Any.pack( - BytesValue.newBuilder() - .setValue( - ByteString.copyFrom( - SerializableUtils.serializeToByteArray(windowFn))) - .build()))) + .setAnyParam(Any.pack(BytesValue.newBuilder().setValue(serializedFn).build())) + .setPayload(serializedFn)) .build(); } } @@ -365,49 +356,41 @@ public static RunnerApi.WindowingStrategy toProto( case GLOBAL_WINDOWS_FN: return new GlobalWindows(); case FIXED_WINDOWS_FN: - StandardWindowFns.FixedWindowsPayload fixedParams = - windowFnSpec - .getSpec() - .getParameter() - .unpack(StandardWindowFns.FixedWindowsPayload.class); + StandardWindowFns.FixedWindowsPayload fixedParams = null; + fixedParams = + StandardWindowFns.FixedWindowsPayload.parseFrom( + windowFnSpec.getSpec().getPayload()); return FixedWindows.of(Duration.millis(Durations.toMillis(fixedParams.getSize()))) .withOffset(Duration.millis(Timestamps.toMillis(fixedParams.getOffset()))); case SLIDING_WINDOWS_FN: StandardWindowFns.SlidingWindowsPayload slidingParams = - windowFnSpec - .getSpec() - .getParameter() - .unpack(StandardWindowFns.SlidingWindowsPayload.class); + StandardWindowFns.SlidingWindowsPayload.parseFrom( + windowFnSpec.getSpec().getPayload()); return SlidingWindows.of(Duration.millis(Durations.toMillis(slidingParams.getSize()))) .every(Duration.millis(Durations.toMillis(slidingParams.getPeriod()))) .withOffset(Duration.millis(Timestamps.toMillis(slidingParams.getOffset()))); case SESSION_WINDOWS_FN: StandardWindowFns.SessionsPayload sessionParams = - windowFnSpec.getSpec().getParameter().unpack(StandardWindowFns.SessionsPayload.class); + StandardWindowFns.SessionsPayload.parseFrom(windowFnSpec.getSpec().getPayload()); return Sessions.withGapDuration( Duration.millis(Durations.toMillis(sessionParams.getGapSize()))); case SERIALIZED_JAVA_WINDOWFN_URN: case OLD_SERIALIZED_JAVA_WINDOWFN_URN: return (WindowFn) SerializableUtils.deserializeFromByteArray( - windowFnSpec - .getSpec() - .getParameter() - .unpack(BytesValue.class) - .getValue() - .toByteArray(), - "WindowFn"); + windowFnSpec.getSpec().getPayload().toByteArray(), "WindowFn"); default: throw new IllegalArgumentException( "Unknown or unsupported WindowFn: " + windowFnSpec.getSpec().getUrn()); } - } catch (InvalidProtocolBufferException exc) { + } catch (InvalidProtocolBufferException e) { throw new IllegalArgumentException( String.format( "%s for %s with URN %s did not contain expected proto message for payload", FunctionSpec.class.getSimpleName(), WindowFn.class.getSimpleName(), - windowFnSpec.getSpec().getUrn())); + windowFnSpec.getSpec().getUrn()), + e); } } } diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/WriteFilesTranslation.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/WriteFilesTranslation.java index 7954b0ea811e0..aeefd4fc06bf7 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/WriteFilesTranslation.java +++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/WriteFilesTranslation.java @@ -25,9 +25,7 @@ import com.google.common.annotations.VisibleForTesting; import com.google.common.collect.Lists; import com.google.common.collect.Maps; -import com.google.protobuf.Any; import com.google.protobuf.ByteString; -import com.google.protobuf.BytesValue; import java.io.IOException; import java.io.Serializable; import java.util.Collections; @@ -83,13 +81,9 @@ private static SdkFunctionSpec toProto(String urn, Serializable serializable) { .setSpec( FunctionSpec.newBuilder() .setUrn(urn) - .setParameter( - Any.pack( - BytesValue.newBuilder() - .setValue( - ByteString.copyFrom( - SerializableUtils.serializeToByteArray(serializable))) - .build()))) + .setPayload( + ByteString.copyFrom(SerializableUtils.serializeToByteArray(serializable))) + .build()) .build(); } @@ -102,8 +96,7 @@ private static SdkFunctionSpec toProto(String urn, Serializable serializable) { FunctionSpec.class.getSimpleName(), sinkProto.getSpec().getUrn()); - byte[] serializedSink = - sinkProto.getSpec().getParameter().unpack(BytesValue.class).getValue().toByteArray(); + byte[] serializedSink = sinkProto.getSpec().getPayload().toByteArray(); return (FileBasedSink) SerializableUtils.deserializeFromByteArray( @@ -163,11 +156,13 @@ private static WriteFilesPayload getWriteFilesPayload( AppliedPTransform, PDone, ? extends PTransform, PDone>> transform) throws IOException { - return PTransformTranslation.toProto( - transform, Collections.>emptyList(), SdkComponents.create()) - .getSpec() - .getParameter() - .unpack(WriteFilesPayload.class); + return WriteFilesPayload.parseFrom( + PTransformTranslation.toProto( + transform, + Collections.>emptyList(), + SdkComponents.create()) + .getSpec() + .getPayload()); } static class WriteFilesTranslator implements TransformPayloadTranslator> { @@ -181,7 +176,7 @@ public FunctionSpec translate( AppliedPTransform> transform, SdkComponents components) { return FunctionSpec.newBuilder() .setUrn(getUrn(transform.getTransform())) - .setParameter(Any.pack(toProto(transform.getTransform()))) + .setPayload(toProto(transform.getTransform()).toByteString()) .build(); } } diff --git a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/CreatePCollectionViewTranslationTest.java b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/CreatePCollectionViewTranslationTest.java index 0d209a0425fcb..4f57af8b2e05e 100644 --- a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/CreatePCollectionViewTranslationTest.java +++ b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/CreatePCollectionViewTranslationTest.java @@ -21,7 +21,6 @@ import static org.junit.Assert.assertThat; import com.google.common.collect.ImmutableList; -import com.google.protobuf.BytesValue; import org.apache.beam.sdk.common.runner.v1.RunnerApi.FunctionSpec; import org.apache.beam.sdk.common.runner.v1.RunnerApi.ParDoPayload; import org.apache.beam.sdk.runners.AppliedPTransform; @@ -98,8 +97,7 @@ public void testEncodedProto() throws Exception { PCollectionView deserializedView = (PCollectionView) SerializableUtils.deserializeFromByteArray( - payload.getParameter().unpack(BytesValue.class).getValue().toByteArray(), - PCollectionView.class.getSimpleName()); + payload.getPayload().toByteArray(), PCollectionView.class.getSimpleName()); assertThat( deserializedView, Matchers.>equalTo(createViewTransform.getView())); @@ -126,7 +124,7 @@ public void testExtractionDirectFromTransform() throws Exception { PCollectionView deserializedView = (PCollectionView) SerializableUtils.deserializeFromByteArray( - payload.getParameter().unpack(BytesValue.class).getValue().toByteArray(), + payload.getPayload().toByteArray(), PCollectionView.class.getSimpleName()); assertThat( diff --git a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/ParDoTranslationTest.java b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/ParDoTranslationTest.java index c31e803a950a1..680f94060c201 100644 --- a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/ParDoTranslationTest.java +++ b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/ParDoTranslationTest.java @@ -156,8 +156,7 @@ public void toAndFromTransformProto() throws Exception { // Decode Pipeline rehydratedPipeline = Pipeline.create(); - ParDoPayload parDoPayload = - protoTransform.getSpec().getParameter().unpack(ParDoPayload.class); + ParDoPayload parDoPayload = ParDoPayload.parseFrom(protoTransform.getSpec().getPayload()); for (PCollectionView view : parDo.getSideInputs()) { SideInput sideInput = parDoPayload.getSideInputsOrThrow(view.getTagInternal().getId()); PCollectionView restoredView = diff --git a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/TestStreamTranslationTest.java b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/TestStreamTranslationTest.java index e4336dfb938ab..893f4b9755148 100644 --- a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/TestStreamTranslationTest.java +++ b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/TestStreamTranslationTest.java @@ -29,6 +29,7 @@ import org.apache.beam.sdk.coders.VarIntCoder; import org.apache.beam.sdk.common.runner.v1.RunnerApi; import org.apache.beam.sdk.common.runner.v1.RunnerApi.ParDoPayload; +import org.apache.beam.sdk.common.runner.v1.RunnerApi.TestStreamPayload; import org.apache.beam.sdk.runners.AppliedPTransform; import org.apache.beam.sdk.testing.TestPipeline; import org.apache.beam.sdk.testing.TestStream; @@ -100,8 +101,7 @@ public void testRegistrarEncodedProto() throws Exception { assertThat(spec.getUrn(), equalTo(TEST_STREAM_TRANSFORM_URN)); - RunnerApi.TestStreamPayload payload = - spec.getParameter().unpack(RunnerApi.TestStreamPayload.class); + RunnerApi.TestStreamPayload payload = TestStreamPayload.parseFrom(spec.getPayload()); verifyTestStreamEncoding( testStream, payload, RehydratedComponents.forComponents(components.toComponents())); diff --git a/sdks/common/runner-api/src/main/proto/beam_runner_api.proto b/sdks/common/runner-api/src/main/proto/beam_runner_api.proto index 9afb565cca6d1..fb5d47e3670a5 100644 --- a/sdks/common/runner-api/src/main/proto/beam_runner_api.proto +++ b/sdks/common/runner-api/src/main/proto/beam_runner_api.proto @@ -755,9 +755,12 @@ message FunctionSpec { // passed as-is. string urn = 1; + // (Deprecated) + google.protobuf.Any any_param = 2; + // (Optional) The data specifying any parameters to the URN. If // the URN does not require any arguments, this may be omitted. - google.protobuf.Any parameter = 2; + bytes payload = 3; } // TODO: transfer javadoc here diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/BeamFnDataReadRunner.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/BeamFnDataReadRunner.java index 1e611db2b1696..df0e5a29604da 100644 --- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/BeamFnDataReadRunner.java +++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/BeamFnDataReadRunner.java @@ -129,8 +129,8 @@ public BeamFnDataReadRunner createRunnerForPTransform( BeamFnDataClient beamFnDataClientFactory, Collection>> consumers) throws IOException { - this.apiServiceDescriptor = functionSpec.getParameter().unpack(BeamFnApi.RemoteGrpcPort.class) - .getApiServiceDescriptor(); + this.apiServiceDescriptor = + BeamFnApi.RemoteGrpcPort.parseFrom(functionSpec.getPayload()).getApiServiceDescriptor(); this.inputTarget = inputTarget; this.processBundleInstructionIdSupplier = processBundleInstructionIdSupplier; this.beamFnDataClientFactory = beamFnDataClientFactory; diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/BeamFnDataWriteRunner.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/BeamFnDataWriteRunner.java index bbed75301bd80..48b450aded292 100644 --- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/BeamFnDataWriteRunner.java +++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/BeamFnDataWriteRunner.java @@ -120,8 +120,8 @@ public BeamFnDataWriteRunner createRunnerForPTransform( Map coders, BeamFnDataClient beamFnDataClientFactory) throws IOException { - this.apiServiceDescriptor = functionSpec.getParameter().unpack(BeamFnApi.RemoteGrpcPort.class) - .getApiServiceDescriptor(); + this.apiServiceDescriptor = + BeamFnApi.RemoteGrpcPort.parseFrom(functionSpec.getPayload()).getApiServiceDescriptor(); this.beamFnDataClientFactory = beamFnDataClientFactory; this.processBundleInstructionIdSupplier = processBundleInstructionIdSupplier; this.outputTarget = outputTarget; diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/BoundedSourceRunner.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/BoundedSourceRunner.java index 4702e0529ab24..5f6509f1b51ec 100644 --- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/BoundedSourceRunner.java +++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/BoundedSourceRunner.java @@ -22,7 +22,6 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Multimap; -import com.google.protobuf.BytesValue; import com.google.protobuf.InvalidProtocolBufferException; import java.io.IOException; import java.util.Collection; @@ -122,17 +121,14 @@ public BoundedSourceRunner createRunnerForPTransform( public void start() throws Exception { try { // The representation here is defined as the java serialized representation of the - // bounded source object packed into a protobuf Any using a protobuf BytesValue wrapper. - byte[] bytes = definition.getParameter().unpack(BytesValue.class).getValue().toByteArray(); + // bounded source object in a ByteString wrapper. + byte[] bytes = definition.getPayload().toByteArray(); @SuppressWarnings("unchecked") InputT boundedSource = (InputT) SerializableUtils.deserializeFromByteArray(bytes, definition.toString()); runReadLoop(WindowedValue.valueInGlobalWindow(boundedSource)); } catch (InvalidProtocolBufferException e) { - throw new IOException( - String.format("Failed to decode %s, expected %s", - definition.getParameter().getTypeUrl(), BytesValue.getDescriptor().getFullName()), - e); + throw new IOException(String.format("Failed to decode %s", definition.getUrn()), e); } } diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/FnApiDoFnRunner.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/FnApiDoFnRunner.java index 97bd71c9dba64..86168f97fe81b 100644 --- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/FnApiDoFnRunner.java +++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/FnApiDoFnRunner.java @@ -25,8 +25,6 @@ import com.google.common.collect.ImmutableMultimap; import com.google.common.collect.Multimap; import com.google.protobuf.ByteString; -import com.google.protobuf.BytesValue; -import com.google.protobuf.InvalidProtocolBufferException; import java.util.Collection; import java.util.HashSet; import java.util.Iterator; @@ -109,13 +107,7 @@ public DoFnRunner createRunnerForPTransform( outputMapBuilder.build(); // Get the DoFnInfo from the serialized blob. - ByteString serializedFn; - try { - serializedFn = pTransform.getSpec().getParameter().unpack(BytesValue.class).getValue(); - } catch (InvalidProtocolBufferException e) { - throw new IllegalArgumentException( - String.format("Unable to unwrap DoFn %s", pTransform.getSpec()), e); - } + ByteString serializedFn = pTransform.getSpec().getPayload(); @SuppressWarnings({"unchecked", "rawtypes"}) DoFnInfo doFnInfo = (DoFnInfo) SerializableUtils.deserializeFromByteArray( serializedFn.toByteArray(), "DoFnInfo"); diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/BeamFnDataReadRunnerTest.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/BeamFnDataReadRunnerTest.java index d712f5fa4738a..92e60885a87ff 100644 --- a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/BeamFnDataReadRunnerTest.java +++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/BeamFnDataReadRunnerTest.java @@ -37,7 +37,6 @@ import com.google.common.collect.Iterables; import com.google.common.collect.Multimap; import com.google.common.util.concurrent.Uninterruptibles; -import com.google.protobuf.Any; import java.io.IOException; import java.util.ArrayList; import java.util.List; @@ -80,7 +79,7 @@ public class BeamFnDataReadRunnerTest { private static final BeamFnApi.RemoteGrpcPort PORT_SPEC = BeamFnApi.RemoteGrpcPort.newBuilder() .setApiServiceDescriptor(BeamFnApi.ApiServiceDescriptor.getDefaultInstance()).build(); private static final RunnerApi.FunctionSpec FUNCTION_SPEC = RunnerApi.FunctionSpec.newBuilder() - .setParameter(Any.pack(PORT_SPEC)).build(); + .setPayload(PORT_SPEC.toByteString()).build(); private static final Coder> CODER = WindowedValue.getFullCoder(StringUtf8Coder.of(), GlobalWindow.Coder.INSTANCE); private static final String CODER_SPEC_ID = "string-coder-id"; @@ -131,7 +130,7 @@ public void testCreatingAndProcessingBeamFnDataReadRunner() throws Exception { RunnerApi.FunctionSpec functionSpec = RunnerApi.FunctionSpec.newBuilder() .setUrn("urn:org.apache.beam:source:runner:0.1") - .setParameter(Any.pack(PORT_SPEC)) + .setPayload(PORT_SPEC.toByteString()) .build(); RunnerApi.PTransform pTransform = RunnerApi.PTransform.newBuilder() diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/BeamFnDataWriteRunnerTest.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/BeamFnDataWriteRunnerTest.java index 0caf19e318ef9..ffa3a2d9876df 100644 --- a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/BeamFnDataWriteRunnerTest.java +++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/BeamFnDataWriteRunnerTest.java @@ -37,7 +37,6 @@ import com.google.common.collect.ImmutableMap; import com.google.common.collect.Iterables; import com.google.common.collect.Multimap; -import com.google.protobuf.Any; import java.io.IOException; import java.util.ArrayList; import java.util.List; @@ -75,7 +74,7 @@ public class BeamFnDataWriteRunnerTest { private static final BeamFnApi.RemoteGrpcPort PORT_SPEC = BeamFnApi.RemoteGrpcPort.newBuilder() .setApiServiceDescriptor(BeamFnApi.ApiServiceDescriptor.getDefaultInstance()).build(); private static final RunnerApi.FunctionSpec FUNCTION_SPEC = RunnerApi.FunctionSpec.newBuilder() - .setParameter(Any.pack(PORT_SPEC)).build(); + .setPayload(PORT_SPEC.toByteString()).build(); private static final String CODER_ID = "string-coder-id"; private static final Coder> CODER = WindowedValue.getFullCoder(StringUtf8Coder.of(), GlobalWindow.Coder.INSTANCE); @@ -117,7 +116,7 @@ public void testCreatingAndProcessingBeamFnDataWriteRunner() throws Exception { RunnerApi.FunctionSpec functionSpec = RunnerApi.FunctionSpec.newBuilder() .setUrn("urn:org.apache.beam:sink:runner:0.1") - .setParameter(Any.pack(PORT_SPEC)) + .setPayload(PORT_SPEC.toByteString()) .build(); RunnerApi.PTransform pTransform = RunnerApi.PTransform.newBuilder() diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/BoundedSourceRunnerTest.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/BoundedSourceRunnerTest.java index 7aec16183c240..b9f22e854e764 100644 --- a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/BoundedSourceRunnerTest.java +++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/BoundedSourceRunnerTest.java @@ -31,9 +31,7 @@ import com.google.common.collect.ImmutableMap; import com.google.common.collect.Iterables; import com.google.common.collect.Multimap; -import com.google.protobuf.Any; import com.google.protobuf.ByteString; -import com.google.protobuf.BytesValue; import java.util.ArrayList; import java.util.Collection; import java.util.List; @@ -107,8 +105,7 @@ public void testStart() throws Exception { BoundedSourceRunner, Long> runner = new BoundedSourceRunner<>( PipelineOptionsFactory.create(), - RunnerApi.FunctionSpec.newBuilder().setParameter( - Any.pack(BytesValue.newBuilder().setValue(encodedSource).build())).build(), + RunnerApi.FunctionSpec.newBuilder().setPayload(encodedSource).build(), consumers); runner.start(); @@ -127,13 +124,12 @@ public void testCreatingAndProcessingSourceFromFactory() throws Exception { List startFunctions = new ArrayList<>(); List finishFunctions = new ArrayList<>(); - RunnerApi.FunctionSpec functionSpec = RunnerApi.FunctionSpec.newBuilder() - .setUrn("urn:org.apache.beam:source:java:0.1") - .setParameter(Any.pack(BytesValue.newBuilder() - .setValue(ByteString.copyFrom( - SerializableUtils.serializeToByteArray(CountingSource.upTo(3)))) - .build())) - .build(); + RunnerApi.FunctionSpec functionSpec = + RunnerApi.FunctionSpec.newBuilder() + .setUrn("urn:org.apache.beam:source:java:0.1") + .setPayload( + ByteString.copyFrom(SerializableUtils.serializeToByteArray(CountingSource.upTo(3)))) + .build(); RunnerApi.PTransform pTransform = RunnerApi.PTransform.newBuilder() .setSpec(functionSpec) diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/FnApiDoFnRunnerTest.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/FnApiDoFnRunnerTest.java index e269bcc595671..efa8fcf195b18 100644 --- a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/FnApiDoFnRunnerTest.java +++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/FnApiDoFnRunnerTest.java @@ -31,9 +31,7 @@ import com.google.common.collect.ImmutableMap; import com.google.common.collect.Iterables; import com.google.common.collect.Multimap; -import com.google.protobuf.Any; import com.google.protobuf.ByteString; -import com.google.protobuf.BytesValue; import java.util.ArrayList; import java.util.List; import java.util.ServiceLoader; @@ -102,12 +100,11 @@ public void testCreatingAndProcessingDoFn() throws Exception { ImmutableMap.of( Long.parseLong(mainOutputId), TestDoFn.mainOutput, Long.parseLong(additionalOutputId), TestDoFn.additionalOutput)); - RunnerApi.FunctionSpec functionSpec = RunnerApi.FunctionSpec.newBuilder() - .setUrn(ParDoTranslation.CUSTOM_JAVA_DO_FN_URN) - .setParameter(Any.pack(BytesValue.newBuilder() - .setValue(ByteString.copyFrom(SerializableUtils.serializeToByteArray(doFnInfo))) - .build())) - .build(); + RunnerApi.FunctionSpec functionSpec = + RunnerApi.FunctionSpec.newBuilder() + .setUrn(ParDoTranslation.CUSTOM_JAVA_DO_FN_URN) + .setPayload(ByteString.copyFrom(SerializableUtils.serializeToByteArray(doFnInfo))) + .build(); RunnerApi.PTransform pTransform = RunnerApi.PTransform.newBuilder() .setSpec(functionSpec) .putInputs("inputA", "inputATarget") diff --git a/sdks/python/apache_beam/coders/coders.py b/sdks/python/apache_beam/coders/coders.py index c56ef52301b7d..7ced5a9e12be1 100644 --- a/sdks/python/apache_beam/coders/coders.py +++ b/sdks/python/apache_beam/coders/coders.py @@ -206,22 +206,21 @@ def to_runner_api(self, context): """For internal use only; no backwards-compatibility guarantees. """ # TODO(BEAM-115): Use specialized URNs and components. + serialized_coder = serialize_coder(self) return beam_runner_api_pb2.Coder( spec=beam_runner_api_pb2.SdkFunctionSpec( spec=beam_runner_api_pb2.FunctionSpec( urn=urns.PICKLED_CODER, - parameter=proto_utils.pack_Any( + any_param=proto_utils.pack_Any( google.protobuf.wrappers_pb2.BytesValue( - value=serialize_coder(self)))))) + value=serialized_coder)), + payload=serialized_coder))) @staticmethod def from_runner_api(proto, context): """For internal use only; no backwards-compatibility guarantees. """ - any_proto = proto.spec.spec.parameter - bytes_proto = google.protobuf.wrappers_pb2.BytesValue() - any_proto.Unpack(bytes_proto) - return deserialize_coder(bytes_proto.value) + return deserialize_coder(proto.spec.spec.payload) class StrUtf8Coder(Coder): diff --git a/sdks/python/apache_beam/runners/portability/fn_api_runner.py b/sdks/python/apache_beam/runners/portability/fn_api_runner.py index 3222bcb9ba469..7c0c06fe11109 100644 --- a/sdks/python/apache_beam/runners/portability/fn_api_runner.py +++ b/sdks/python/apache_beam/runners/portability/fn_api_runner.py @@ -218,16 +218,16 @@ def is_flatten(self): def side_inputs(self): for transform in self.transforms: if transform.spec.urn == urns.PARDO_TRANSFORM: - payload = proto_utils.unpack_Any( - transform.spec.parameter, beam_runner_api_pb2.ParDoPayload) + payload = proto_utils.parse_Bytes( + transform.spec.payload, beam_runner_api_pb2.ParDoPayload) for side_input in payload.side_inputs: yield transform.inputs[side_input] def has_as_main_input(self, pcoll): for transform in self.transforms: if transform.spec.urn == urns.PARDO_TRANSFORM: - payload = proto_utils.unpack_Any( - transform.spec.parameter, beam_runner_api_pb2.ParDoPayload) + payload = proto_utils.parse_Bytes( + transform.spec.payload, beam_runner_api_pb2.ParDoPayload) local_side_inputs = payload.side_inputs else: local_side_inputs = {} @@ -257,9 +257,7 @@ def expand_gbk(stages): transform = stage.transforms[0] if transform.spec.urn == urns.GROUP_BY_KEY_ONLY_TRANSFORM: # This is used later to correlate the read and write. - param = proto_utils.pack_Any( - wrappers_pb2.BytesValue( - value=str("group:%s" % stage.name))) + param = str("group:%s" % stage.name) gbk_write = Stage( transform.unique_name + '/Write', [beam_runner_api_pb2.PTransform( @@ -267,7 +265,9 @@ def expand_gbk(stages): inputs=transform.inputs, spec=beam_runner_api_pb2.FunctionSpec( urn=bundle_processor.DATA_OUTPUT_URN, - parameter=param))], + any_param=proto_utils.pack_Any( + wrappers_pb2.BytesValue(value=param)), + payload=param))], downstream_side_inputs=frozenset(), must_follow=stage.must_follow) yield gbk_write @@ -279,7 +279,9 @@ def expand_gbk(stages): outputs=transform.outputs, spec=beam_runner_api_pb2.FunctionSpec( urn=bundle_processor.DATA_INPUT_URN, - parameter=param))], + any_param=proto_utils.pack_Any( + wrappers_pb2.BytesValue(value=param)), + payload=param))], downstream_side_inputs=frozenset(), must_follow=union(frozenset([gbk_write]), stage.must_follow)) else: @@ -299,9 +301,7 @@ def sink_flattens(stages): transform = stage.transforms[0] if transform.spec.urn == urns.FLATTEN_TRANSFORM: # This is used later to correlate the read and writes. - param = proto_utils.pack_Any( - wrappers_pb2.BytesValue( - value=str("materialize:%s" % transform.unique_name))) + param = str("materialize:%s" % transform.unique_name) output_pcoll_id, = transform.outputs.values() output_coder_id = pcollections[output_pcoll_id].coder_id flatten_writes = [] @@ -337,7 +337,10 @@ def sink_flattens(stages): inputs={local_in: transcoded_pcollection}, spec=beam_runner_api_pb2.FunctionSpec( urn=bundle_processor.DATA_OUTPUT_URN, - parameter=param))], + any_param=proto_utils.pack_Any( + wrappers_pb2.BytesValue( + value=param)), + payload=param))], downstream_side_inputs=frozenset(), must_follow=stage.must_follow) flatten_writes.append(flatten_write) @@ -350,7 +353,10 @@ def sink_flattens(stages): outputs=transform.outputs, spec=beam_runner_api_pb2.FunctionSpec( urn=bundle_processor.DATA_INPUT_URN, - parameter=param))], + any_param=proto_utils.pack_Any( + wrappers_pb2.BytesValue( + value=param)), + payload=param))], downstream_side_inputs=frozenset(), must_follow=union(frozenset(flatten_writes), stage.must_follow)) @@ -439,9 +445,7 @@ def fuse(producer, consumer): # Now try to fuse away all pcollections. for pcoll, producer in producers_by_pcoll.items(): - pcoll_as_param = proto_utils.pack_Any( - wrappers_pb2.BytesValue( - value=str("materialize:%s" % pcoll))) + pcoll_as_param = str("materialize:%s" % pcoll) write_pcoll = None for consumer in consumers_by_pcoll[pcoll]: producer = replacement(producer) @@ -461,7 +465,10 @@ def fuse(producer, consumer): inputs={'in': pcoll}, spec=beam_runner_api_pb2.FunctionSpec( urn=bundle_processor.DATA_OUTPUT_URN, - parameter=pcoll_as_param))]) + any_param=proto_utils.pack_Any( + wrappers_pb2.BytesValue( + value=pcoll_as_param)), + payload=pcoll_as_param))]) fuse(producer, write_pcoll) if consumer.has_as_main_input(pcoll): read_pcoll = Stage( @@ -471,7 +478,10 @@ def fuse(producer, consumer): outputs={'out': pcoll}, spec=beam_runner_api_pb2.FunctionSpec( urn=bundle_processor.DATA_INPUT_URN, - parameter=pcoll_as_param))], + any_param=proto_utils.pack_Any( + wrappers_pb2.BytesValue( + value=pcoll_as_param)), + payload=pcoll_as_param))], must_follow={write_pcoll}) fuse(read_pcoll, consumer) @@ -567,8 +577,7 @@ def extract_endpoints(stage): data_side_input = {} data_output = {} for transform in stage.transforms: - pcoll_id = proto_utils.unpack_Any( - transform.spec.parameter, wrappers_pb2.BytesValue).value + pcoll_id = transform.spec.payload if transform.spec.urn in (bundle_processor.DATA_INPUT_URN, bundle_processor.DATA_OUTPUT_URN): if transform.spec.urn == bundle_processor.DATA_INPUT_URN: @@ -580,9 +589,11 @@ def extract_endpoints(stage): else: raise NotImplementedError if data_operation_spec: - transform.spec.parameter.CopyFrom(data_operation_spec) + transform.spec.payload = data_operation_spec + transform.spec.any_param.CopyFrom(data_operation_spec) else: - transform.spec.parameter.Clear() + transform.spec.payload = "" + transform.spec.any_param.Clear() return data_input, data_side_input, data_output logging.info('Running %s', stage.name) @@ -728,7 +739,9 @@ def get_outputs(op_ix): runner_sinks[(transform_id, target_name)] = operation transform_spec = beam_runner_api_pb2.FunctionSpec( urn=bundle_processor.DATA_OUTPUT_URN, - parameter=proto_utils.pack_Any(data_operation_spec)) + any_param=proto_utils.pack_Any(data_operation_spec), + payload=data_operation_spec.SerializeToString() \ + if data_operation_spec is not None else None) elif isinstance(operation, operation_specs.WorkerRead): # A Read from an in-memory source is done over the data plane. @@ -742,19 +755,23 @@ def get_outputs(op_ix): operation.source.source.default_output_coder()) transform_spec = beam_runner_api_pb2.FunctionSpec( urn=bundle_processor.DATA_INPUT_URN, - parameter=proto_utils.pack_Any(data_operation_spec)) + any_param=proto_utils.pack_Any(data_operation_spec), + payload=data_operation_spec.SerializeToString() \ + if data_operation_spec is not None else None) else: # Otherwise serialize the source and execute it there. # TODO: Use SDFs with an initial impulse. # The Dataflow runner harness strips the base64 encoding. do the same # here until we get the same thing back that we sent in. + source_bytes = base64.b64decode( + pickler.dumps(operation.source.source)) transform_spec = beam_runner_api_pb2.FunctionSpec( urn=bundle_processor.PYTHON_SOURCE_URN, - parameter=proto_utils.pack_Any( + any_param=proto_utils.pack_Any( wrappers_pb2.BytesValue( - value=base64.b64decode( - pickler.dumps(operation.source.source))))) + value=source_bytes)), + payload=source_bytes) elif isinstance(operation, operation_specs.WorkerDoFn): # Record the contents of each side input for access via the state api. @@ -773,8 +790,9 @@ def get_outputs(op_ix): (operation.serialized_fn, side_input_extras)) transform_spec = beam_runner_api_pb2.FunctionSpec( urn=bundle_processor.PYTHON_DOFN_URN, - parameter=proto_utils.pack_Any( - wrappers_pb2.BytesValue(value=augmented_serialized_fn))) + any_param=proto_utils.pack_Any( + wrappers_pb2.BytesValue(value=augmented_serialized_fn)), + payload=augmented_serialized_fn) elif isinstance(operation, operation_specs.WorkerFlatten): # Flatten is nice and simple. diff --git a/sdks/python/apache_beam/runners/worker/bundle_processor.py b/sdks/python/apache_beam/runners/worker/bundle_processor.py index 9474eda4725f5..16c888c63c60f 100644 --- a/sdks/python/apache_beam/runners/worker/bundle_processor.py +++ b/sdks/python/apache_beam/runners/worker/bundle_processor.py @@ -282,9 +282,9 @@ def wrapper(func): def create_operation(self, transform_id, consumers): transform_proto = self.descriptor.transforms[transform_id] creator, parameter_type = self._known_urns[transform_proto.spec.urn] - parameter = proto_utils.unpack_Any( - transform_proto.spec.parameter, parameter_type) - return creator(self, transform_id, transform_proto, parameter, consumers) + payload = proto_utils.parse_Bytes( + transform_proto.spec.payload, parameter_type) + return creator(self, transform_id, transform_proto, payload, consumers) def get_coder(self, coder_id): coder_proto = self.descriptor.coders[coder_id] @@ -293,9 +293,7 @@ def get_coder(self, coder_id): else: # No URN, assume cloud object encoding json bytes. return operation_specs.get_coder_from_spec( - json.loads( - proto_utils.unpack_Any(coder_proto.spec.spec.parameter, - wrappers_pb2.BytesValue).value)) + json.loads(coder_proto.spec.spec.payload)) def get_output_coders(self, transform_proto): return { @@ -360,10 +358,10 @@ def create(factory, transform_id, transform_proto, grpc_port, consumers): data_channel=factory.data_channel_factory.create_data_channel(grpc_port)) -@BeamTransformFactory.register_urn(PYTHON_SOURCE_URN, wrappers_pb2.BytesValue) +@BeamTransformFactory.register_urn(PYTHON_SOURCE_URN, None) def create(factory, transform_id, transform_proto, parameter, consumers): # The Dataflow runner harness strips the base64 encoding. - source = pickler.loads(base64.b64encode(parameter.value)) + source = pickler.loads(base64.b64encode(parameter)) spec = operation_specs.WorkerRead( iobase.SourceBundle(1.0, source, None, None), [WindowedValueCoder(source.default_output_coder())]) @@ -395,9 +393,9 @@ def create(factory, transform_id, transform_proto, parameter, consumers): consumers) -@BeamTransformFactory.register_urn(PYTHON_DOFN_URN, wrappers_pb2.BytesValue) +@BeamTransformFactory.register_urn(PYTHON_DOFN_URN, None) def create(factory, transform_id, transform_proto, parameter, consumers): - dofn_data = pickler.loads(parameter.value) + dofn_data = pickler.loads(parameter) if len(dofn_data) == 2: # Has side input data. serialized_fn, side_input_data = dofn_data @@ -413,8 +411,7 @@ def create(factory, transform_id, transform_proto, parameter, consumers): urns.PARDO_TRANSFORM, beam_runner_api_pb2.ParDoPayload) def create(factory, transform_id, transform_proto, parameter, consumers): assert parameter.do_fn.spec.urn == urns.PICKLED_DO_FN_INFO - serialized_fn = proto_utils.unpack_Any( - parameter.do_fn.spec.parameter, wrappers_pb2.BytesValue).value + serialized_fn = parameter.do_fn.spec.payload dofn_data = pickler.loads(serialized_fn) if len(dofn_data) == 2: # Has side input data. diff --git a/sdks/python/apache_beam/transforms/core.py b/sdks/python/apache_beam/transforms/core.py index 3f92ce95a3bad..9018a496cdd99 100644 --- a/sdks/python/apache_beam/transforms/core.py +++ b/sdks/python/apache_beam/transforms/core.py @@ -699,24 +699,24 @@ def _pardo_fn_data(self): def to_runner_api_parameter(self, context): assert self.__class__ is ParDo + picked_pardo_fn_data = pickler.dumps(self._pardo_fn_data()) return ( urns.PARDO_TRANSFORM, beam_runner_api_pb2.ParDoPayload( do_fn=beam_runner_api_pb2.SdkFunctionSpec( spec=beam_runner_api_pb2.FunctionSpec( urn=urns.PICKLED_DO_FN_INFO, - parameter=proto_utils.pack_Any( + any_param=proto_utils.pack_Any( wrappers_pb2.BytesValue( - value=pickler.dumps( - self._pardo_fn_data()))))))) + value=picked_pardo_fn_data)), + payload=picked_pardo_fn_data)))) @PTransform.register_urn( urns.PARDO_TRANSFORM, beam_runner_api_pb2.ParDoPayload) def from_runner_api_parameter(pardo_payload, context): assert pardo_payload.do_fn.spec.urn == urns.PICKLED_DO_FN_INFO fn, args, kwargs, si_tags_and_types, windowing = pickler.loads( - proto_utils.unpack_Any( - pardo_payload.do_fn.spec.parameter, wrappers_pb2.BytesValue).value) + pardo_payload.do_fn.spec.payload) if si_tags_and_types: raise NotImplementedError('deferred side inputs') elif windowing: diff --git a/sdks/python/apache_beam/transforms/ptransform.py b/sdks/python/apache_beam/transforms/ptransform.py index da113e055c946..a798fa1b3b250 100644 --- a/sdks/python/apache_beam/transforms/ptransform.py +++ b/sdks/python/apache_beam/transforms/ptransform.py @@ -442,7 +442,9 @@ def to_runner_api(self, context): urn, typed_param = self.to_runner_api_parameter(context) return beam_runner_api_pb2.FunctionSpec( urn=urn, - parameter=proto_utils.pack_Any(typed_param)) + any_param=proto_utils.pack_Any(typed_param), + payload=typed_param.SerializeToString() + if typed_param is not None else None) @classmethod def from_runner_api(cls, proto, context): @@ -450,7 +452,7 @@ def from_runner_api(cls, proto, context): return None parameter_type, constructor = cls._known_urns[proto.urn] return constructor( - proto_utils.unpack_Any(proto.parameter, parameter_type), + proto_utils.parse_Bytes(proto.payload, parameter_type), context) def to_runner_api_parameter(self, context): diff --git a/sdks/python/apache_beam/utils/proto_utils.py b/sdks/python/apache_beam/utils/proto_utils.py index af8f21897b876..d7693f3f78395 100644 --- a/sdks/python/apache_beam/utils/proto_utils.py +++ b/sdks/python/apache_beam/utils/proto_utils.py @@ -46,6 +46,17 @@ def unpack_Any(any_msg, msg_class): return msg +def parse_Bytes(bytes, msg_class): + """Parses the String of bytes into msg_class. + + Returns the input bytes if msg_class is None.""" + if msg_class is None: + return bytes + msg = msg_class() + msg.ParseFromString(bytes) + return msg + + def pack_Struct(**kwargs): """Returns a struct containing the values indicated by kwargs. """ diff --git a/sdks/python/apache_beam/utils/urns.py b/sdks/python/apache_beam/utils/urns.py index 0013cb3b6e2c5..acf729f2dc8e1 100644 --- a/sdks/python/apache_beam/utils/urns.py +++ b/sdks/python/apache_beam/utils/urns.py @@ -120,7 +120,9 @@ def to_runner_api(self, context): return beam_runner_api_pb2.SdkFunctionSpec( spec=beam_runner_api_pb2.FunctionSpec( urn=urn, - parameter=proto_utils.pack_Any(typed_param))) + any_param=proto_utils.pack_Any(typed_param), + payload=typed_param.SerializeToString() + if typed_param is not None else None)) @classmethod def from_runner_api(cls, fn_proto, context): @@ -130,5 +132,5 @@ def from_runner_api(cls, fn_proto, context): """ parameter_type, constructor = cls._known_urns[fn_proto.spec.urn] return constructor( - proto_utils.unpack_Any(fn_proto.spec.parameter, parameter_type), + proto_utils.parse_Bytes(fn_proto.spec.payload, parameter_type), context) From 76db0aa30c632296a6a882c012f9da2d21f775b5 Mon Sep 17 00:00:00 2001 From: Sourabh Bajaj Date: Wed, 2 Aug 2017 10:49:48 -0700 Subject: [PATCH 256/346] [BEAM-2431] Add experimental python rpc direct runner --- .../runners/experimental/__init__.py | 16 +++ .../python_rpc_direct/__init__.py | 22 ++++ .../python_rpc_direct_runner.py | 111 ++++++++++++++++++ .../experimental/python_rpc_direct/server.py | 111 ++++++++++++++++++ .../apache_beam/runners/job/__init__.py | 16 +++ .../python/apache_beam/runners/job/manager.py | 52 ++++++++ sdks/python/apache_beam/runners/job/utils.py | 32 +++++ sdks/python/apache_beam/runners/runner.py | 6 + 8 files changed, 366 insertions(+) create mode 100644 sdks/python/apache_beam/runners/experimental/__init__.py create mode 100644 sdks/python/apache_beam/runners/experimental/python_rpc_direct/__init__.py create mode 100644 sdks/python/apache_beam/runners/experimental/python_rpc_direct/python_rpc_direct_runner.py create mode 100644 sdks/python/apache_beam/runners/experimental/python_rpc_direct/server.py create mode 100644 sdks/python/apache_beam/runners/job/__init__.py create mode 100644 sdks/python/apache_beam/runners/job/manager.py create mode 100644 sdks/python/apache_beam/runners/job/utils.py diff --git a/sdks/python/apache_beam/runners/experimental/__init__.py b/sdks/python/apache_beam/runners/experimental/__init__.py new file mode 100644 index 0000000000000..cce3acad34a49 --- /dev/null +++ b/sdks/python/apache_beam/runners/experimental/__init__.py @@ -0,0 +1,16 @@ +# +# 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. +# diff --git a/sdks/python/apache_beam/runners/experimental/python_rpc_direct/__init__.py b/sdks/python/apache_beam/runners/experimental/python_rpc_direct/__init__.py new file mode 100644 index 0000000000000..5d14030139329 --- /dev/null +++ b/sdks/python/apache_beam/runners/experimental/python_rpc_direct/__init__.py @@ -0,0 +1,22 @@ +# +# 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. +# + +"""This is the experimental direct runner for testing the job api that +sends a runner API proto over the API and then runs it on the other side. +""" + +from apache_beam.runners.experimental.python_rpc_direct.python_rpc_direct_runner import PythonRPCDirectRunner diff --git a/sdks/python/apache_beam/runners/experimental/python_rpc_direct/python_rpc_direct_runner.py b/sdks/python/apache_beam/runners/experimental/python_rpc_direct/python_rpc_direct_runner.py new file mode 100644 index 0000000000000..247ce1f0e1956 --- /dev/null +++ b/sdks/python/apache_beam/runners/experimental/python_rpc_direct/python_rpc_direct_runner.py @@ -0,0 +1,111 @@ +# +# 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. +# + +"""A runner implementation that submits a job for remote execution. +""" + +import logging +import random +import string + +import grpc + +from apache_beam.portability.api import beam_job_api_pb2 +from apache_beam.runners.job import utils as job_utils +from apache_beam.runners.job.manager import DockerRPCManager +from apache_beam.runners.runner import PipelineResult +from apache_beam.runners.runner import PipelineRunner + + +__all__ = ['PythonRPCDirectRunner'] + + +class PythonRPCDirectRunner(PipelineRunner): + """Executes a single pipeline on the local machine inside a container.""" + + # A list of PTransformOverride objects to be applied before running a pipeline + # using DirectRunner. + # Currently this only works for overrides where the input and output types do + # not change. + # For internal SDK use only. This should not be updated by Beam pipeline + # authors. + _PTRANSFORM_OVERRIDES = [] + + def __init__(self): + self._cache = None + + def run(self, pipeline): + """Remotely executes entire pipeline or parts reachable from node.""" + + # Performing configured PTransform overrides. + pipeline.replace_all(PythonRPCDirectRunner._PTRANSFORM_OVERRIDES) + + # Start the RPC co-process + manager = DockerRPCManager() + + # Submit the job to the RPC co-process + jobName = ('Job-' + + ''.join(random.choice(string.ascii_uppercase) for _ in range(6))) + options = {k: v for k, v in pipeline._options.get_all_options().iteritems() + if v is not None} + + try: + response = manager.service.run(beam_job_api_pb2.SubmitJobRequest( + pipeline=pipeline.to_runner_api(), + pipelineOptions=job_utils.dict_to_struct(options), + jobName=jobName)) + + logging.info('Submitted a job with id: %s', response.jobId) + + # Return the result object that references the manager instance + result = PythonRPCDirectPipelineResult(response.jobId, manager) + return result + except grpc.RpcError: + logging.error('Failed to run the job with name: %s', jobName) + raise + + +class PythonRPCDirectPipelineResult(PipelineResult): + """Represents the state of a pipeline run on the Dataflow service.""" + + def __init__(self, job_id, job_manager): + self.job_id = job_id + self.manager = job_manager + + @property + def state(self): + return self.manager.service.getState( + beam_job_api_pb2.GetJobStateRequest(jobId=self.job_id)) + + def wait_until_finish(self): + messages_request = beam_job_api_pb2.JobMessagesRequest(jobId=self.job_id) + for message in self.manager.service.getMessageStream(messages_request): + if message.HasField('stateResponse'): + logging.info( + 'Current state of job: %s', + beam_job_api_pb2.JobState.JobStateType.Name( + message.stateResponse.state)) + else: + logging.info('Message %s', message.messageResponse) + logging.info('Job with id: %s in terminal state now.', self.job_id) + + def cancel(self): + return self.manager.service.cancel( + beam_job_api_pb2.CancelJobRequest(jobId=self.job_id)) + + def metrics(self): + raise NotImplementedError diff --git a/sdks/python/apache_beam/runners/experimental/python_rpc_direct/server.py b/sdks/python/apache_beam/runners/experimental/python_rpc_direct/server.py new file mode 100644 index 0000000000000..3addf92811d47 --- /dev/null +++ b/sdks/python/apache_beam/runners/experimental/python_rpc_direct/server.py @@ -0,0 +1,111 @@ +# +# 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. +# + +"""A runner implementation that submits a job for remote execution. +""" +from concurrent import futures +import time +import uuid + +import grpc + +from apache_beam.portability.api import beam_job_api_pb2 +from apache_beam.portability.api import beam_job_api_pb2_grpc +from apache_beam.pipeline import Pipeline +from apache_beam.options.pipeline_options import PipelineOptions +from apache_beam.runners.runner import PipelineState + +_ONE_DAY_IN_SECONDS = 60 * 60 * 24 + + +class JobService(beam_job_api_pb2_grpc.JobServiceServicer): + + def __init__(self): + self.jobs = {} + + def run(self, request, context): + job_id = uuid.uuid4().get_hex() + pipeline_result = Pipeline.from_runner_api( + request.pipeline, + 'DirectRunner', + PipelineOptions()).run() + self.jobs[job_id] = pipeline_result + return beam_job_api_pb2.SubmitJobResponse(jobId=job_id) + + def getState(self, request, context): + pipeline_result = self.jobs[request.jobId] + return beam_job_api_pb2.GetJobStateResponse( + state=self._map_state_to_jobState(pipeline_result.state)) + + def cancel(self, request, context): + pipeline_result = self.jobs[request.jobId] + pipeline_result.cancel() + return beam_job_api_pb2.CancelJobResponse( + state=self._map_state_to_jobState(pipeline_result.state)) + + def getMessageStream(self, request, context): + pipeline_result = self.jobs[request.jobId] + pipeline_result.wait_until_finish() + yield beam_job_api_pb2.JobMessagesResponse( + stateResponse=beam_job_api_pb2.GetJobStateResponse( + state=self._map_state_to_jobState(pipeline_result.state))) + + def getStateStream(self, request, context): + context.set_details('Not Implemented for direct runner!') + context.set_code(grpc.StatusCode.UNIMPLEMENTED) + return + + @staticmethod + def _map_state_to_jobState(state): + if state == PipelineState.UNKNOWN: + return beam_job_api_pb2.JobState.UNKNOWN + elif state == PipelineState.STOPPED: + return beam_job_api_pb2.JobState.STOPPED + elif state == PipelineState.RUNNING: + return beam_job_api_pb2.JobState.RUNNING + elif state == PipelineState.DONE: + return beam_job_api_pb2.JobState.DONE + elif state == PipelineState.FAILED: + return beam_job_api_pb2.JobState.FAILED + elif state == PipelineState.CANCELLED: + return beam_job_api_pb2.JobState.CANCELLED + elif state == PipelineState.UPDATED: + return beam_job_api_pb2.JobState.UPDATED + elif state == PipelineState.DRAINING: + return beam_job_api_pb2.JobState.DRAINING + elif state == PipelineState.DRAINED: + return beam_job_api_pb2.JobState.DRAINED + else: + raise ValueError('Unknown pipeline state') + + +def serve(): + server = grpc.server(futures.ThreadPoolExecutor(max_workers=10)) + beam_job_api_pb2_grpc.add_JobServiceServicer_to_server(JobService(), server) + + server.add_insecure_port('[::]:50051') + server.start() + + try: + while True: + time.sleep(_ONE_DAY_IN_SECONDS) + except KeyboardInterrupt: + server.stop(0) + + +if __name__ == '__main__': + serve() diff --git a/sdks/python/apache_beam/runners/job/__init__.py b/sdks/python/apache_beam/runners/job/__init__.py new file mode 100644 index 0000000000000..cce3acad34a49 --- /dev/null +++ b/sdks/python/apache_beam/runners/job/__init__.py @@ -0,0 +1,16 @@ +# +# 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. +# diff --git a/sdks/python/apache_beam/runners/job/manager.py b/sdks/python/apache_beam/runners/job/manager.py new file mode 100644 index 0000000000000..4d88a1189f230 --- /dev/null +++ b/sdks/python/apache_beam/runners/job/manager.py @@ -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. +# + +"""A object to control to the Job API Co-Process +""" + +import logging +import subprocess +import time + +import grpc + +from apache_beam.portability.api import beam_job_api_pb2_grpc + + +class DockerRPCManager(object): + """A native co-process to start a contianer that speaks the JobApi + """ + def __init__(self, run_command=None): + # TODO(BEAM-2431): Change this to a docker container from a command. + self.process = subprocess.Popen( + ['python', + '-m', + 'apache_beam.runners.experimental.python_rpc_direct.server']) + + self.channel = grpc.insecure_channel('localhost:50051') + self.service = beam_job_api_pb2_grpc.JobServiceStub(self.channel) + + # Sleep for 2 seconds for process to start completely + # This is just for the co-process and would be removed + # once we migrate to docker. + time.sleep(2) + + def __del__(self): + """Terminate the co-process when the manager is GC'ed + """ + logging.info('Shutting the co-process') + self.process.terminate() diff --git a/sdks/python/apache_beam/runners/job/utils.py b/sdks/python/apache_beam/runners/job/utils.py new file mode 100644 index 0000000000000..84c727fb4ebcb --- /dev/null +++ b/sdks/python/apache_beam/runners/job/utils.py @@ -0,0 +1,32 @@ +# +# 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. +# + +"""Utility functions for efficiently processing with the job API +""" + +import json + +from google.protobuf import json_format +from google.protobuf import struct_pb2 + + +def dict_to_struct(dict_obj): + return json_format.Parse(json.dumps(dict_obj), struct_pb2.Struct()) + + +def struct_to_dict(struct_obj): + return json.loads(json_format.MessageToJson(struct_obj)) diff --git a/sdks/python/apache_beam/runners/runner.py b/sdks/python/apache_beam/runners/runner.py index af00d8f27ee08..7ce9a03876f8a 100644 --- a/sdks/python/apache_beam/runners/runner.py +++ b/sdks/python/apache_beam/runners/runner.py @@ -41,7 +41,11 @@ def _get_runner_map(runner_names, module_path): _DATAFLOW_RUNNER_PATH = ( 'apache_beam.runners.dataflow.dataflow_runner.') _TEST_RUNNER_PATH = 'apache_beam.runners.test.' +_PYTHON_RPC_DIRECT_RUNNER = ( + 'apache_beam.runners.experimental.python_rpc_direct.' + 'python_rpc_direct_runner.') +_KNOWN_PYTHON_RPC_DIRECT_RUNNER = ('PythonRPCDirectRunner',) _KNOWN_DIRECT_RUNNERS = ('DirectRunner', 'EagerRunner') _KNOWN_DATAFLOW_RUNNERS = ('DataflowRunner',) _KNOWN_TEST_RUNNERS = ('TestDataflowRunner',) @@ -51,6 +55,8 @@ def _get_runner_map(runner_names, module_path): _DIRECT_RUNNER_PATH)) _RUNNER_MAP.update(_get_runner_map(_KNOWN_DATAFLOW_RUNNERS, _DATAFLOW_RUNNER_PATH)) +_RUNNER_MAP.update(_get_runner_map(_KNOWN_PYTHON_RPC_DIRECT_RUNNER, + _PYTHON_RPC_DIRECT_RUNNER)) _RUNNER_MAP.update(_get_runner_map(_KNOWN_TEST_RUNNERS, _TEST_RUNNER_PATH)) From cc699ece9e4321c3460c2aab04c74fa086c7a3cd Mon Sep 17 00:00:00 2001 From: Chuan Yu Foo Date: Mon, 31 Jul 2017 17:10:45 -0700 Subject: [PATCH 257/346] Add support for Python's native type hint types in Beam's type hint annotations. --- .../apache_beam/typehints/decorators.py | 18 +- .../typehints/native_type_compatibility.py | 164 ++++++++++++++++++ .../native_type_compatibility_test.py | 92 ++++++++++ .../typehints/typed_pipeline_test.py | 27 ++- .../python/apache_beam/typehints/typehints.py | 4 +- sdks/python/setup.py | 1 + 6 files changed, 300 insertions(+), 6 deletions(-) create mode 100644 sdks/python/apache_beam/typehints/native_type_compatibility.py create mode 100644 sdks/python/apache_beam/typehints/native_type_compatibility_test.py diff --git a/sdks/python/apache_beam/typehints/decorators.py b/sdks/python/apache_beam/typehints/decorators.py index 6ed388a85217a..d5954e21af497 100644 --- a/sdks/python/apache_beam/typehints/decorators.py +++ b/sdks/python/apache_beam/typehints/decorators.py @@ -86,6 +86,7 @@ def foo((a, b)): import inspect import types +from apache_beam.typehints import native_type_compatibility from apache_beam.typehints import typehints from apache_beam.typehints.typehints import check_constraint from apache_beam.typehints.typehints import CompositeTypeHintError @@ -347,13 +348,22 @@ def increment(ls): for all received function arguments. """ + converted_positional_hints = ( + native_type_compatibility.convert_to_beam_types(positional_hints)) + converted_keyword_hints = ( + native_type_compatibility.convert_to_beam_types(keyword_hints)) + del positional_hints + del keyword_hints + def annotate(f): if isinstance(f, types.FunctionType): - for t in list(positional_hints) + list(keyword_hints.values()): + for t in (list(converted_positional_hints) + + list(converted_keyword_hints.values())): validate_composite_type_param( t, error_msg_prefix='All type hint arguments') - get_type_hints(f).set_input_types(*positional_hints, **keyword_hints) + get_type_hints(f).set_input_types(*converted_positional_hints, + **converted_keyword_hints) return f return annotate @@ -410,7 +420,8 @@ def negate(p): "order to specify multiple return types, use the 'Tuple' " "type-hint.") - return_type_hint = return_type_hint[0] + return_type_hint = native_type_compatibility.convert_to_beam_type( + return_type_hint[0]) validate_composite_type_param( return_type_hint, @@ -420,6 +431,7 @@ def negate(p): def annotate(f): get_type_hints(f).set_output_types(return_type_hint) return f + return annotate diff --git a/sdks/python/apache_beam/typehints/native_type_compatibility.py b/sdks/python/apache_beam/typehints/native_type_compatibility.py new file mode 100644 index 0000000000000..d88f93308dd11 --- /dev/null +++ b/sdks/python/apache_beam/typehints/native_type_compatibility.py @@ -0,0 +1,164 @@ +# +# 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. +# + +"""Module to convert Python's native typing types to Beam types.""" + +import collections +import typing +from apache_beam.typehints import typehints + +# Describes an entry in the type map in convert_to_beam_type. +# match is a function that takes a user type and returns whether the conversion +# should trigger. +# arity is the expected arity of the user type. -1 means it's variadic. +# beam_type is the Beam type the user type should map to. +_TypeMapEntry = collections.namedtuple( + '_TypeMapEntry', ['match', 'arity', 'beam_type']) + + +def _get_arg(typ, index): + """Returns the index-th argument to the given type.""" + return typ.__args__[index] + + +def _len_arg(typ): + """Returns the length of the arguments to the given type.""" + try: + return len(typ.__args__) + except AttributeError: + # For Any type, which takes no arguments. + return 0 + + +def _safe_issubclass(derived, parent): + """Like issubclass, but swallows TypeErrors. + + This is useful for when either parameter might not actually be a class, + e.g. typing.Union isn't actually a class. + + Args: + derived: As in issubclass. + parent: As in issubclass. + + Returns: + issubclass(derived, parent), or False if a TypeError was raised. + """ + try: + return issubclass(derived, parent) + except TypeError: + return False + + +def _match_issubclass(match_against): + return lambda user_type: _safe_issubclass(user_type, match_against) + + +def _match_same_type(match_against): + # For Union types. They can't be compared with isinstance either, so we + # have to compare their types directly. + return lambda user_type: type(user_type) == type(match_against) + + +def _match_is_named_tuple(user_type): + return (_safe_issubclass(user_type, typing.Tuple) and + hasattr(user_type, '_field_types')) + + +def convert_to_beam_type(typ): + """Convert a given typing type to a Beam type. + + Args: + typ: typing type. + + Returns: + The given type converted to a Beam type as far as we can do the conversion. + + Raises: + ValueError: The type was malformed. + """ + + type_map = [ + _TypeMapEntry( + match=_match_same_type(typing.Any), + arity=0, + beam_type=typehints.Any), + _TypeMapEntry( + match=_match_issubclass(typing.Dict), + arity=2, + beam_type=typehints.Dict), + _TypeMapEntry( + match=_match_issubclass(typing.List), + arity=1, + beam_type=typehints.List), + _TypeMapEntry( + match=_match_issubclass(typing.Set), + arity=1, + beam_type=typehints.Set), + # NamedTuple is a subclass of Tuple, but it needs special handling. + # We just convert it to Any for now. + # This MUST appear before the entry for the normal Tuple. + _TypeMapEntry( + match=_match_is_named_tuple, arity=0, beam_type=typehints.Any), + _TypeMapEntry( + match=_match_issubclass(typing.Tuple), + arity=-1, + beam_type=typehints.Tuple), + _TypeMapEntry( + match=_match_same_type(typing.Union), + arity=-1, + beam_type=typehints.Union) + ] + + # Find the first matching entry. + matched_entry = next((entry for entry in type_map if entry.match(typ)), None) + if not matched_entry: + # No match: return original type. + return typ + + if matched_entry.arity == -1: + arity = _len_arg(typ) + else: + arity = matched_entry.arity + if _len_arg(typ) != arity: + raise ValueError('expecting type %s to have arity %d, had arity %d ' + 'instead' % (str(typ), arity, _len_arg(typ))) + typs = [convert_to_beam_type(_get_arg(typ, i)) for i in xrange(arity)] + if arity == 0: + # Nullary types (e.g. Any) don't accept empty tuples as arguments. + return matched_entry.beam_type + elif arity == 1: + # Unary types (e.g. Set) don't accept 1-tuples as arguments + return matched_entry.beam_type[typs[0]] + else: + return matched_entry.beam_type[tuple(typs)] + + +def convert_to_beam_types(args): + """Convert the given list or dictionary of args to Beam types. + + Args: + args: Either an iterable of types, or a dictionary where the values are + types. + + Returns: + If given an iterable, a list of converted types. If given a dictionary, + a dictionary with the same keys, and values which have been converted. + """ + if isinstance(args, dict): + return {k: convert_to_beam_type(v) for k, v in args.iteritems()} + else: + return [convert_to_beam_type(v) for v in args] diff --git a/sdks/python/apache_beam/typehints/native_type_compatibility_test.py b/sdks/python/apache_beam/typehints/native_type_compatibility_test.py new file mode 100644 index 0000000000000..d0cafe16d1129 --- /dev/null +++ b/sdks/python/apache_beam/typehints/native_type_compatibility_test.py @@ -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. +# + +"""Test for Beam type compatibility library.""" + +import typing +import unittest + +from apache_beam.typehints import typehints +from apache_beam.typehints import native_type_compatibility + +_TestNamedTuple = typing.NamedTuple('_TestNamedTuple', + [('age', int), ('name', bytes)]) +_TestFlatAlias = typing.Tuple[bytes, float] +_TestNestedAlias = typing.List[_TestFlatAlias] + + +class _TestClass(object): + pass + + +class NativeTypeCompatibilityTest(unittest.TestCase): + + def test_convert_to_beam_type(self): + test_cases = [ + ('raw bytes', bytes, bytes), + ('raw int', int, int), + ('raw float', float, float), + ('any', typing.Any, typehints.Any), + ('simple dict', typing.Dict[bytes, int], + typehints.Dict[bytes, int]), + ('simple list', typing.List[int], typehints.List[int]), + ('simple optional', typing.Optional[int], typehints.Optional[int]), + ('simple set', typing.Set[float], typehints.Set[float]), + ('simple unary tuple', typing.Tuple[bytes], + typehints.Tuple[bytes]), + ('simple union', typing.Union[int, bytes, float], + typehints.Union[int, bytes, float]), + ('namedtuple', _TestNamedTuple, typehints.Any), + ('test class', _TestClass, _TestClass), + ('test class in list', typing.List[_TestClass], + typehints.List[_TestClass]), + ('complex tuple', typing.Tuple[bytes, typing.List[typing.Tuple[ + bytes, typing.Union[int, bytes, float]]]], + typehints.Tuple[bytes, typehints.List[typehints.Tuple[ + bytes, typehints.Union[int, bytes, float]]]]), + ('flat alias', _TestFlatAlias, typehints.Tuple[bytes, float]), + ('nested alias', _TestNestedAlias, + typehints.List[typehints.Tuple[bytes, float]]), + ('complex dict', + typing.Dict[bytes, typing.List[typing.Tuple[bytes, _TestClass]]], + typehints.Dict[bytes, typehints.List[typehints.Tuple[ + bytes, _TestClass]]]) + ] + + for test_case in test_cases: + # Unlike typing types, Beam types are guaranteed to compare equal. + description = test_case[0] + typing_type = test_case[1] + beam_type = test_case[2] + self.assertEqual( + native_type_compatibility.convert_to_beam_type(typing_type), + beam_type, description) + + def test_convert_to_beam_types(self): + typing_types = [bytes, typing.List[bytes], + typing.List[typing.Tuple[bytes, int]], + typing.Union[int, typing.List[int]]] + beam_types = [bytes, typehints.List[bytes], + typehints.List[typehints.Tuple[bytes, int]], + typehints.Union[int, typehints.List[int]]] + self.assertEqual( + native_type_compatibility.convert_to_beam_types(typing_types), + beam_types) + + +if __name__ == '__main__': + unittest.main() diff --git a/sdks/python/apache_beam/typehints/typed_pipeline_test.py b/sdks/python/apache_beam/typehints/typed_pipeline_test.py index c81ef320e0279..58274f31238f1 100644 --- a/sdks/python/apache_beam/typehints/typed_pipeline_test.py +++ b/sdks/python/apache_beam/typehints/typed_pipeline_test.py @@ -17,9 +17,9 @@ """Unit tests for the type-hint objects and decorators.""" import inspect +import typing import unittest - import apache_beam as beam from apache_beam import pvalue from apache_beam import typehints @@ -98,6 +98,31 @@ def process(self, element): [1, 2, 3] | (beam.ParDo(my_do_fn) | 'again' >> beam.ParDo(my_do_fn)) +class NativeTypesTest(unittest.TestCase): + + def test_good_main_input(self): + @typehints.with_input_types(typing.Tuple[str, int]) + def munge((s, i)): + return (s + 's', i * 2) + result = [('apple', 5), ('pear', 3)] | beam.Map(munge) + self.assertEqual([('apples', 10), ('pears', 6)], sorted(result)) + + def test_bad_main_input(self): + @typehints.with_input_types(typing.Tuple[str, str]) + def munge((s, i)): + return (s + 's', i * 2) + with self.assertRaises(typehints.TypeCheckError): + [('apple', 5), ('pear', 3)] | beam.Map(munge) + + def test_bad_main_output(self): + @typehints.with_input_types(typing.Tuple[int, int]) + @typehints.with_output_types(typing.Tuple[str, str]) + def munge((a, b)): + return (str(a), str(b)) + with self.assertRaises(typehints.TypeCheckError): + [(5, 4), (3, 2)] | beam.Map(munge) | 'Again' >> beam.Map(munge) + + class SideInputTest(unittest.TestCase): def _run_repeat_test(self, repeat): diff --git a/sdks/python/apache_beam/typehints/typehints.py b/sdks/python/apache_beam/typehints/typehints.py index cc430be4a00e2..6039e0e7e20a0 100644 --- a/sdks/python/apache_beam/typehints/typehints.py +++ b/sdks/python/apache_beam/typehints/typehints.py @@ -498,7 +498,7 @@ def __getitem__(self, type_params): class OptionalHint(UnionHint): """An Option type-hint. Optional[X] accepts instances of X or None. - The Optional[X] factory function proxies to Union[X, None] + The Optional[X] factory function proxies to Union[X, type(None)] """ def __getitem__(self, py_type): @@ -507,7 +507,7 @@ def __getitem__(self, py_type): raise TypeError('An Option type-hint only accepts a single type ' 'parameter.') - return Union[py_type, None] + return Union[py_type, type(None)] class TupleHint(CompositeTypeHint): diff --git a/sdks/python/setup.py b/sdks/python/setup.py index da82466822e77..c13da8e326c82 100644 --- a/sdks/python/setup.py +++ b/sdks/python/setup.py @@ -104,6 +104,7 @@ def get_version(): 'oauth2client>=2.0.1,<4.0.0', 'protobuf>=3.2.0,<=3.3.0', 'pyyaml>=3.12,<4.0.0', + 'typing>=3.6.0,<3.7.0', ] REQUIRED_SETUP_PACKAGES = [ From 6aadf24f595acee5c0fe4de8b224c31fa1977a33 Mon Sep 17 00:00:00 2001 From: Pablo Date: Thu, 3 Aug 2017 17:46:09 -0700 Subject: [PATCH 258/346] Preparing support for Structured Names in Dataflow counters --- .../runners/dataflow/internal/apiclient.py | 51 ++++++---- sdks/python/apache_beam/utils/counters.py | 92 +++++++++---------- 2 files changed, 77 insertions(+), 66 deletions(-) diff --git a/sdks/python/apache_beam/runners/dataflow/internal/apiclient.py b/sdks/python/apache_beam/runners/dataflow/internal/apiclient.py index dcaf74e1543d2..a1f9301b19bad 100644 --- a/sdks/python/apache_beam/runners/dataflow/internal/apiclient.py +++ b/sdks/python/apache_beam/runners/dataflow/internal/apiclient.py @@ -710,10 +710,6 @@ def translate_value(value, metric_update_proto): metric_update_proto.integer = to_split_int(value) -def translate_scalar(accumulator, metric_update): - metric_update.scalar = to_json_value(accumulator.value, with_type=True) - - def translate_mean(accumulator, metric_update): if accumulator.count: metric_update.meanSum = to_json_value(accumulator.sum, with_type=True) @@ -733,20 +729,43 @@ def _use_fnapi(pipeline_options): # To enable a counter on the service, add it to this dictionary. -metric_translations = { - cy_combiners.CountCombineFn: ('sum', translate_scalar), - cy_combiners.SumInt64Fn: ('sum', translate_scalar), - cy_combiners.MinInt64Fn: ('min', translate_scalar), - cy_combiners.MaxInt64Fn: ('max', translate_scalar), - cy_combiners.MeanInt64Fn: ('mean', translate_mean), - cy_combiners.SumFloatFn: ('sum', translate_scalar), - cy_combiners.MinFloatFn: ('min', translate_scalar), - cy_combiners.MaxFloatFn: ('max', translate_scalar), - cy_combiners.MeanFloatFn: ('mean', translate_mean), - cy_combiners.AllCombineFn: ('and', translate_scalar), - cy_combiners.AnyCombineFn: ('or', translate_scalar), +structured_counter_translations = { + cy_combiners.CountCombineFn: ( + dataflow.CounterMetadata.KindValueValuesEnum.SUM, + MetricUpdateTranslators.translate_scalar_counter_int), + cy_combiners.SumInt64Fn: ( + dataflow.CounterMetadata.KindValueValuesEnum.SUM, + MetricUpdateTranslators.translate_scalar_counter_int), + cy_combiners.MinInt64Fn: ( + dataflow.CounterMetadata.KindValueValuesEnum.MIN, + MetricUpdateTranslators.translate_scalar_counter_int), + cy_combiners.MaxInt64Fn: ( + dataflow.CounterMetadata.KindValueValuesEnum.MAX, + MetricUpdateTranslators.translate_scalar_counter_int), + cy_combiners.MeanInt64Fn: ( + dataflow.CounterMetadata.KindValueValuesEnum.MEAN, + MetricUpdateTranslators.translate_scalar_mean_int), + cy_combiners.SumFloatFn: ( + dataflow.CounterMetadata.KindValueValuesEnum.SUM, + MetricUpdateTranslators.translate_scalar_counter_float), + cy_combiners.MinFloatFn: ( + dataflow.CounterMetadata.KindValueValuesEnum.MIN, + MetricUpdateTranslators.translate_scalar_counter_float), + cy_combiners.MaxFloatFn: ( + dataflow.CounterMetadata.KindValueValuesEnum.MAX, + MetricUpdateTranslators.translate_scalar_counter_float), + cy_combiners.MeanFloatFn: ( + dataflow.CounterMetadata.KindValueValuesEnum.MEAN, + MetricUpdateTranslators.translate_scalar_mean_float), + cy_combiners.AllCombineFn: ( + dataflow.CounterMetadata.KindValueValuesEnum.AND, + MetricUpdateTranslators.translate_boolean), + cy_combiners.AnyCombineFn: ( + dataflow.CounterMetadata.KindValueValuesEnum.OR, + MetricUpdateTranslators.translate_boolean), } + counter_translations = { cy_combiners.CountCombineFn: ( dataflow.NameAndKind.KindValueValuesEnum.SUM, diff --git a/sdks/python/apache_beam/utils/counters.py b/sdks/python/apache_beam/utils/counters.py index b3794618ff859..5d029dcc03fa1 100644 --- a/sdks/python/apache_beam/utils/counters.py +++ b/sdks/python/apache_beam/utils/counters.py @@ -27,6 +27,46 @@ from apache_beam.transforms import cy_combiners +class CounterName(object): + """Naming information for a counter.""" + SYSTEM = object() + USER = object() + + def __init__(self, name, stage_name=None, step_name=None, + system_name=None, namespace=None, + origin=None, output_index=None): + self.name = name + self.origin = origin or CounterName.SYSTEM + self.namespace = namespace + self.stage_name = stage_name + self.step_name = step_name + self.system_name = system_name + self.output_index = output_index + + def __hash__(self): + return hash((self.name, + self.origin, + self.namespace, + self.stage_name, + self.step_name, + self.system_name, + self.output_index)) + + def __str__(self): + return '%s' % self._str_internal() + + def __repr__(self): + return '<%s at %s>' % (self._str_internal(), hex(id(self))) + + def _str_internal(self): + if self.origin == CounterName.USER: + return 'user-%s-%s' % (self.step_name, self.name) + elif self.origin == CounterName.SYSTEM and self.output_index: + return '%s-out%s-%s' % (self.step_name, self.output_index, self.name) + else: + return '%s-%s-%s' % (self.stage_name, self.step_name, self.name) + + class Counter(object): """A counter aggregates a series of values. @@ -52,8 +92,8 @@ def __init__(self, name, combine_fn): """Creates a Counter object. Args: - name: the name of this counter. Typically has three parts: - "step-output-counter". + name: the name of this counter. It may be a string, + or a CounterName object. combine_fn: the CombineFn to use for aggregation """ self.name = name @@ -90,10 +130,6 @@ def update(self, value): self._fast_add_input(value) -# Counters that represent Accumulators have names starting with this -USER_COUNTER_PREFIX = 'user-' - - class CounterFactory(object): """Keeps track of unique counters.""" @@ -128,21 +164,6 @@ def get_counter(self, name, combine_fn): self.counters[name] = counter return counter - def get_aggregator_counter(self, step_name, aggregator): - """Returns an AggregationCounter for this step's aggregator. - - Passing in the same values will return the same counter. - - Args: - step_name: the name of this step. - aggregator: an Aggregator object. - Returns: - A new or existing counter. - """ - return self.get_counter( - '%s%s-%s' % (USER_COUNTER_PREFIX, step_name, aggregator.name), - aggregator.combine_fn) - def get_counters(self): """Returns the current set of counters. @@ -154,32 +175,3 @@ def get_counters(self): """ with self._lock: return self.counters.values() - - def get_aggregator_values(self, aggregator_or_name): - """Returns dict of step names to values of the aggregator.""" - with self._lock: - return get_aggregator_values( - aggregator_or_name, self.counters, lambda counter: counter.value()) - - -def get_aggregator_values(aggregator_or_name, counter_dict, - value_extractor=None): - """Extracts the named aggregator value from a set of counters. - - Args: - aggregator_or_name: an Aggregator object or the name of one. - counter_dict: a dict object of {name: value_wrapper} - value_extractor: a function to convert the value_wrapper into a value. - If None, no extraction is done and the value is return unchanged. - - Returns: - dict of step names to values of the aggregator. - """ - name = aggregator_or_name - if value_extractor is None: - value_extractor = lambda x: x - if not isinstance(aggregator_or_name, basestring): - name = aggregator_or_name.name - return {n: value_extractor(c) for n, c in counter_dict.iteritems() - if n.startswith(USER_COUNTER_PREFIX) - and n.endswith('-%s' % name)} From 70cd34c7fd29e096b08ecd9148a7bccedd1b5782 Mon Sep 17 00:00:00 2001 From: "chamikara@google.com" Date: Mon, 7 Aug 2017 23:47:57 -0700 Subject: [PATCH 259/346] Updates BEAM_CONTAINER_VERSION to 2.2.0. --- sdks/python/apache_beam/runners/dataflow/internal/dependency.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sdks/python/apache_beam/runners/dataflow/internal/dependency.py b/sdks/python/apache_beam/runners/dataflow/internal/dependency.py index 62c09ed141352..f5dd2bd357329 100644 --- a/sdks/python/apache_beam/runners/dataflow/internal/dependency.py +++ b/sdks/python/apache_beam/runners/dataflow/internal/dependency.py @@ -78,7 +78,7 @@ # Update this version to the next version whenever there is a change that will # require changes to legacy Dataflow worker execution environment. # This should be in the beam-[version]-[date] format, date is optional. -BEAM_CONTAINER_VERSION = 'beam-2.1.0-20170626' +BEAM_CONTAINER_VERSION = 'beam-2.2.0-20170807' # Update this version to the next version whenever there is a change that # requires changes to SDK harness container or SDK harness launcher. # This should be in the beam-[version]-[date] format, date is optional. From a17978a7825433342a7f2371f80d1612c8cda055 Mon Sep 17 00:00:00 2001 From: Mark Liu Date: Thu, 3 Aug 2017 16:33:07 -0700 Subject: [PATCH 260/346] [BEAM-1584] Add file clean up util for integration tests --- .../apache_beam/examples/wordcount_it_test.py | 4 ++ sdks/python/apache_beam/testing/test_utils.py | 18 ++++++ .../apache_beam/testing/test_utils_test.py | 59 +++++++++++++++++++ sdks/python/apache_beam/utils/retry.py | 7 +++ 4 files changed, 88 insertions(+) create mode 100644 sdks/python/apache_beam/testing/test_utils_test.py diff --git a/sdks/python/apache_beam/examples/wordcount_it_test.py b/sdks/python/apache_beam/examples/wordcount_it_test.py index 4bee1279c8f0f..8d2e73e28f6bc 100644 --- a/sdks/python/apache_beam/examples/wordcount_it_test.py +++ b/sdks/python/apache_beam/examples/wordcount_it_test.py @@ -25,6 +25,7 @@ from nose.plugins.attrib import attr from apache_beam.examples import wordcount +from apache_beam.testing.test_utils import delete_files from apache_beam.testing.pipeline_verifiers import FileChecksumMatcher from apache_beam.testing.pipeline_verifiers import PipelineStateMatcher from apache_beam.testing.test_pipeline import TestPipeline @@ -56,6 +57,9 @@ def test_wordcount_it(self): extra_opts = {'output': output, 'on_success_matcher': all_of(*pipeline_verifiers)} + # Register clean up before pipeline execution + self.addCleanup(delete_files, [output + '*']) + # Get pipeline options from command argument: --test-pipeline-options, # and start pipeline job by calling pipeline main function. wordcount.run(test_pipeline.get_full_options_as_args(**extra_opts)) diff --git a/sdks/python/apache_beam/testing/test_utils.py b/sdks/python/apache_beam/testing/test_utils.py index 9feb80e144e62..26ca03d1375df 100644 --- a/sdks/python/apache_beam/testing/test_utils.py +++ b/sdks/python/apache_beam/testing/test_utils.py @@ -24,6 +24,7 @@ import imp from mock import Mock, patch +from apache_beam.io.filesystems import FileSystems from apache_beam.utils import retry @@ -71,3 +72,20 @@ def remove_patches(): imp.reload(module) testcase.addCleanup(remove_patches) + + +@retry.with_exponential_backoff( + num_retries=3, + retry_filter=retry.retry_on_beam_io_error_filter) +def delete_files(file_paths): + """A function to clean up files or directories using ``FileSystems``. + + Glob is supported in file path and directories will be deleted recursively. + + Args: + file_paths: A list of strings contains file paths or directories. + """ + if len(file_paths) == 0: + raise RuntimeError('Clean up failed. Invalid file path: %s.' % + file_paths) + FileSystems.delete(file_paths) diff --git a/sdks/python/apache_beam/testing/test_utils_test.py b/sdks/python/apache_beam/testing/test_utils_test.py new file mode 100644 index 0000000000000..bee0bd361e27b --- /dev/null +++ b/sdks/python/apache_beam/testing/test_utils_test.py @@ -0,0 +1,59 @@ +# +# 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. +# + +"""Unittest for testing utilities,""" + +import logging +import tempfile +import unittest +from mock import patch + +from apache_beam.io.filesystem import BeamIOError +from apache_beam.io.filesystems import FileSystems +from apache_beam.testing import test_utils as utils + + +class TestUtilsTest(unittest.TestCase): + + def setUp(self): + utils.patch_retry(self, utils) + self.tmpdir = tempfile.mkdtemp() + + def test_delete_files_succeeds(self): + f = tempfile.NamedTemporaryFile(dir=self.tmpdir, delete=False) + assert FileSystems.exists(f.name) + utils.delete_files([f.name]) + assert not FileSystems.exists(f.name) + + @patch.object(FileSystems, 'delete', side_effect=BeamIOError('')) + def test_delete_files_fails_with_io_error(self, mocked_delete): + f = tempfile.NamedTemporaryFile(dir=self.tmpdir, delete=False) + assert FileSystems.exists(f.name) + + with self.assertRaises(BeamIOError): + utils.delete_files([f.name]) + self.assertTrue(mocked_delete.called) + self.assertEqual(mocked_delete.call_count, 4) + + def test_delete_files_fails_with_invalid_arg(self): + with self.assertRaises(RuntimeError): + utils.delete_files([]) + + +if __name__ == '__main__': + logging.getLogger().setLevel(logging.INFO) + unittest.main() diff --git a/sdks/python/apache_beam/utils/retry.py b/sdks/python/apache_beam/utils/retry.py index 1a8b90714d744..08223b3f1f7f1 100644 --- a/sdks/python/apache_beam/utils/retry.py +++ b/sdks/python/apache_beam/utils/retry.py @@ -31,6 +31,8 @@ import time import traceback +from apache_beam.io.filesystem import BeamIOError + # Protect against environments where apitools library is not available. # pylint: disable=wrong-import-order, wrong-import-position # TODO(sourabhbajaj): Remove the GCP specific error code to a submodule @@ -99,6 +101,11 @@ def retry_on_server_errors_and_timeout_filter(exception): return retry_on_server_errors_filter(exception) +def retry_on_beam_io_error_filter(exception): + """Filter allowing retries on Beam IO errors.""" + return isinstance(exception, BeamIOError) + + SERVER_ERROR_OR_TIMEOUT_CODES = [408, 500, 502, 503, 504, 598, 599] From cc7689c802b58a1786106f9ffb2954aa958e34e1 Mon Sep 17 00:00:00 2001 From: Etienne Chauchot Date: Thu, 10 Aug 2017 12:22:29 +0200 Subject: [PATCH 261/346] [BEAM-2760] Disable testMergingCustomWindows* validatesRunner tests in Gearpump runner --- runners/gearpump/pom.xml | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/runners/gearpump/pom.xml b/runners/gearpump/pom.xml index 3a4722fd7cef5..30bc354e5b756 100644 --- a/runners/gearpump/pom.xml +++ b/runners/gearpump/pom.xml @@ -71,7 +71,8 @@ org.apache.beam.sdk.testing.UsesSplittableParDo, org.apache.beam.sdk.testing.UsesAttemptedMetrics, org.apache.beam.sdk.testing.UsesCommittedMetrics, - org.apache.beam.sdk.testing.UsesTestStream + org.apache.beam.sdk.testing.UsesTestStream, + org.apache.beam.sdk.testing.UsesCustomWindowMerging none true From 24ee742d5c2cc4082a8eecdc01d7b34d8fc8ef54 Mon Sep 17 00:00:00 2001 From: Thomas Groh Date: Mon, 7 Aug 2017 15:05:59 -0700 Subject: [PATCH 262/346] Update Custom Window Merging Tests Update the merge windows to be order-independent. --- .../sdk/transforms/windowing/WindowTest.java | 125 +++++++++--------- 1 file changed, 61 insertions(+), 64 deletions(-) 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 5b6d0460f10c1..e2f8c260a76dd 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 @@ -38,8 +38,12 @@ import java.util.ArrayList; import java.util.Collection; import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; import java.util.List; +import java.util.Map; import java.util.Objects; +import java.util.Set; import java.util.concurrent.atomic.AtomicBoolean; import org.apache.beam.sdk.Pipeline.PipelineVisitor; import org.apache.beam.sdk.coders.Coder; @@ -581,20 +585,23 @@ public void testDisplayDataExcludesDefaults() { assertThat(data, not(hasDisplayItem("trigger"))); assertThat(data, not(hasDisplayItem("allowedLateness"))); } + @Test @Category({ValidatesRunner.class, UsesCustomWindowMerging.class}) public void testMergingCustomWindows() { Instant startInstant = new Instant(0L); List> input = new ArrayList<>(); - input.add(TimestampedValue.of("big", startInstant.plus(Duration.standardSeconds(10)))); - input.add(TimestampedValue.of("small1", startInstant.plus(Duration.standardSeconds(20)))); - // This one will be outside of bigWindow thus not merged - input.add(TimestampedValue.of("small2", startInstant.plus(Duration.standardSeconds(39)))); - PCollection inputCollection = pipeline.apply(Create.timestamped(input)); - PCollection windowedCollection = inputCollection - .apply(Window.into(new CustomWindowFn())); - PCollection count = windowedCollection - .apply(Combine.globally(Count.combineFn()).withoutDefaults()); + PCollection inputCollection = + pipeline.apply( + Create.timestamped( + TimestampedValue.of("big", startInstant.plus(Duration.standardSeconds(10))), + TimestampedValue.of("small1", startInstant.plus(Duration.standardSeconds(20))), + // This one will be outside of bigWindow thus not merged + TimestampedValue.of("small2", startInstant.plus(Duration.standardSeconds(39))))); + PCollection windowedCollection = + inputCollection.apply(Window.into(new CustomWindowFn())); + PCollection count = + windowedCollection.apply(Combine.globally(Count.combineFn()).withoutDefaults()); // "small1" and "big" elements merged into bigWindow "small2" not merged // because timestamp is not in bigWindow PAssert.that("Wrong number of elements in output collection", count).containsInAnyOrder(2L, 1L); @@ -607,61 +614,54 @@ public void testMergingCustomWindows() { @Category({ValidatesRunner.class, UsesCustomWindowMerging.class}) public void testMergingCustomWindowsKeyedCollection() { Instant startInstant = new Instant(0L); - List>> input = new ArrayList<>(); - input - .add(TimestampedValue.of(KV.of(0, "big"), startInstant.plus(Duration.standardSeconds(10)))); - input.add( - TimestampedValue.of(KV.of(1, "small1"), startInstant.plus(Duration.standardSeconds(20)))); - // This one will be outside of bigWindow thus not merged - input.add( - TimestampedValue.of(KV.of(2, "small2"), startInstant.plus(Duration.standardSeconds(39)))); - PCollection> inputCollection = pipeline.apply(Create.timestamped(input)); - PCollection> windowedCollection = inputCollection - .apply(Window.into(new CustomWindowFn>())); - PCollection count = windowedCollection - .apply(Combine.globally(Count.>combineFn()).withoutDefaults()); + PCollection> inputCollection = + pipeline.apply( + Create.timestamped( + TimestampedValue.of( + KV.of(0, "big"), startInstant.plus(Duration.standardSeconds(10))), + TimestampedValue.of( + KV.of(1, "small1"), startInstant.plus(Duration.standardSeconds(20))), + // This element is not contained within the bigWindow and not merged + TimestampedValue.of( + KV.of(2, "small2"), startInstant.plus(Duration.standardSeconds(39))))); + PCollection> windowedCollection = + inputCollection.apply(Window.into(new CustomWindowFn>())); + PCollection count = + windowedCollection.apply( + Combine.globally(Count.>combineFn()).withoutDefaults()); // "small1" and "big" elements merged into bigWindow "small2" not merged - // because timestamp is not in bigWindow + // because it is not contained in bigWindow PAssert.that("Wrong number of elements in output collection", count).containsInAnyOrder(2L, 1L); pipeline.run(); } private static class CustomWindow extends IntervalWindow { - private boolean isBig; - - CustomWindow(Instant start, Instant end) { - super(start, end); - this.isBig = false; - } - CustomWindow(Instant start, Instant end, boolean isBig) { super(start, end); this.isBig = isBig; } - @Override public boolean equals(Object o) { + @Override + public boolean equals(Object o) { if (this == o) { return true; } if (o == null || getClass() != o.getClass()) { return false; } - if (!super.equals(o)) { - return false; - } CustomWindow that = (CustomWindow) o; - return isBig == that.isBig; + return super.equals(o) && this.isBig == that.isBig; } - @Override public int hashCode() { + @Override + public int hashCode() { return Objects.hash(super.hashCode(), isBig); } } - private static class CustomWindowCoder extends - CustomCoder { + private static class CustomWindowCoder extends CustomCoder { private static final CustomWindowCoder INSTANCE = new CustomWindowCoder(); private static final Coder INTERVAL_WINDOW_CODER = IntervalWindow.getCoder(); @@ -672,8 +672,7 @@ public static CustomWindowCoder of() { } @Override - public void encode(CustomWindow window, OutputStream outStream) - throws IOException { + public void encode(CustomWindow window, OutputStream outStream) throws IOException { INTERVAL_WINDOW_CODER.encode(window, outStream); VAR_INT_CODER.encode(window.isBig ? 1 : 0, outStream); } @@ -693,12 +692,12 @@ public void verifyDeterministic() throws NonDeterministicException { } private static class CustomWindowFn extends WindowFn { - - @Override public Collection assignWindows(AssignContext c) throws Exception { + @Override + public Collection assignWindows(AssignContext c) throws Exception { String element; // It loses genericity of type T but this is not a big deal for a test. // And it allows to avoid duplicating CustomWindowFn to support PCollection - if (c.element() instanceof KV){ + if (c.element() instanceof KV) { element = ((KV) c.element()).getValue(); } else { element = (String) c.element(); @@ -706,33 +705,34 @@ private static class CustomWindowFn extends WindowFn { // put big elements in windows of 30s and small ones in windows of 5s if ("big".equals(element)) { return Collections.singletonList( - new CustomWindow(c.timestamp(), c.timestamp().plus(Duration.standardSeconds(30)), - true)); + new CustomWindow( + c.timestamp(), c.timestamp().plus(Duration.standardSeconds(30)), true)); } else { return Collections.singletonList( - new CustomWindow(c.timestamp(), c.timestamp().plus(Duration.standardSeconds(5)), - false)); + new CustomWindow( + c.timestamp(), c.timestamp().plus(Duration.standardSeconds(5)), false)); } } @Override public void mergeWindows(MergeContext c) throws Exception { - List toBeMerged = new ArrayList<>(); - CustomWindow bigWindow = null; - for (CustomWindow customWindow : c.windows()) { - if (customWindow.isBig) { - bigWindow = customWindow; - toBeMerged.add(customWindow); - } else if (bigWindow != null - && customWindow.start().isAfter(bigWindow.start()) - && customWindow.end().isBefore(bigWindow.end())) { - toBeMerged.add(customWindow); + Map> windowsToMerge = new HashMap<>(); + for (CustomWindow window : c.windows()) { + if (window.isBig) { + HashSet windows = new HashSet<>(); + windows.add(window); + windowsToMerge.put(window, windows); + } + } + for (CustomWindow window : c.windows()) { + for (Map.Entry> bigWindow : windowsToMerge.entrySet()) { + if (bigWindow.getKey().contains(window)) { + bigWindow.getValue().add(window); + } } } - // in case bigWindow has not been seen yet - if (bigWindow != null) { - // merge small windows into big windows - c.merge(toBeMerged, bigWindow); + for (Map.Entry> mergeEntry : windowsToMerge.entrySet()) { + c.merge(mergeEntry.getValue(), mergeEntry.getKey()); } } @@ -750,8 +750,5 @@ public Coder windowCoder() { public WindowMappingFn getDefaultWindowMappingFn() { throw new UnsupportedOperationException("side inputs not supported"); } - - } - } From 5e99853225baff818a7c23020b33ff25b28b23a2 Mon Sep 17 00:00:00 2001 From: "chamikara@google.com" Date: Fri, 28 Jul 2017 19:39:02 -0700 Subject: [PATCH 263/346] Adds two new Read PTransforms that can be used to read a massive number of files. textio.ReadAllFromText is for reading a PCollection of text files/file patterns. avroio.ReadAllFromAvro is for reading a PCollection of Avro files/file patterns. --- sdks/python/apache_beam/io/avroio.py | 103 ++++++++--- sdks/python/apache_beam/io/avroio_test.py | 33 +++- sdks/python/apache_beam/io/filebasedsource.py | 165 +++++++++++++++--- sdks/python/apache_beam/io/range_trackers.py | 42 +++++ .../apache_beam/io/range_trackers_test.py | 37 ++++ sdks/python/apache_beam/io/textio.py | 82 ++++++++- sdks/python/apache_beam/io/textio_test.py | 95 +++++++++- 7 files changed, 495 insertions(+), 62 deletions(-) diff --git a/sdks/python/apache_beam/io/avroio.py b/sdks/python/apache_beam/io/avroio.py index 7df9983b1f250..47ea282ceaf3f 100644 --- a/sdks/python/apache_beam/io/avroio.py +++ b/sdks/python/apache_beam/io/avroio.py @@ -14,11 +14,38 @@ # See the License for the specific language governing permissions and # limitations under the License. # -"""Implements a source for reading Avro files.""" +"""``PTransforms`` for reading from and writing to Avro files. + +Provides two read ``PTransform``s, ``ReadFromAvro`` and ``ReadAllFromAvro``, +that produces a ``PCollection`` of records. +Each record of this ``PCollection`` will contain a single record read from +an Avro file. Records that are of simple types will be mapped into +corresponding Python types. Records that are of Avro type 'RECORD' will be +mapped to Python dictionaries that comply with the schema contained in the +Avro file that contains those records. In this case, keys of each dictionary +will contain the corresponding field names and will be of type ``string`` +while the values of the dictionary will be of the type defined in the +corresponding Avro schema. + +For example, if schema of the Avro file is the following. +{"namespace": "example.avro","type": "record","name": "User","fields": +[{"name": "name", "type": "string"}, +{"name": "favorite_number", "type": ["int", "null"]}, +{"name": "favorite_color", "type": ["string", "null"]}]} + +Then records generated by read transforms will be dictionaries of the +following form. +{u'name': u'Alyssa', u'favorite_number': 256, u'favorite_color': None}). + +Additionally, this module provides a write ``PTransform`` ``WriteToAvro`` +that can be used to write a given ``PCollection`` of Python objects to an +Avro file. +""" import cStringIO import os import zlib +from functools import partial import avro from avro import datafile @@ -33,40 +60,25 @@ from apache_beam.io.iobase import Read from apache_beam.transforms import PTransform -__all__ = ['ReadFromAvro', 'WriteToAvro'] +__all__ = ['ReadFromAvro', 'ReadAllFromAvro', 'WriteToAvro'] class ReadFromAvro(PTransform): - """A ``PTransform`` for reading avro files.""" + """A ``PTransform`` for reading Avro files. + + Uses source '_AvroSource' to read a set of Avro files defined by a given + file pattern. + If '/mypath/myavrofiles*' is a file-pattern that points to a set of Avro + files, a ``PCollection`` for the records in these Avro files can be created + in the following manner. + + p = df.Pipeline(argv=pipeline_args) + records = p | 'Read' >> df.io.ReadFromAvro('/mypath/myavrofiles*') + """ def __init__(self, file_pattern=None, min_bundle_size=0, validate=True): """Initializes ``ReadFromAvro``. - Uses source '_AvroSource' to read a set of Avro files defined by a given - file pattern. - If '/mypath/myavrofiles*' is a file-pattern that points to a set of Avro - files, a ``PCollection`` for the records in these Avro files can be created - in the following manner. - p = df.Pipeline(argv=pipeline_args) - records = p | 'Read' >> df.io.ReadFromAvro('/mypath/myavrofiles*') - - Each record of this ``PCollection`` will contain a single record read from a - source. Records that are of simple types will be mapped into corresponding - Python types. Records that are of Avro type 'RECORD' will be mapped to - Python dictionaries that comply with the schema contained in the Avro file - that contains those records. In this case, keys of each dictionary - will contain the corresponding field names and will be of type ``string`` - while the values of the dictionary will be of the type defined in the - corresponding Avro schema. - For example, if schema of the Avro file is the following. - {"namespace": "example.avro","type": "record","name": "User","fields": - [{"name": "name", "type": "string"}, - {"name": "favorite_number", "type": ["int", "null"]}, - {"name": "favorite_color", "type": ["string", "null"]}]} - Then records generated by ``AvroSource`` will be dictionaries of the - following form. - {u'name': u'Alyssa', u'favorite_number': 256, u'favorite_color': None}). - Args: file_pattern: the set of files to be read. min_bundle_size: the minimum size in bytes, to be considered when @@ -84,6 +96,35 @@ def display_data(self): return {'source_dd': self._source} +class ReadAllFromAvro(PTransform): + """A ``PTransform`` for reading ``PCollection`` of Avro files. + + Uses source '_AvroSource' to read a ``PCollection`` of Avro files or + file patterns and produce a ``PCollection`` of Avro records. + """ + + DEFAULT_DESIRED_BUNDLE_SIZE = 64 * 1024 * 1024 # 64MB + + def __init__(self, min_bundle_size=0, + desired_bundle_size=DEFAULT_DESIRED_BUNDLE_SIZE): + """Initializes ``ReadAllFromAvro``. + + Args: + min_bundle_size: the minimum size in bytes, to be considered when + splitting the input into bundles. + desired_bundle_size: the desired size in bytes, to be considered when + splitting the input into bundles. + """ + source_from_file = partial( + _create_avro_source, min_bundle_size=min_bundle_size) + self._read_all_files = filebasedsource.ReadAllFiles( + True, CompressionTypes.AUTO, desired_bundle_size, min_bundle_size, + source_from_file) + + def expand(self, pvalue): + return pvalue | 'ReadAllFiles' >> self._read_all_files + + class _AvroUtils(object): @staticmethod @@ -176,6 +217,12 @@ def advance_file_past_next_sync_marker(f, sync_marker): data = f.read(buf_size) +def _create_avro_source(file_pattern=None, min_bundle_size=None): + return _AvroSource( + file_pattern=file_pattern, min_bundle_size=min_bundle_size, + validate=False) + + class _AvroBlock(object): """Represents a block of an Avro file.""" diff --git a/sdks/python/apache_beam/io/avroio_test.py b/sdks/python/apache_beam/io/avroio_test.py index 6dcf121f6fc40..969f440508546 100644 --- a/sdks/python/apache_beam/io/avroio_test.py +++ b/sdks/python/apache_beam/io/avroio_test.py @@ -22,6 +22,7 @@ import unittest import apache_beam as beam +from apache_beam import Create from apache_beam.io import iobase from apache_beam.io import avroio from apache_beam.io import filebasedsource @@ -346,11 +347,41 @@ def test_corrupted_file(self): source_test_utils.read_from_source(source, None, None) self.assertEqual(0, exn.exception.message.find('Unexpected sync marker')) - def test_source_transform(self): + def test_read_from_avro(self): path = self._write_data() with TestPipeline() as p: assert_that(p | avroio.ReadFromAvro(path), equal_to(self.RECORDS)) + def test_read_all_from_avro_single_file(self): + path = self._write_data() + with TestPipeline() as p: + assert_that(p | Create([path]) | avroio.ReadAllFromAvro(), + equal_to(self.RECORDS)) + + def test_read_all_from_avro_many_single_files(self): + path1 = self._write_data() + path2 = self._write_data() + path3 = self._write_data() + with TestPipeline() as p: + assert_that(p | Create([path1, path2, path3]) | avroio.ReadAllFromAvro(), + equal_to(self.RECORDS * 3)) + + def test_read_all_from_avro_file_pattern(self): + file_pattern = self._write_pattern(5) + with TestPipeline() as p: + assert_that(p | Create([file_pattern]) | avroio.ReadAllFromAvro(), + equal_to(self.RECORDS * 5)) + + def test_read_all_from_avro_many_file_patterns(self): + file_pattern1 = self._write_pattern(5) + file_pattern2 = self._write_pattern(2) + file_pattern3 = self._write_pattern(3) + with TestPipeline() as p: + assert_that(p + | Create([file_pattern1, file_pattern2, file_pattern3]) + | avroio.ReadAllFromAvro(), + equal_to(self.RECORDS * 10)) + def test_sink_transform(self): with tempfile.NamedTemporaryFile() as dst: path = dst.name diff --git a/sdks/python/apache_beam/io/filebasedsource.py b/sdks/python/apache_beam/io/filebasedsource.py index bb9efc4b7d021..f78bf3ff0e3d1 100644 --- a/sdks/python/apache_beam/io/filebasedsource.py +++ b/sdks/python/apache_beam/io/filebasedsource.py @@ -24,17 +24,26 @@ For an example implementation of ``FileBasedSource`` see ``avroio.AvroSource``. """ - +import uuid + +from apache_beam.transforms.core import DoFn +from apache_beam.transforms.core import ParDo +from apache_beam.transforms.core import GroupByKey +from apache_beam.transforms.core import PTransform +from apache_beam.transforms.core import FlatMap +from apache_beam.transforms.core import Map from apache_beam.internal import pickler from apache_beam.io import concat_source from apache_beam.io import iobase from apache_beam.io import range_trackers from apache_beam.io.filesystem import CompressionTypes from apache_beam.io.filesystems import FileSystems +from apache_beam.io.range_trackers import OffsetRange from apache_beam.transforms.display import DisplayDataItem from apache_beam.options.value_provider import ValueProvider from apache_beam.options.value_provider import StaticValueProvider from apache_beam.options.value_provider import check_accessible +from apache_beam.transforms.trigger import DefaultTrigger MAX_NUM_THREADS_FOR_SIZE_ESTIMATION = 25 @@ -95,12 +104,7 @@ def __init__(self, raise TypeError('compression_type must be CompressionType object but ' 'was %s' % type(compression_type)) self._compression_type = compression_type - if compression_type in (CompressionTypes.UNCOMPRESSED, - CompressionTypes.AUTO): - self._splittable = splittable - else: - # We can't split compressed files efficiently so turn off splitting. - self._splittable = False + self._splittable = splittable if validate and file_pattern.is_accessible(): self._validate() @@ -132,13 +136,10 @@ def _get_concat_source(self): continue # Ignoring empty file. # We determine splittability of this specific file. - splittable = self.splittable - if (splittable and - self._compression_type == CompressionTypes.AUTO): - compression_type = CompressionTypes.detect_compression_type( - file_name) - if compression_type != CompressionTypes.UNCOMPRESSED: - splittable = False + splittable = ( + self.splittable and + _determine_splittability_from_compression_type( + file_name, self._compression_type)) single_file_source = _SingleFileSource( file_based_source_ref, file_name, @@ -211,6 +212,14 @@ def splittable(self): return self._splittable +def _determine_splittability_from_compression_type( + file_path, compression_type): + if compression_type == CompressionTypes.AUTO: + compression_type = CompressionTypes.detect_compression_type(file_path) + + return compression_type == CompressionTypes.UNCOMPRESSED + + class _SingleFileSource(iobase.BoundedSource): """Denotes a source for a specific file type.""" @@ -244,24 +253,21 @@ def split(self, desired_bundle_size, start_offset=None, stop_offset=None): stop_offset = self._stop_offset if self._splittable: - bundle_size = max(desired_bundle_size, self._min_bundle_size) - - bundle_start = start_offset - while bundle_start < stop_offset: - bundle_stop = min(bundle_start + bundle_size, stop_offset) + splits = OffsetRange(start_offset, stop_offset).split( + desired_bundle_size, self._min_bundle_size) + for split in splits: yield iobase.SourceBundle( - bundle_stop - bundle_start, + split.stop - split.start, _SingleFileSource( # Copying this so that each sub-source gets a fresh instance. pickler.loads(pickler.dumps(self._file_based_source)), self._file_name, - bundle_start, - bundle_stop, + split.start, + split.stop, min_bundle_size=self._min_bundle_size, splittable=self._splittable), - bundle_start, - bundle_stop) - bundle_start = bundle_stop + split.start, + split.stop) else: # Returning a single sub-source with end offset set to OFFSET_INFINITY (so # that all data of the source gets read) since this source is @@ -308,3 +314,112 @@ def read(self, range_tracker): def default_output_coder(self): return self._file_based_source.default_output_coder() + + +class _ExpandIntoRanges(DoFn): + + def __init__( + self, splittable, compression_type, desired_bundle_size, min_bundle_size): + self._desired_bundle_size = desired_bundle_size + self._min_bundle_size = min_bundle_size + self._splittable = splittable + self._compression_type = compression_type + + def process(self, element, *args, **kwargs): + match_results = FileSystems.match([element]) + for metadata in match_results[0].metadata_list: + splittable = ( + self._splittable and + _determine_splittability_from_compression_type( + metadata.path, self._compression_type)) + + if splittable: + for split in OffsetRange( + 0, metadata.size_in_bytes).split( + self._desired_bundle_size, self._min_bundle_size): + yield (metadata, split) + else: + yield (metadata, OffsetRange( + 0, range_trackers.OffsetRangeTracker.OFFSET_INFINITY)) + + +# Replace following with a generic reshard transform once +# https://issues.apache.org/jira/browse/BEAM-1872 is implemented. +class _Reshard(PTransform): + + def expand(self, pvalue): + keyed_pc = (pvalue + | 'AssignKey' >> Map(lambda x: (uuid.uuid4(), x))) + if keyed_pc.windowing.windowfn.is_merging(): + raise ValueError('Transform ReadAllFiles cannot be used in the presence ' + 'of merging windows') + if not isinstance(keyed_pc.windowing.triggerfn, DefaultTrigger): + raise ValueError('Transform ReadAllFiles cannot be used in the presence ' + 'of non-trivial triggers') + + return (keyed_pc | 'GroupByKey' >> GroupByKey() + # Using FlatMap below due to the possibility of key collisions. + | 'DropKey' >> FlatMap(lambda (k, values): values)) + + +class _ReadRange(DoFn): + + def __init__(self, source_from_file): + self._source_from_file = source_from_file + + def process(self, element, *args, **kwargs): + metadata, range = element + source = self._source_from_file(metadata.path) + # Following split() operation has to be performed to create a proper + # _SingleFileSource. Otherwise what we have is a ConcatSource that contains + # a single _SingleFileSource. ConcatSource.read() expects a RangeTraker for + # sub-source range and reads full sub-sources (not byte ranges). + source = list(source.split(float('inf')))[0].source + for record in source.read(range.new_tracker()): + yield record + + +class ReadAllFiles(PTransform): + """A Read transform that reads a PCollection of files. + + Pipeline authors should not use this directly. This is to be used by Read + PTransform authors who wishes to implement file-based Read transforms that + read a PCollection of files. + """ + + def __init__( + self, splittable, compression_type, desired_bundle_size, min_bundle_size, + source_from_file): + """ + Args: + splittable: If True, files won't be split into sub-ranges. If False, files + may or may not be split into data ranges. + compression_type: A ``CompressionType`` object that specifies the + compression type of the files that will be processed. If + ``CompressionType.AUTO``, system will try to automatically + determine the compression type based on the extension of + files. + desired_bundle_size: the desired size of data ranges that should be + generated when splitting a file into data ranges. + min_bundle_size: minimum size of data ranges that should be generated when + splitting a file into data ranges. + source_from_file: a function that produces a ``BoundedSource`` given a + file name. System will use this function to generate + ``BoundedSource`` objects for file paths. Note that file + paths passed to this will be for individual files, not + for file patterns even if the ``PCollection`` of files + processed by the transform consist of file patterns. + """ + self._splittable = splittable + self._compression_type = compression_type + self._desired_bundle_size = desired_bundle_size + self._min_bundle_size = min_bundle_size + self._source_from_file = source_from_file + + def expand(self, pvalue): + return (pvalue + | 'ExpandIntoRanges' >> ParDo(_ExpandIntoRanges( + self._splittable, self._compression_type, + self._desired_bundle_size, self._min_bundle_size)) + | 'Reshard' >> _Reshard() + | 'ReadRange' >> ParDo(_ReadRange(self._source_from_file))) diff --git a/sdks/python/apache_beam/io/range_trackers.py b/sdks/python/apache_beam/io/range_trackers.py index bef77d4004768..4bd19f82087f6 100644 --- a/sdks/python/apache_beam/io/range_trackers.py +++ b/sdks/python/apache_beam/io/range_trackers.py @@ -28,6 +28,48 @@ 'OrderedPositionRangeTracker', 'UnsplittableRangeTracker'] +class OffsetRange(object): + + def __init__(self, start, stop): + if start >= stop: + raise ValueError( + 'Start offset must be smaller than the stop offset. ' + 'Received %d and %d respectively.', start, stop) + self.start = start + self.stop = stop + + def __eq__(self, other): + if not isinstance(other, OffsetRange): + return False + + return self.start == other.start and self.stop == other.stop + + def __ne__(self, other): + if not isinstance(other, OffsetRange): + return True + + return not (self.start == other.start and self.stop == other.stop) + + def split(self, desired_num_offsets_per_split, min_num_offsets_per_split=1): + current_split_start = self.start + max_split_size = max(desired_num_offsets_per_split, + min_num_offsets_per_split) + while current_split_start < self.stop: + current_split_stop = min(current_split_start + max_split_size, self.stop) + remaining = self.stop - current_split_stop + + # Avoiding a small split at the end. + if (remaining < desired_num_offsets_per_split / 4 or + remaining < min_num_offsets_per_split): + current_split_stop = self.stop + + yield OffsetRange(current_split_start, current_split_stop) + current_split_start = current_split_stop + + def new_tracker(self): + return OffsetRangeTracker(self.start, self.stop) + + class OffsetRangeTracker(iobase.RangeTracker): """A 'RangeTracker' for non-negative positions of type 'long'.""" diff --git a/sdks/python/apache_beam/io/range_trackers_test.py b/sdks/python/apache_beam/io/range_trackers_test.py index 3e926634c85f9..762d6547891ec 100644 --- a/sdks/python/apache_beam/io/range_trackers_test.py +++ b/sdks/python/apache_beam/io/range_trackers_test.py @@ -23,6 +23,43 @@ import unittest from apache_beam.io import range_trackers +from apache_beam.io.range_trackers import OffsetRange + + +class OffsetRangeTest(unittest.TestCase): + + def test_create(self): + OffsetRange(0, 10) + OffsetRange(10, 100) + + with self.assertRaises(ValueError): + OffsetRange(10, 9) + + def test_split_respects_desired_num_splits(self): + range = OffsetRange(10, 100) + splits = list(range.split(desired_num_offsets_per_split=25)) + self.assertEqual(4, len(splits)) + self.assertIn(OffsetRange(10, 35), splits) + self.assertIn(OffsetRange(35, 60), splits) + self.assertIn(OffsetRange(60, 85), splits) + self.assertIn(OffsetRange(85, 100), splits) + + def test_split_respects_min_num_splits(self): + range = OffsetRange(10, 100) + splits = list(range.split(desired_num_offsets_per_split=5, + min_num_offsets_per_split=25)) + self.assertEqual(3, len(splits)) + self.assertIn(OffsetRange(10, 35), splits) + self.assertIn(OffsetRange(35, 60), splits) + self.assertIn(OffsetRange(60, 100), splits) + + def test_split_no_small_split_at_end(self): + range = OffsetRange(10, 90) + splits = list(range.split(desired_num_offsets_per_split=25)) + self.assertEqual(3, len(splits)) + self.assertIn(OffsetRange(10, 35), splits) + self.assertIn(OffsetRange(35, 60), splits) + self.assertIn(OffsetRange(60, 90), splits) class OffsetRangeTrackerTest(unittest.TestCase): diff --git a/sdks/python/apache_beam/io/textio.py b/sdks/python/apache_beam/io/textio.py index 60e1512b02659..9c6532ee0e98c 100644 --- a/sdks/python/apache_beam/io/textio.py +++ b/sdks/python/apache_beam/io/textio.py @@ -19,19 +19,21 @@ from __future__ import absolute_import +from functools import partial import logging from apache_beam.coders import coders from apache_beam.io import filebasedsource from apache_beam.io import filebasedsink from apache_beam.io import iobase +from apache_beam.io.filebasedsource import ReadAllFiles from apache_beam.io.filesystem import CompressionTypes from apache_beam.io.iobase import Read from apache_beam.io.iobase import Write from apache_beam.transforms import PTransform from apache_beam.transforms.display import DisplayDataItem -__all__ = ['ReadFromText', 'WriteToText'] +__all__ = ['ReadFromText', 'ReadAllFromText', 'WriteToText'] class _TextSource(filebasedsource.FileBasedSource): @@ -342,8 +344,80 @@ def write_encoded_record(self, file_handle, encoded_value): file_handle.write('\n') +def _create_text_source( + file_pattern=None, min_bundle_size=None, compression_type=None, + strip_trailing_newlines=None, coder=None, skip_header_lines=None): + return _TextSource( + file_pattern=file_pattern, min_bundle_size=min_bundle_size, + compression_type=compression_type, + strip_trailing_newlines=strip_trailing_newlines, + coder=coder, validate=False, skip_header_lines=skip_header_lines) + + +class ReadAllFromText(PTransform): + """A ``PTransform`` for reading a ``PCollection`` of text files. + + Reads a ``PCollection`` of text files or file patterns and and produces a + ``PCollection`` of strings. + + Parses a text file as newline-delimited elements, by default assuming + UTF-8 encoding. Supports newline delimiters '\\n' and '\\r\\n'. + + This implementation only supports reading text encoded using UTF-8 or ASCII. + This does not support other encodings such as UTF-16 or UTF-32. + """ + + DEFAULT_DESIRED_BUNDLE_SIZE = 64 * 1024 * 1024 # 64MB + + def __init__( + self, + min_bundle_size=0, + desired_bundle_size=DEFAULT_DESIRED_BUNDLE_SIZE, + compression_type=CompressionTypes.AUTO, + strip_trailing_newlines=True, + coder=coders.StrUtf8Coder(), + skip_header_lines=0, + **kwargs): + """Initialize the ``ReadAllFromText`` transform. + + Args: + min_bundle_size: Minimum size of bundles that should be generated when + splitting this source into bundles. See ``FileBasedSource`` for more + details. + desired_bundle_size: Desired size of bundles that should be generated when + splitting this source into bundles. See ``FileBasedSource`` for more + details. + compression_type: Used to handle compressed input files. Typical value + is ``CompressionTypes.AUTO``, in which case the underlying file_path's + extension will be used to detect the compression. + strip_trailing_newlines: Indicates whether this source should remove + the newline char in each line it reads before decoding that line. + validate: flag to verify that the files exist during the pipeline + creation time. + skip_header_lines: Number of header lines to skip. Same number is skipped + from each source file. Must be 0 or higher. Large number of skipped + lines might impact performance. + coder: Coder used to decode each line. + """ + super(ReadAllFromText, self).__init__(**kwargs) + source_from_file = partial( + _create_text_source, min_bundle_size=min_bundle_size, + compression_type=compression_type, + strip_trailing_newlines=strip_trailing_newlines, coder=coder, + skip_header_lines=skip_header_lines) + self._desired_bundle_size = desired_bundle_size + self._min_bundle_size = min_bundle_size + self._compression_type = compression_type + self._read_all_files = ReadAllFiles( + True, compression_type, desired_bundle_size, min_bundle_size, + source_from_file) + + def expand(self, pvalue): + return pvalue | 'ReadAllFiles' >> self._read_all_files + + class ReadFromText(PTransform): - """A PTransform for reading text files. + """A ``PTransform`` for reading text files. Parses a text file as newline-delimited elements, by default assuming UTF-8 encoding. Supports newline delimiters '\\n' and '\\r\\n'. @@ -361,7 +435,7 @@ def __init__( validate=True, skip_header_lines=0, **kwargs): - """Initialize the ReadFromText transform. + """Initialize the ``ReadFromText`` transform. Args: file_pattern: The file path to read from as a local file path or a GCS @@ -371,7 +445,7 @@ def __init__( splitting this source into bundles. See ``FileBasedSource`` for more details. compression_type: Used to handle compressed input files. Typical value - is CompressionTypes.AUTO, in which case the underlying file_path's + is ``CompressionTypes.AUTO``, in which case the underlying file_path's extension will be used to detect the compression. strip_trailing_newlines: Indicates whether this source should remove the newline char in each line it reads before decoding that line. diff --git a/sdks/python/apache_beam/io/textio_test.py b/sdks/python/apache_beam/io/textio_test.py index 8bd7116ce295a..b29ca5a971375 100644 --- a/sdks/python/apache_beam/io/textio_test.py +++ b/sdks/python/apache_beam/io/textio_test.py @@ -27,7 +27,7 @@ import unittest import apache_beam as beam -from apache_beam.io import iobase +from apache_beam.io import iobase, ReadAllFromText import apache_beam.io.source_test_utils as source_test_utils # Importing following private classes for testing. @@ -47,6 +47,8 @@ from apache_beam.testing.util import assert_that from apache_beam.testing.util import equal_to +from apache_beam.transforms.core import Create + # TODO: Refactor code so all io tests are using same library # TestCaseWithTempDirCleanup class. @@ -334,7 +336,7 @@ def test_dynamic_work_rebalancing_mixed_eol(self): splits[0].source, splits[0].start_position, splits[0].stop_position, perform_multi_threaded_test=False) - def test_dataflow_single_file(self): + def test_read_from_text_single_file(self): file_name, expected_data = write_data(5) assert len(expected_data) == 5 pipeline = TestPipeline() @@ -342,7 +344,53 @@ def test_dataflow_single_file(self): assert_that(pcoll, equal_to(expected_data)) pipeline.run() - def test_dataflow_single_file_with_coder(self): + def test_read_all_single_file(self): + file_name, expected_data = write_data(5) + assert len(expected_data) == 5 + pipeline = TestPipeline() + pcoll = pipeline | 'Create' >> Create( + [file_name]) |'ReadAll' >> ReadAllFromText() + assert_that(pcoll, equal_to(expected_data)) + pipeline.run() + + def test_read_all_many_single_files(self): + file_name1, expected_data1 = write_data(5) + assert len(expected_data1) == 5 + file_name2, expected_data2 = write_data(10) + assert len(expected_data2) == 10 + file_name3, expected_data3 = write_data(15) + assert len(expected_data3) == 15 + expected_data = [] + expected_data.extend(expected_data1) + expected_data.extend(expected_data2) + expected_data.extend(expected_data3) + pipeline = TestPipeline() + pcoll = pipeline | 'Create' >> Create( + [file_name1, file_name2, file_name3]) |'ReadAll' >> ReadAllFromText() + assert_that(pcoll, equal_to(expected_data)) + pipeline.run() + + def test_read_all_unavailable_files_ignored(self): + file_name1, expected_data1 = write_data(5) + assert len(expected_data1) == 5 + file_name2, expected_data2 = write_data(10) + assert len(expected_data2) == 10 + file_name3, expected_data3 = write_data(15) + assert len(expected_data3) == 15 + file_name4 = "/unavailable_file" + expected_data = [] + expected_data.extend(expected_data1) + expected_data.extend(expected_data2) + expected_data.extend(expected_data3) + pipeline = TestPipeline() + pcoll = (pipeline + | 'Create' >> Create( + [file_name1, file_name2, file_name3, file_name4]) + |'ReadAll' >> ReadAllFromText()) + assert_that(pcoll, equal_to(expected_data)) + pipeline.run() + + def test_read_from_text_single_file_with_coder(self): class DummyCoder(coders.Coder): def encode(self, x): raise ValueError @@ -357,7 +405,7 @@ def decode(self, x): assert_that(pcoll, equal_to([record * 2 for record in expected_data])) pipeline.run() - def test_dataflow_file_pattern(self): + def test_read_from_text_file_pattern(self): pattern, expected_data = write_pattern([5, 3, 12, 8, 8, 4]) assert len(expected_data) == 40 pipeline = TestPipeline() @@ -365,6 +413,33 @@ def test_dataflow_file_pattern(self): assert_that(pcoll, equal_to(expected_data)) pipeline.run() + def test_read_all_file_pattern(self): + pattern, expected_data = write_pattern([5, 3, 12, 8, 8, 4]) + assert len(expected_data) == 40 + pipeline = TestPipeline() + pcoll = (pipeline + | 'Create' >> Create([pattern]) + |'ReadAll' >> ReadAllFromText()) + assert_that(pcoll, equal_to(expected_data)) + pipeline.run() + + def test_read_all_many_file_patterns(self): + pattern1, expected_data1 = write_pattern([5, 3, 12, 8, 8, 4]) + assert len(expected_data1) == 40 + pattern2, expected_data2 = write_pattern([3, 7, 9]) + assert len(expected_data2) == 19 + pattern3, expected_data3 = write_pattern([11, 20, 5, 5]) + assert len(expected_data3) == 41 + expected_data = [] + expected_data.extend(expected_data1) + expected_data.extend(expected_data2) + expected_data.extend(expected_data3) + pipeline = TestPipeline() + pcoll = pipeline | 'Create' >> Create( + [pattern1, pattern2, pattern3]) |'ReadAll' >> ReadAllFromText() + assert_that(pcoll, equal_to(expected_data)) + pipeline.run() + def test_read_auto_bzip2(self): _, lines = write_data(15) file_name = self._create_temp_file(suffix='.bz2') @@ -528,6 +603,18 @@ def test_read_gzip_concat(self): expected = ['a', 'b', 'c', 'p', 'q', 'r', 'x', 'y', 'z'] assert_that(lines, equal_to(expected)) + + def test_read_all_gzip(self): + _, lines = write_data(100) + file_name = self._create_temp_file() + with gzip.GzipFile(file_name, 'wb') as f: + f.write('\n'.join(lines)) + pipeline = TestPipeline() + pcoll = (pipeline + | Create([file_name]) + | 'ReadAll' >> ReadAllFromText( + compression_type=CompressionTypes.GZIP)) + assert_that(pcoll, equal_to(lines)) pipeline.run() def test_read_gzip_large(self): From 02eb09135ce1ae234052caf7ff2787256908f918 Mon Sep 17 00:00:00 2001 From: Alex Filatov Date: Thu, 10 Aug 2017 23:02:37 +0300 Subject: [PATCH 264/346] [BEAM-2684] Fix flaky AmqpIOTest by introducing ActiveMQ AMQP broker instead of peer-to-peer mode --- sdks/java/io/amqp/pom.xml | 22 ++++ .../apache/beam/sdk/io/amqp/AmqpIOTest.java | 112 +++++++----------- 2 files changed, 66 insertions(+), 68 deletions(-) diff --git a/sdks/java/io/amqp/pom.xml b/sdks/java/io/amqp/pom.xml index 4369bb8250f97..c28436ba19a22 100644 --- a/sdks/java/io/amqp/pom.xml +++ b/sdks/java/io/amqp/pom.xml @@ -30,6 +30,10 @@ Apache Beam :: SDKs :: Java :: IO :: AMQP IO to read and write using AMQP 1.0 protocol (http://www.amqp.org). + + 5.13.1 + + org.apache.beam @@ -96,6 +100,24 @@ beam-runners-direct-java test + + org.apache.activemq + activemq-broker + ${activemq.version} + test + + + org.apache.activemq + activemq-amqp + ${activemq.version} + test + + + org.apache.activemq.tooling + activemq-junit + ${activemq.version} + test + \ No newline at end of file diff --git a/sdks/java/io/amqp/src/test/java/org/apache/beam/sdk/io/amqp/AmqpIOTest.java b/sdks/java/io/amqp/src/test/java/org/apache/beam/sdk/io/amqp/AmqpIOTest.java index c8fe4e80f834e..947929f3a3277 100644 --- a/sdks/java/io/amqp/src/test/java/org/apache/beam/sdk/io/amqp/AmqpIOTest.java +++ b/sdks/java/io/amqp/src/test/java/org/apache/beam/sdk/io/amqp/AmqpIOTest.java @@ -20,11 +20,11 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; -import java.net.ServerSocket; import java.util.ArrayList; import java.util.Collections; import java.util.List; +import org.apache.activemq.junit.EmbeddedActiveMQBroker; import org.apache.beam.sdk.testing.PAssert; import org.apache.beam.sdk.testing.TestPipeline; import org.apache.beam.sdk.transforms.Count; @@ -33,7 +33,6 @@ import org.apache.qpid.proton.amqp.messaging.AmqpValue; import org.apache.qpid.proton.message.Message; import org.apache.qpid.proton.messenger.Messenger; -import org.junit.Before; import org.junit.Rule; import org.junit.Test; import org.junit.runner.RunWith; @@ -49,95 +48,57 @@ public class AmqpIOTest { private static final Logger LOG = LoggerFactory.getLogger(AmqpIOTest.class); - private int port; - @Rule public TestPipeline pipeline = TestPipeline.create(); - @Before - public void findFreeNetworkPort() throws Exception { - LOG.info("Finding free network port"); - ServerSocket socket = new ServerSocket(0); - port = socket.getLocalPort(); - socket.close(); - } + @Rule public EmbeddedAmqpBroker broker = new EmbeddedAmqpBroker(); @Test public void testRead() throws Exception { PCollection output = pipeline.apply(AmqpIO.read() .withMaxNumRecords(100) - .withAddresses(Collections.singletonList("amqp://~localhost:" + port))); + .withAddresses(Collections.singletonList(broker.getQueueUri("testRead")))); PAssert.thatSingleton(output.apply(Count.globally())).isEqualTo(100L); - Thread sender = new Thread() { - public void run() { - try { - Thread.sleep(500); - Messenger sender = Messenger.Factory.create(); - sender.start(); - for (int i = 0; i < 100; i++) { - Message message = Message.Factory.create(); - message.setAddress("amqp://localhost:" + port); - message.setBody(new AmqpValue("Test " + i)); - sender.put(message); - sender.send(); - } - sender.stop(); - } catch (Exception e) { - LOG.error("Sender error", e); - } - } - }; - try { - sender.start(); - pipeline.run(); - } finally { - sender.join(); + Messenger sender = Messenger.Factory.create(); + sender.start(); + for (int i = 0; i < 100; i++) { + Message message = Message.Factory.create(); + message.setAddress(broker.getQueueUri("testRead")); + message.setBody(new AmqpValue("Test " + i)); + sender.put(message); + sender.send(); } + sender.stop(); + + pipeline.run(); } @Test public void testWrite() throws Exception { - final List received = new ArrayList<>(); - Thread receiver = new Thread() { - @Override - public void run() { - try { - Messenger messenger = Messenger.Factory.create(); - messenger.start(); - messenger.subscribe("amqp://~localhost:" + port); - while (received.size() < 100) { - messenger.recv(); - while (messenger.incoming() > 0) { - Message message = messenger.get(); - LOG.info("Received: " + message.getBody().toString()); - received.add(message.getBody().toString()); - } - } - messenger.stop(); - } catch (Exception e) { - LOG.error("Receiver error", e); - } - } - }; - LOG.info("Starting AMQP receiver"); - receiver.start(); - List data = new ArrayList<>(); for (int i = 0; i < 100; i++) { Message message = Message.Factory.create(); message.setBody(new AmqpValue("Test " + i)); - message.setAddress("amqp://localhost:" + port); + message.setAddress(broker.getQueueUri("testWrite")); message.setSubject("test"); data.add(message); } pipeline.apply(Create.of(data).withCoder(AmqpMessageCoder.of())).apply(AmqpIO.write()); - LOG.info("Starting pipeline"); - try { - pipeline.run(); - } finally { - LOG.info("Join receiver thread"); - receiver.join(); + pipeline.run().waitUntilFinish(); + + List received = new ArrayList<>(); + Messenger messenger = Messenger.Factory.create(); + messenger.start(); + messenger.subscribe(broker.getQueueUri("testWrite")); + while (received.size() < 100) { + messenger.recv(); + while (messenger.incoming() > 0) { + Message message = messenger.get(); + LOG.info("Received: " + message.getBody().toString()); + received.add(message.getBody().toString()); + } } + messenger.stop(); assertEquals(100, received.size()); for (int i = 0; i < 100; i++) { @@ -145,4 +106,19 @@ public void run() { } } + private static class EmbeddedAmqpBroker extends EmbeddedActiveMQBroker { + @Override + protected void configure() { + try { + getBrokerService().addConnector("amqp://localhost:0"); + } catch (Exception e) { + throw new RuntimeException(e); + } + } + + public String getQueueUri(String queueName) { + return getBrokerService().getDefaultSocketURIString() + "/" + queueName; + } + } + } From 34d9d913fa70033c6de0799b61250c820d4336af Mon Sep 17 00:00:00 2001 From: Pablo Date: Thu, 10 Aug 2017 12:49:50 -0700 Subject: [PATCH 265/346] Updating protobuf for dataflow api --- .../clients/dataflow/dataflow_v1b3_client.py | 259 +++++++++++++++++- .../dataflow/dataflow_v1b3_messages.py | 90 ++++-- 2 files changed, 328 insertions(+), 21 deletions(-) diff --git a/sdks/python/apache_beam/runners/dataflow/internal/clients/dataflow/dataflow_v1b3_client.py b/sdks/python/apache_beam/runners/dataflow/internal/clients/dataflow/dataflow_v1b3_client.py index f280217a6bd6f..61d02730ab546 100644 --- a/sdks/python/apache_beam/runners/dataflow/internal/clients/dataflow/dataflow_v1b3_client.py +++ b/sdks/python/apache_beam/runners/dataflow/internal/clients/dataflow/dataflow_v1b3_client.py @@ -29,7 +29,7 @@ class DataflowV1b3(base_api.BaseApiClient): BASE_URL = u'https://dataflow.googleapis.com/' _PACKAGE = u'dataflow' - _SCOPES = [u'https://www.googleapis.com/auth/cloud-platform', u'https://www.googleapis.com/auth/userinfo.email'] + _SCOPES = [u'https://www.googleapis.com/auth/cloud-platform', u'https://www.googleapis.com/auth/compute', u'https://www.googleapis.com/auth/compute.readonly', u'https://www.googleapis.com/auth/userinfo.email'] _VERSION = u'v1b3' _CLIENT_ID = '1042881264118.apps.googleusercontent.com' _CLIENT_SECRET = 'x_Tw5K8nnjoRAqULM9PFAC2b' @@ -56,9 +56,11 @@ def __init__(self, url='', credentials=None, self.projects_jobs_messages = self.ProjectsJobsMessagesService(self) self.projects_jobs_workItems = self.ProjectsJobsWorkItemsService(self) self.projects_jobs = self.ProjectsJobsService(self) + self.projects_locations_jobs_debug = self.ProjectsLocationsJobsDebugService(self) self.projects_locations_jobs_messages = self.ProjectsLocationsJobsMessagesService(self) self.projects_locations_jobs_workItems = self.ProjectsLocationsJobsWorkItemsService(self) self.projects_locations_jobs = self.ProjectsLocationsJobsService(self) + self.projects_locations_templates = self.ProjectsLocationsTemplatesService(self) self.projects_locations = self.ProjectsLocationsService(self) self.projects_templates = self.ProjectsTemplatesService(self) self.projects = self.ProjectsService(self) @@ -235,6 +237,18 @@ class ProjectsJobsService(base_api.BaseApiService): def __init__(self, client): super(DataflowV1b3.ProjectsJobsService, self).__init__(client) self._method_configs = { + 'Aggregated': base_api.ApiMethodInfo( + http_method=u'GET', + method_id=u'dataflow.projects.jobs.aggregated', + ordered_params=[u'projectId'], + path_params=[u'projectId'], + query_params=[u'filter', u'location', u'pageSize', u'pageToken', u'view'], + relative_path=u'v1b3/projects/{projectId}/jobs:aggregated', + request_field='', + request_type_name=u'DataflowProjectsJobsAggregatedRequest', + response_type_name=u'ListJobsResponse', + supports_download=False, + ), 'Create': base_api.ApiMethodInfo( http_method=u'POST', method_id=u'dataflow.projects.jobs.create', @@ -300,6 +314,19 @@ def __init__(self, client): self._upload_configs = { } + def Aggregated(self, request, global_params=None): + """List the jobs of a project across all regions. + + Args: + request: (DataflowProjectsJobsAggregatedRequest) input message + global_params: (StandardQueryParameters, default: None) global arguments + Returns: + (ListJobsResponse) The response message. + """ + config = self.GetMethodConfig('Aggregated') + return self._RunMethod( + config, request, global_params=global_params) + def Create(self, request, global_params=None): """Creates a Cloud Dataflow job. @@ -340,7 +367,7 @@ def GetMetrics(self, request, global_params=None): config, request, global_params=global_params) def List(self, request, global_params=None): - """List the jobs of a project. + """List the jobs of a project in a given region. Args: request: (DataflowProjectsJobsListRequest) input message @@ -365,6 +392,69 @@ def Update(self, request, global_params=None): return self._RunMethod( config, request, global_params=global_params) + class ProjectsLocationsJobsDebugService(base_api.BaseApiService): + """Service class for the projects_locations_jobs_debug resource.""" + + _NAME = u'projects_locations_jobs_debug' + + def __init__(self, client): + super(DataflowV1b3.ProjectsLocationsJobsDebugService, self).__init__(client) + self._method_configs = { + 'GetConfig': base_api.ApiMethodInfo( + http_method=u'POST', + method_id=u'dataflow.projects.locations.jobs.debug.getConfig', + ordered_params=[u'projectId', u'location', u'jobId'], + path_params=[u'jobId', u'location', u'projectId'], + query_params=[], + relative_path=u'v1b3/projects/{projectId}/locations/{location}/jobs/{jobId}/debug/getConfig', + request_field=u'getDebugConfigRequest', + request_type_name=u'DataflowProjectsLocationsJobsDebugGetConfigRequest', + response_type_name=u'GetDebugConfigResponse', + supports_download=False, + ), + 'SendCapture': base_api.ApiMethodInfo( + http_method=u'POST', + method_id=u'dataflow.projects.locations.jobs.debug.sendCapture', + ordered_params=[u'projectId', u'location', u'jobId'], + path_params=[u'jobId', u'location', u'projectId'], + query_params=[], + relative_path=u'v1b3/projects/{projectId}/locations/{location}/jobs/{jobId}/debug/sendCapture', + request_field=u'sendDebugCaptureRequest', + request_type_name=u'DataflowProjectsLocationsJobsDebugSendCaptureRequest', + response_type_name=u'SendDebugCaptureResponse', + supports_download=False, + ), + } + + self._upload_configs = { + } + + def GetConfig(self, request, global_params=None): + """Get encoded debug configuration for component. Not cacheable. + + Args: + request: (DataflowProjectsLocationsJobsDebugGetConfigRequest) input message + global_params: (StandardQueryParameters, default: None) global arguments + Returns: + (GetDebugConfigResponse) The response message. + """ + config = self.GetMethodConfig('GetConfig') + return self._RunMethod( + config, request, global_params=global_params) + + def SendCapture(self, request, global_params=None): + """Send encoded debug capture data for component. + + Args: + request: (DataflowProjectsLocationsJobsDebugSendCaptureRequest) input message + global_params: (StandardQueryParameters, default: None) global arguments + Returns: + (SendDebugCaptureResponse) The response message. + """ + config = self.GetMethodConfig('SendCapture') + return self._RunMethod( + config, request, global_params=global_params) + class ProjectsLocationsJobsMessagesService(base_api.BaseApiService): """Service class for the projects_locations_jobs_messages resource.""" @@ -579,7 +669,7 @@ def GetMetrics(self, request, global_params=None): config, request, global_params=global_params) def List(self, request, global_params=None): - """List the jobs of a project. + """List the jobs of a project in a given region. Args: request: (DataflowProjectsLocationsJobsListRequest) input message @@ -604,6 +694,94 @@ def Update(self, request, global_params=None): return self._RunMethod( config, request, global_params=global_params) + class ProjectsLocationsTemplatesService(base_api.BaseApiService): + """Service class for the projects_locations_templates resource.""" + + _NAME = u'projects_locations_templates' + + def __init__(self, client): + super(DataflowV1b3.ProjectsLocationsTemplatesService, self).__init__(client) + self._method_configs = { + 'Create': base_api.ApiMethodInfo( + http_method=u'POST', + method_id=u'dataflow.projects.locations.templates.create', + ordered_params=[u'projectId', u'location'], + path_params=[u'location', u'projectId'], + query_params=[], + relative_path=u'v1b3/projects/{projectId}/locations/{location}/templates', + request_field=u'createJobFromTemplateRequest', + request_type_name=u'DataflowProjectsLocationsTemplatesCreateRequest', + response_type_name=u'Job', + supports_download=False, + ), + 'Get': base_api.ApiMethodInfo( + http_method=u'GET', + method_id=u'dataflow.projects.locations.templates.get', + ordered_params=[u'projectId', u'location'], + path_params=[u'location', u'projectId'], + query_params=[u'gcsPath', u'view'], + relative_path=u'v1b3/projects/{projectId}/locations/{location}/templates:get', + request_field='', + request_type_name=u'DataflowProjectsLocationsTemplatesGetRequest', + response_type_name=u'GetTemplateResponse', + supports_download=False, + ), + 'Launch': base_api.ApiMethodInfo( + http_method=u'POST', + method_id=u'dataflow.projects.locations.templates.launch', + ordered_params=[u'projectId', u'location'], + path_params=[u'location', u'projectId'], + query_params=[u'gcsPath', u'validateOnly'], + relative_path=u'v1b3/projects/{projectId}/locations/{location}/templates:launch', + request_field=u'launchTemplateParameters', + request_type_name=u'DataflowProjectsLocationsTemplatesLaunchRequest', + response_type_name=u'LaunchTemplateResponse', + supports_download=False, + ), + } + + self._upload_configs = { + } + + def Create(self, request, global_params=None): + """Creates a Cloud Dataflow job from a template. + + Args: + request: (DataflowProjectsLocationsTemplatesCreateRequest) input message + global_params: (StandardQueryParameters, default: None) global arguments + Returns: + (Job) The response message. + """ + config = self.GetMethodConfig('Create') + return self._RunMethod( + config, request, global_params=global_params) + + def Get(self, request, global_params=None): + """Get the template associated with a template. + + Args: + request: (DataflowProjectsLocationsTemplatesGetRequest) input message + global_params: (StandardQueryParameters, default: None) global arguments + Returns: + (GetTemplateResponse) The response message. + """ + config = self.GetMethodConfig('Get') + return self._RunMethod( + config, request, global_params=global_params) + + def Launch(self, request, global_params=None): + """Launch a template. + + Args: + request: (DataflowProjectsLocationsTemplatesLaunchRequest) input message + global_params: (StandardQueryParameters, default: None) global arguments + Returns: + (LaunchTemplateResponse) The response message. + """ + config = self.GetMethodConfig('Launch') + return self._RunMethod( + config, request, global_params=global_params) + class ProjectsLocationsService(base_api.BaseApiService): """Service class for the projects_locations resource.""" @@ -612,11 +790,36 @@ class ProjectsLocationsService(base_api.BaseApiService): def __init__(self, client): super(DataflowV1b3.ProjectsLocationsService, self).__init__(client) self._method_configs = { + 'WorkerMessages': base_api.ApiMethodInfo( + http_method=u'POST', + method_id=u'dataflow.projects.locations.workerMessages', + ordered_params=[u'projectId', u'location'], + path_params=[u'location', u'projectId'], + query_params=[], + relative_path=u'v1b3/projects/{projectId}/locations/{location}/WorkerMessages', + request_field=u'sendWorkerMessagesRequest', + request_type_name=u'DataflowProjectsLocationsWorkerMessagesRequest', + response_type_name=u'SendWorkerMessagesResponse', + supports_download=False, + ), } self._upload_configs = { } + def WorkerMessages(self, request, global_params=None): + """Send a worker_message to the service. + + Args: + request: (DataflowProjectsLocationsWorkerMessagesRequest) input message + global_params: (StandardQueryParameters, default: None) global arguments + Returns: + (SendWorkerMessagesResponse) The response message. + """ + config = self.GetMethodConfig('WorkerMessages') + return self._RunMethod( + config, request, global_params=global_params) + class ProjectsTemplatesService(base_api.BaseApiService): """Service class for the projects_templates resource.""" @@ -637,6 +840,30 @@ def __init__(self, client): response_type_name=u'Job', supports_download=False, ), + 'Get': base_api.ApiMethodInfo( + http_method=u'GET', + method_id=u'dataflow.projects.templates.get', + ordered_params=[u'projectId'], + path_params=[u'projectId'], + query_params=[u'gcsPath', u'location', u'view'], + relative_path=u'v1b3/projects/{projectId}/templates:get', + request_field='', + request_type_name=u'DataflowProjectsTemplatesGetRequest', + response_type_name=u'GetTemplateResponse', + supports_download=False, + ), + 'Launch': base_api.ApiMethodInfo( + http_method=u'POST', + method_id=u'dataflow.projects.templates.launch', + ordered_params=[u'projectId'], + path_params=[u'projectId'], + query_params=[u'gcsPath', u'location', u'validateOnly'], + relative_path=u'v1b3/projects/{projectId}/templates:launch', + request_field=u'launchTemplateParameters', + request_type_name=u'DataflowProjectsTemplatesLaunchRequest', + response_type_name=u'LaunchTemplateResponse', + supports_download=False, + ), } self._upload_configs = { @@ -655,6 +882,32 @@ def Create(self, request, global_params=None): return self._RunMethod( config, request, global_params=global_params) + def Get(self, request, global_params=None): + """Get the template associated with a template. + + Args: + request: (DataflowProjectsTemplatesGetRequest) input message + global_params: (StandardQueryParameters, default: None) global arguments + Returns: + (GetTemplateResponse) The response message. + """ + config = self.GetMethodConfig('Get') + return self._RunMethod( + config, request, global_params=global_params) + + def Launch(self, request, global_params=None): + """Launch a template. + + Args: + request: (DataflowProjectsTemplatesLaunchRequest) input message + global_params: (StandardQueryParameters, default: None) global arguments + Returns: + (LaunchTemplateResponse) The response message. + """ + config = self.GetMethodConfig('Launch') + return self._RunMethod( + config, request, global_params=global_params) + class ProjectsService(base_api.BaseApiService): """Service class for the projects resource.""" diff --git a/sdks/python/apache_beam/runners/dataflow/internal/clients/dataflow/dataflow_v1b3_messages.py b/sdks/python/apache_beam/runners/dataflow/internal/clients/dataflow/dataflow_v1b3_messages.py index eb88bce8c2374..e81329fca80f5 100644 --- a/sdks/python/apache_beam/runners/dataflow/internal/clients/dataflow/dataflow_v1b3_messages.py +++ b/sdks/python/apache_beam/runners/dataflow/internal/clients/dataflow/dataflow_v1b3_messages.py @@ -242,7 +242,6 @@ class ComputationTopology(_messages.Message): outputs: The outputs from the computation. stateFamilies: The state family values. systemStageName: The system stage name. - userStageName: The user stage name. """ computationId = _messages.StringField(1) @@ -251,7 +250,6 @@ class ComputationTopology(_messages.Message): outputs = _messages.MessageField('StreamLocation', 4, repeated=True) stateFamilies = _messages.MessageField('StateFamilyConfig', 5, repeated=True) systemStageName = _messages.StringField(6) - userStageName = _messages.StringField(7) class ConcatPosition(_messages.Message): @@ -522,6 +520,64 @@ class DataDiskAssignment(_messages.Message): vmInstance = _messages.StringField(2) +class DataflowProjectsJobsAggregatedRequest(_messages.Message): + """A DataflowProjectsJobsAggregatedRequest object. + + Enums: + FilterValueValuesEnum: The kind of filter to use. + ViewValueValuesEnum: Level of information requested in response. Default + is `JOB_VIEW_SUMMARY`. + + Fields: + filter: The kind of filter to use. + location: The location that contains this job. + pageSize: If there are many jobs, limit response to at most this many. The + actual number of jobs returned will be the lesser of max_responses and + an unspecified server-defined limit. + pageToken: Set this to the 'next_page_token' field of a previous response + to request additional results in a long list. + projectId: The project which owns the jobs. + view: Level of information requested in response. Default is + `JOB_VIEW_SUMMARY`. + """ + + class FilterValueValuesEnum(_messages.Enum): + """The kind of filter to use. + + Values: + UNKNOWN: + ALL: + TERMINATED: + ACTIVE: + """ + UNKNOWN = 0 + ALL = 1 + TERMINATED = 2 + ACTIVE = 3 + + class ViewValueValuesEnum(_messages.Enum): + """Level of information requested in response. Default is + `JOB_VIEW_SUMMARY`. + + Values: + JOB_VIEW_UNKNOWN: + JOB_VIEW_SUMMARY: + JOB_VIEW_ALL: + JOB_VIEW_DESCRIPTION: + """ + JOB_VIEW_UNKNOWN = 0 + JOB_VIEW_SUMMARY = 1 + JOB_VIEW_ALL = 2 + JOB_VIEW_DESCRIPTION = 3 + + filter = _messages.EnumField('FilterValueValuesEnum', 1) + location = _messages.StringField(2) + pageSize = _messages.IntegerField(3, variant=_messages.Variant.INT32) + pageToken = _messages.StringField(4) + projectId = _messages.StringField(5, required=True) + view = _messages.EnumField('ViewValueValuesEnum', 6) + + class DataflowProjectsJobsCreateRequest(_messages.Message): """A DataflowProjectsJobsCreateRequest object. @@ -3020,6 +3076,8 @@ class RuntimeEnvironment(_messages.Message): Fields: bypassTempDirValidation: Whether to bypass the safety checks for the job's temporary directory. Use with caution. + machineType: The machine type to use for the job. Defaults to the value + from the template if not specified. maxWorkers: The maximum number of Google Compute Engine instances to be made available to your pipeline during execution, from 1 to 1000. serviceAccountEmail: The email address of the service account to run the @@ -3032,10 +3090,11 @@ class RuntimeEnvironment(_messages.Message): """ bypassTempDirValidation = _messages.BooleanField(1) - maxWorkers = _messages.IntegerField(2, variant=_messages.Variant.INT32) - serviceAccountEmail = _messages.StringField(3) - tempLocation = _messages.StringField(4) - zone = _messages.StringField(5) + machineType = _messages.StringField(2) + maxWorkers = _messages.IntegerField(3, variant=_messages.Variant.INT32) + serviceAccountEmail = _messages.StringField(4) + tempLocation = _messages.StringField(5) + zone = _messages.StringField(6) class SendDebugCaptureRequest(_messages.Message): @@ -3702,7 +3761,7 @@ class Status(_messages.Message): user-facing error message is needed, put the localized message in the error details or localize it in the client. The optional error details may contain arbitrary information about the error. There is a predefined set of error - detail types in the package `google.rpc` which can be used for common error + detail types in the package `google.rpc` that can be used for common error conditions. # Language mapping The `Status` message is the logical representation of the error model, but it is not necessarily the actual wire format. When the `Status` message is exposed in different client libraries @@ -3715,8 +3774,8 @@ class Status(_messages.Message): If a service needs to return partial errors to the client, it may embed the `Status` in the normal response to indicate the partial errors. - Workflow errors. A typical workflow has multiple steps. Each step may - have a `Status` message for error reporting purpose. - Batch operations. If - a client uses batch request and batch response, the `Status` message + have a `Status` message for error reporting. - Batch operations. If a + client uses batch request and batch response, the `Status` message should be used directly inside batch response, one for each error sub- response. - Asynchronous operations. If an API call embeds asynchronous operation results in its response, the status of those operations should @@ -3729,7 +3788,7 @@ class Status(_messages.Message): Fields: code: The status code, which should be an enum value of google.rpc.Code. - details: A list of messages that carry the error details. There will be a + details: A list of messages that carry the error details. There is a common set of message types for APIs to use. message: A developer-facing error message, which should be in English. Any user-facing error message should be localized and sent in the @@ -4096,19 +4155,14 @@ class TemplateMetadata(_messages.Message): """Metadata describing a template. Fields: - bypassTempDirValidation: If true, will bypass the validation that the temp - directory is writable. This should only be used with templates for - pipelines that are guaranteed not to need to write to the temp - directory, which is subject to change based on the optimizer. description: Optional. A description of the template. name: Required. The name of the template. parameters: The parameters for the template. """ - bypassTempDirValidation = _messages.BooleanField(1) - description = _messages.StringField(2) - name = _messages.StringField(3) - parameters = _messages.MessageField('ParameterMetadata', 4, repeated=True) + description = _messages.StringField(1) + name = _messages.StringField(2) + parameters = _messages.MessageField('ParameterMetadata', 3, repeated=True) class TopologyConfig(_messages.Message): From e1baf55d82fcc4a3951057b2321f77319d88b6c3 Mon Sep 17 00:00:00 2001 From: David Cavazos Date: Fri, 21 Jul 2017 09:58:11 -0700 Subject: [PATCH 266/346] Makes Python API reference generation more strict --- sdks/python/apache_beam/__init__.py | 47 ++-- .../apache_beam/internal/gcp/json_value.py | 45 +-- sdks/python/apache_beam/io/avroio.py | 79 ++++-- sdks/python/apache_beam/io/filebasedsink.py | 18 +- sdks/python/apache_beam/io/filebasedsource.py | 67 +++-- sdks/python/apache_beam/io/filesystem.py | 27 +- sdks/python/apache_beam/io/gcp/bigquery.py | 257 ++++++++++-------- sdks/python/apache_beam/io/gcp/gcsio.py | 12 +- sdks/python/apache_beam/io/range_trackers.py | 12 +- .../apache_beam/io/source_test_utils.py | 88 +++--- sdks/python/apache_beam/io/textio.py | 121 +++++---- sdks/python/apache_beam/pipeline.py | 89 +++--- sdks/python/apache_beam/runners/runner.py | 31 ++- .../apache_beam/testing/test_pipeline.py | 48 ++-- sdks/python/apache_beam/transforms/core.py | 165 ++++++----- sdks/python/apache_beam/transforms/display.py | 87 +++--- .../apache_beam/transforms/ptransform.py | 61 +++-- .../apache_beam/typehints/decorators.py | 104 ++++--- .../typehints/native_type_compatibility.py | 7 +- .../python/apache_beam/typehints/typehints.py | 38 +-- sdks/python/generate_pydoc.sh | 134 +++++++-- sdks/python/tox.ini | 1 + 22 files changed, 953 insertions(+), 585 deletions(-) diff --git a/sdks/python/apache_beam/__init__.py b/sdks/python/apache_beam/__init__.py index 8b772c9e875b1..791ebb7a342e8 100644 --- a/sdks/python/apache_beam/__init__.py +++ b/sdks/python/apache_beam/__init__.py @@ -15,11 +15,12 @@ # limitations under the License. # -"""Apache Beam SDK for Python. +""" +Apache Beam SDK for Python +========================== -Apache Beam -provides a simple, powerful programming model for building both batch -and streaming parallel data processing pipelines. +`Apache Beam `_ provides a simple, powerful programming +model for building both batch and streaming parallel data processing pipelines. The Apache Beam SDK for Python provides access to Apache Beam capabilities from the Python programming language. @@ -33,32 +34,40 @@ -------- The key concepts in this programming model are -* PCollection: represents a collection of data, which could be - bounded or unbounded in size. -* PTransform: represents a computation that transforms input - PCollections into output PCollections. -* Pipeline: manages a directed acyclic graph of PTransforms and - PCollections that is ready for execution. -* Runner: specifies where and how the Pipeline should execute. -* Reading and Writing Data: your pipeline can read from an external - source and write to an external data sink. +* :class:`~apache_beam.pvalue.PCollection`: represents a collection of data, + which could be bounded or unbounded in size. +* :class:`~apache_beam.transforms.ptransform.PTransform`: represents a + computation that transforms input PCollections into output PCollections. +* :class:`~apache_beam.pipeline.Pipeline`: manages a directed acyclic graph of + :class:`~apache_beam.transforms.ptransform.PTransform` s and + :class:`~apache_beam.pvalue.PCollection` s that is ready for execution. +* :class:`~apache_beam.runners.runner.PipelineRunner`: specifies where and how + the pipeline should execute. +* :class:`~apache_beam.io.iobase.Read`: read from an external source. +* :class:`~apache_beam.io.iobase.Write`: write to an external data sink. Typical usage ------------- At the top of your source file:: - import apache_beam as beam + import apache_beam as beam After this import statement -* transform classes are available as beam.FlatMap, beam.GroupByKey, etc. -* Pipeline class is available as beam.Pipeline -* text read/write transforms are available as beam.io.ReadfromText, - beam.io.WriteToText +* Transform classes are available as + :class:`beam.FlatMap `, + :class:`beam.GroupByKey `, etc. +* Pipeline class is available as + :class:`beam.Pipeline ` +* Text read/write transforms are available as + :class:`beam.io.ReadFromText `, + :class:`beam.io.WriteToText `. Examples -------- -The examples subdirectory has some examples. +The `examples subdirectory +`_ +has some examples. """ diff --git a/sdks/python/apache_beam/internal/gcp/json_value.py b/sdks/python/apache_beam/internal/gcp/json_value.py index 59f8b60dd9d71..167b173b1f84d 100644 --- a/sdks/python/apache_beam/internal/gcp/json_value.py +++ b/sdks/python/apache_beam/internal/gcp/json_value.py @@ -41,11 +41,12 @@ def get_typed_value_descriptor(obj): obj: A basestring, bool, int, or float to be converted. Returns: - A dictionary containing the keys '@type' and 'value' with the value for - the @type of appropriate type. + A dictionary containing the keys ``@type`` and ``value`` with the value for + the ``@type`` of appropriate type. Raises: - TypeError: if the Python object has a type that is not supported. + ~exceptions.TypeError: if the Python object has a type that is not + supported. """ if isinstance(obj, basestring): type_name = 'Text' @@ -66,21 +67,23 @@ def to_json_value(obj, with_type=False): Converts Python objects into extra_types.JsonValue objects. Args: - obj: Python object to be converted. Can be 'None'. - with_type: If true then the basic types (string, int, float, bool) will - be wrapped in @type/value dictionaries. Otherwise the straight value is - encoded into a JsonValue. + obj: Python object to be converted. Can be :data:`None`. + with_type: If true then the basic types (``string``, ``int``, ``float``, + ``bool``) will be wrapped in ``@type:value`` dictionaries. Otherwise the + straight value is encoded into a ``JsonValue``. Returns: - A JsonValue object using JsonValue, JsonArray and JsonObject types for the - corresponding values, lists, or dictionaries. + A ``JsonValue`` object using ``JsonValue``, ``JsonArray`` and ``JsonObject`` + types for the corresponding values, lists, or dictionaries. Raises: - TypeError: if the Python object contains a type that is not supported. + ~exceptions.TypeError: if the Python object contains a type that is not + supported. - The types supported are str, bool, list, tuple, dict, and None. The Dataflow - API requires JsonValue(s) in many places, and it is quite convenient to be - able to specify these hierarchical objects using Python syntax. + The types supported are ``str``, ``bool``, ``list``, ``tuple``, ``dict``, and + ``None``. The Dataflow API requires JsonValue(s) in many places, and it is + quite convenient to be able to specify these hierarchical objects using + Python syntax. """ if obj is None: return extra_types.JsonValue(is_null=True) @@ -121,21 +124,23 @@ def to_json_value(obj, with_type=False): def from_json_value(v): """For internal use only; no backwards-compatibility guarantees. - Converts extra_types.JsonValue objects into Python objects. + Converts ``extra_types.JsonValue`` objects into Python objects. Args: - v: JsonValue object to be converted. + v: ``JsonValue`` object to be converted. Returns: A Python object structured as values, lists, and dictionaries corresponding - to JsonValue, JsonArray and JsonObject types. + to ``JsonValue``, ``JsonArray`` and ``JsonObject`` types. Raises: - TypeError: if the JsonValue object contains a type that is not supported. + ~exceptions.TypeError: if the ``JsonValue`` object contains a type that is + not supported. - The types supported are str, bool, list, dict, and None. The Dataflow API - returns JsonValue(s) in many places and it is quite convenient to be able to - convert these hierarchical objects to much simpler Python objects. + The types supported are ``str``, ``bool``, ``list``, ``dict``, and ``None``. + The Dataflow API returns JsonValue(s) in many places and it is quite + convenient to be able to convert these hierarchical objects to much simpler + Python objects. """ if isinstance(v, extra_types.JsonValue): if v.string_value is not None: diff --git a/sdks/python/apache_beam/io/avroio.py b/sdks/python/apache_beam/io/avroio.py index 47ea282ceaf3f..cb14c65421760 100644 --- a/sdks/python/apache_beam/io/avroio.py +++ b/sdks/python/apache_beam/io/avroio.py @@ -64,27 +64,74 @@ class ReadFromAvro(PTransform): - """A ``PTransform`` for reading Avro files. + """A :class:`~apache_beam.transforms.ptransform.PTransform` for reading avro + files.""" - Uses source '_AvroSource' to read a set of Avro files defined by a given - file pattern. - If '/mypath/myavrofiles*' is a file-pattern that points to a set of Avro - files, a ``PCollection`` for the records in these Avro files can be created - in the following manner. + def __init__(self, file_pattern=None, min_bundle_size=0, validate=True): + """Initializes :class:`ReadFromAvro`. - p = df.Pipeline(argv=pipeline_args) - records = p | 'Read' >> df.io.ReadFromAvro('/mypath/myavrofiles*') - """ + Uses source :class:`~apache_beam.io._AvroSource` to read a set of Avro + files defined by a given file pattern. - def __init__(self, file_pattern=None, min_bundle_size=0, validate=True): - """Initializes ``ReadFromAvro``. + If ``/mypath/myavrofiles*`` is a file-pattern that points to a set of Avro + files, a :class:`~apache_beam.pvalue.PCollection` for the records in + these Avro files can be created in the following manner. + + .. testcode:: + + with beam.Pipeline() as p: + records = p | 'Read' >> beam.io.ReadFromAvro('/mypath/myavrofiles*') + + .. NOTE: We're not actually interested in this error; but if we get here, + it means that the way of calling this transform hasn't changed. + + .. testoutput:: + :hide: + + Traceback (most recent call last): + ... + IOError: No files found based on the file pattern + + Each record of this :class:`~apache_beam.pvalue.PCollection` will contain + a single record read from a source. Records that are of simple types will be + mapped into corresponding Python types. Records that are of Avro type + ``RECORD`` will be mapped to Python dictionaries that comply with the schema + contained in the Avro file that contains those records. In this case, keys + of each dictionary will contain the corresponding field names and will be of + type :class:`str` while the values of the dictionary will be of the type + defined in the corresponding Avro schema. + + For example, if schema of the Avro file is the following. :: + + { + "namespace": "example.avro", + "type": "record", + "name": "User", + "fields": [ + + {"name": "name", + "type": "string"}, + + {"name": "favorite_number", + "type": ["int", "null"]}, + + {"name": "favorite_color", + "type": ["string", "null"]} + + ] + } + + Then records generated by :class:`~apache_beam.io._AvroSource` will be + dictionaries of the following form. :: + + {u'name': u'Alyssa', u'favorite_number': 256, u'favorite_color': None}). Args: - file_pattern: the set of files to be read. - min_bundle_size: the minimum size in bytes, to be considered when - splitting the input into bundles. - validate: flag to verify that the files exist during the pipeline - creation time. + file_pattern (str): the file glob to read + min_bundle_size (int): the minimum size in bytes, to be considered when + splitting the input into bundles. + validate (bool): flag to verify that the files exist during the pipeline + creation time. """ super(ReadFromAvro, self).__init__() self._source = _AvroSource(file_pattern, min_bundle_size, validate=validate) diff --git a/sdks/python/apache_beam/io/filebasedsink.py b/sdks/python/apache_beam/io/filebasedsink.py index 76c09fc9b7052..eb99d08490175 100644 --- a/sdks/python/apache_beam/io/filebasedsink.py +++ b/sdks/python/apache_beam/io/filebasedsink.py @@ -44,12 +44,13 @@ class FileBasedSink(iobase.Sink): """A sink to a GCS or local files. To implement a file-based sink, extend this class and override - either ``write_record()`` or ``write_encoded_record()``. + either :meth:`.write_record()` or :meth:`.write_encoded_record()`. - If needed, also overwrite ``open()`` and/or ``close()`` to customize the - file handling or write headers and footers. + If needed, also overwrite :meth:`.open()` and/or :meth:`.close()` to customize + the file handling or write headers and footers. - The output of this write is a PCollection of all written shards. + The output of this write is a :class:`~apache_beam.pvalue.PCollection` of + all written shards. """ # Max number of threads to be used for renaming. @@ -65,9 +66,12 @@ def __init__(self, compression_type=CompressionTypes.AUTO): """ Raises: - TypeError: if file path parameters are not a string or ValueProvider, - or if compression_type is not member of CompressionTypes. - ValueError: if shard_name_template is not of expected format. + ~exceptions.TypeError: if file path parameters are not a :class:`str` or + :class:`~apache_beam.options.value_provider.ValueProvider`, or if + **compression_type** is not member of + :class:`~apache_beam.io.filesystem.CompressionTypes`. + ~exceptions.ValueError: if **shard_name_template** is not of expected + format. """ if not isinstance(file_path_prefix, (basestring, ValueProvider)): raise TypeError('file_path_prefix must be a string or ValueProvider;' diff --git a/sdks/python/apache_beam/io/filebasedsource.py b/sdks/python/apache_beam/io/filebasedsource.py index f78bf3ff0e3d1..649693076c5fb 100644 --- a/sdks/python/apache_beam/io/filebasedsource.py +++ b/sdks/python/apache_beam/io/filebasedsource.py @@ -17,12 +17,13 @@ """A framework for developing sources for new file types. -To create a source for a new file type a sub-class of ``FileBasedSource`` should -be created. Sub-classes of ``FileBasedSource`` must implement the method -``FileBasedSource.read_records()``. Please read the documentation of that method -for more details. +To create a source for a new file type a sub-class of :class:`FileBasedSource` +should be created. Sub-classes of :class:`FileBasedSource` must implement the +method :meth:`FileBasedSource.read_records()`. Please read the documentation of +that method for more details. -For an example implementation of ``FileBasedSource`` see ``avroio.AvroSource``. +For an example implementation of :class:`FileBasedSource` see +:class:`~apache_beam.io._AvroSource`. """ import uuid @@ -51,7 +52,8 @@ class FileBasedSource(iobase.BoundedSource): - """A ``BoundedSource`` for reading a file glob of a given type.""" + """A :class:`~apache_beam.io.iobase.BoundedSource` for reading a file glob of + a given type.""" MIN_NUMBER_OF_FILES_TO_STAT = 100 MIN_FRACTION_OF_FILES_TO_STAT = 0.01 @@ -62,31 +64,40 @@ def __init__(self, compression_type=CompressionTypes.AUTO, splittable=True, validate=True): - """Initializes ``FileBasedSource``. + """Initializes :class:`FileBasedSource`. Args: - file_pattern: the file glob to read a string or a ValueProvider - (placeholder to inject a runtime value). - min_bundle_size: minimum size of bundles that should be generated when - performing initial splitting on this source. - compression_type: compression type to use - splittable: whether FileBasedSource should try to logically split a single - file into data ranges so that different parts of the same file - can be read in parallel. If set to False, FileBasedSource will - prevent both initial and dynamic splitting of sources for - single files. File patterns that represent multiple files may - still get split into sources for individual files. Even if set - to True by the user, FileBasedSource may choose to not split - the file, for example, for compressed files where currently - it is not possible to efficiently read a data range without - decompressing the whole file. - validate: Boolean flag to verify that the files exist during the pipeline - creation time. + file_pattern (str): the file glob to read a string or a + :class:`~apache_beam.options.value_provider.ValueProvider` + (placeholder to inject a runtime value). + min_bundle_size (str): minimum size of bundles that should be generated + when performing initial splitting on this source. + compression_type (str): Used to handle compressed output files. + Typical value is :attr:`CompressionTypes.AUTO + `, + in which case the final file path's extension will be used to detect + the compression. + splittable (bool): whether :class:`FileBasedSource` should try to + logically split a single file into data ranges so that different parts + of the same file can be read in parallel. If set to :data:`False`, + :class:`FileBasedSource` will prevent both initial and dynamic splitting + of sources for single files. File patterns that represent multiple files + may still get split into sources for individual files. Even if set to + :data:`True` by the user, :class:`FileBasedSource` may choose to not + split the file, for example, for compressed files where currently it is + not possible to efficiently read a data range without decompressing the + whole file. + validate (bool): Boolean flag to verify that the files exist during the + pipeline creation time. + Raises: - TypeError: when compression_type is not valid or if file_pattern is not a - string or a ValueProvider. - ValueError: when compression and splittable files are specified. - IOError: when the file pattern specified yields an empty result. + ~exceptions.TypeError: when **compression_type** is not valid or if + **file_pattern** is not a :class:`str` or a + :class:`~apache_beam.options.value_provider.ValueProvider`. + ~exceptions.ValueError: when compression and splittable files are + specified. + ~exceptions.IOError: when the file pattern specified yields an empty + result. """ if not isinstance(file_pattern, (basestring, ValueProvider)): diff --git a/sdks/python/apache_beam/io/filesystem.py b/sdks/python/apache_beam/io/filesystem.py index ef3040c9e727f..5804d0048af00 100644 --- a/sdks/python/apache_beam/io/filesystem.py +++ b/sdks/python/apache_beam/io/filesystem.py @@ -299,23 +299,28 @@ def seek(self, offset, whence=os.SEEK_SET): """Set the file's current offset. Seeking behavior: - * seeking from the end (SEEK_END) the whole file is decompressed once to - determine it's size. Therefore it is preferred to use - SEEK_SET or SEEK_CUR to avoid the processing overhead - * seeking backwards from the current position rewinds the file to 0 + + * seeking from the end :data:`os.SEEK_END` the whole file is decompressed + once to determine it's size. Therefore it is preferred to use + :data:`os.SEEK_SET` or :data:`os.SEEK_CUR` to avoid the processing + overhead + * seeking backwards from the current position rewinds the file to ``0`` and decompresses the chunks to the requested offset * seeking is only supported in files opened for reading - * if the new offset is out of bound, it is adjusted to either 0 or EOF. + * if the new offset is out of bound, it is adjusted to either ``0`` or + ``EOF``. Args: - offset: seek offset in the uncompressed content represented as number - whence: seek mode. Supported modes are os.SEEK_SET (absolute seek), - os.SEEK_CUR (seek relative to the current position), and os.SEEK_END - (seek relative to the end, offset should be negative). + offset (int): seek offset in the uncompressed content represented as + number + whence (int): seek mode. Supported modes are :data:`os.SEEK_SET` + (absolute seek), :data:`os.SEEK_CUR` (seek relative to the current + position), and :data:`os.SEEK_END` (seek relative to the end, offset + should be negative). Raises: - IOError: When this buffer is closed. - ValueError: When whence is invalid or the file is not seekable + ~exceptions.IOError: When this buffer is closed. + ~exceptions.ValueError: When whence is invalid or the file is not seekable """ if whence == os.SEEK_SET: absolute_offset = offset diff --git a/sdks/python/apache_beam/io/gcp/bigquery.py b/sdks/python/apache_beam/io/gcp/bigquery.py index db6715ad714d8..33d67bf7c53ad 100644 --- a/sdks/python/apache_beam/io/gcp/bigquery.py +++ b/sdks/python/apache_beam/io/gcp/bigquery.py @@ -330,45 +330,49 @@ class BigQuerySource(dataflow_io.NativeSource): def __init__(self, table=None, dataset=None, project=None, query=None, validate=False, coder=None, use_standard_sql=False, flatten_results=True): - """Initialize a BigQuerySource. + """Initialize a :class:`BigQuerySource`. Args: - table: The ID of a BigQuery table. If specified all data of the table - will be used as input of the current source. The ID must contain only - letters (a-z, A-Z), numbers (0-9), or underscores (_). If dataset - and query arguments are None then the table argument must contain the - entire table reference specified as: 'DATASET.TABLE' or - 'PROJECT:DATASET.TABLE'. - dataset: The ID of the dataset containing this table or null if the table - reference is specified entirely by the table argument or a query is - specified. - project: The ID of the project containing this table or null if the table - reference is specified entirely by the table argument or a query is - specified. - query: A query to be used instead of arguments table, dataset, and + table (str): The ID of a BigQuery table. If specified all data of the + table will be used as input of the current source. The ID must contain + only letters ``a-z``, ``A-Z``, numbers ``0-9``, or underscores + ``_``. If dataset and query arguments are :data:`None` then the table + argument must contain the entire table reference specified as: + ``'DATASET.TABLE'`` or ``'PROJECT:DATASET.TABLE'``. + dataset (str): The ID of the dataset containing this table or + :data:`None` if the table reference is specified entirely by the table + argument or a query is specified. + project (str): The ID of the project containing this table or + :data:`None` if the table reference is specified entirely by the table + argument or a query is specified. + query (str): A query to be used instead of arguments table, dataset, and project. - validate: If true, various checks will be done when source gets - initialized (e.g., is table present?). This should be True for most - scenarios in order to catch errors as early as possible (pipeline - construction instead of pipeline execution). It should be False if the - table is created during pipeline execution by a previous step. - coder: The coder for the table rows if serialized to disk. If None, then - the default coder is RowAsDictJsonCoder, which will interpret every line - in a file as a JSON serialized dictionary. This argument needs a value - only in special cases when returning table rows as dictionaries is not - desirable. - use_standard_sql: Specifies whether to use BigQuery's standard - SQL dialect for this query. The default value is False. If set to True, - the query will use BigQuery's updated SQL dialect with improved - standards compliance. This parameter is ignored for table inputs. - flatten_results: Flattens all nested and repeated fields in the - query results. The default value is true. + validate (bool): If :data:`True`, various checks will be done when source + gets initialized (e.g., is table present?). This should be + :data:`True` for most scenarios in order to catch errors as early as + possible (pipeline construction instead of pipeline execution). It + should be :data:`False` if the table is created during pipeline + execution by a previous step. + coder (~apache_beam.coders.coders.Coder): The coder for the table + rows if serialized to disk. If :data:`None`, then the default coder is + :class:`~apache_beam.io.gcp.bigquery.RowAsDictJsonCoder`, + which will interpret every line in a file as a JSON serialized + dictionary. This argument needs a value only in special cases when + returning table rows as dictionaries is not desirable. + use_standard_sql (bool): Specifies whether to use BigQuery's standard SQL + dialect for this query. The default value is :data:`False`. + If set to :data:`True`, the query will use BigQuery's updated SQL + dialect with improved standards compliance. + This parameter is ignored for table inputs. + flatten_results (bool): Flattens all nested and repeated fields in the + query results. The default value is :data:`True`. Raises: - ValueError: if any of the following is true - (1) the table reference as a string does not match the expected format - (2) neither a table nor a query is specified - (3) both a table and a query is specified. + ~exceptions.ValueError: if any of the following is true: + + 1) the table reference as a string does not match the expected format + 2) neither a table nor a query is specified + 3) both a table and a query is specified. """ # Import here to avoid adding the dependency for local running scenarios. @@ -439,46 +443,62 @@ def __init__(self, table, dataset=None, project=None, schema=None, """Initialize a BigQuerySink. Args: - table: The ID of the table. The ID must contain only letters - (a-z, A-Z), numbers (0-9), or underscores (_). If dataset argument is - None then the table argument must contain the entire table reference - specified as: 'DATASET.TABLE' or 'PROJECT:DATASET.TABLE'. - dataset: The ID of the dataset containing this table or null if the table - reference is specified entirely by the table argument. - project: The ID of the project containing this table or null if the table - reference is specified entirely by the table argument. - schema: The schema to be used if the BigQuery table to write has to be - created. This can be either specified as a 'bigquery.TableSchema' object - or a single string of the form 'field1:type1,field2:type2,field3:type3' - that defines a comma separated list of fields. Here 'type' should - specify the BigQuery type of the field. Single string based schemas do - not support nested fields, repeated fields, or specifying a BigQuery - mode for fields (mode will always be set to 'NULLABLE'). - create_disposition: A string describing what happens if the table does not - exist. Possible values are: - - BigQueryDisposition.CREATE_IF_NEEDED: create if does not exist. - - BigQueryDisposition.CREATE_NEVER: fail the write if does not exist. - write_disposition: A string describing what happens if the table has - already some data. Possible values are: - - BigQueryDisposition.WRITE_TRUNCATE: delete existing rows. - - BigQueryDisposition.WRITE_APPEND: add to existing rows. - - BigQueryDisposition.WRITE_EMPTY: fail the write if table not empty. - validate: If true, various checks will be done when sink gets - initialized (e.g., is table present given the disposition arguments?). - This should be True for most scenarios in order to catch errors as early - as possible (pipeline construction instead of pipeline execution). It - should be False if the table is created during pipeline execution by a - previous step. - coder: The coder for the table rows if serialized to disk. If None, then - the default coder is RowAsDictJsonCoder, which will interpret every - element written to the sink as a dictionary that will be JSON serialized - as a line in a file. This argument needs a value only in special cases - when writing table rows as dictionaries is not desirable. + table (str): The ID of the table. The ID must contain only letters + ``a-z``, ``A-Z``, numbers ``0-9``, or underscores ``_``. If + **dataset** argument is :data:`None` then the table argument must + contain the entire table reference specified as: ``'DATASET.TABLE'`` or + ``'PROJECT:DATASET.TABLE'``. + dataset (str): The ID of the dataset containing this table or + :data:`None` if the table reference is specified entirely by the table + argument. + project (str): The ID of the project containing this table or + :data:`None` if the table reference is specified entirely by the table + argument. + schema (str): The schema to be used if the BigQuery table to write has + to be created. This can be either specified as a + :class:`~apache_beam.io.gcp.internal.clients.bigquery.\ +bigquery_v2_messages.TableSchema` object or a single string of the form + ``'field1:type1,field2:type2,field3:type3'`` that defines a comma + separated list of fields. Here ``'type'`` should specify the BigQuery + type of the field. Single string based schemas do not support nested + fields, repeated fields, or specifying a BigQuery mode for fields (mode + will always be set to ``'NULLABLE'``). + create_disposition (BigQueryDisposition): A string describing what + happens if the table does not exist. Possible values are: + + * :attr:`BigQueryDisposition.CREATE_IF_NEEDED`: create if does not + exist. + * :attr:`BigQueryDisposition.CREATE_NEVER`: fail the write if does not + exist. + + write_disposition (BigQueryDisposition): A string describing what + happens if the table has already some data. Possible values are: + + * :attr:`BigQueryDisposition.WRITE_TRUNCATE`: delete existing rows. + * :attr:`BigQueryDisposition.WRITE_APPEND`: add to existing rows. + * :attr:`BigQueryDisposition.WRITE_EMPTY`: fail the write if table not + empty. + + validate (bool): If :data:`True`, various checks will be done when sink + gets initialized (e.g., is table present given the disposition + arguments?). This should be :data:`True` for most scenarios in order to + catch errors as early as possible (pipeline construction instead of + pipeline execution). It should be :data:`False` if the table is created + during pipeline execution by a previous step. + coder (~apache_beam.coders.coders.Coder): The coder for the + table rows if serialized to disk. If :data:`None`, then the default + coder is :class:`~apache_beam.io.gcp.bigquery.RowAsDictJsonCoder`, + which will interpret every element written to the sink as a dictionary + that will be JSON serialized as a line in a file. This argument needs a + value only in special cases when writing table rows as dictionaries is + not desirable. Raises: - TypeError: if the schema argument is not a string or a TableSchema object. - ValueError: if the table reference as a string does not match the expected - format. + ~exceptions.TypeError: if the schema argument is not a :class:`str` or a + :class:`~apache_beam.io.gcp.internal.clients.bigquery.\ +bigquery_v2_messages.TableSchema` object. + ~exceptions.ValueError: if the table reference as a string does not + match the expected format. """ # Import here to avoid adding the dependency for local running scenarios. @@ -1261,32 +1281,47 @@ def __init__(self, table, dataset=None, project=None, schema=None, """Initialize a WriteToBigQuery transform. Args: - table: The ID of the table. The ID must contain only letters - (a-z, A-Z), numbers (0-9), or underscores (_). If dataset argument is - None then the table argument must contain the entire table reference - specified as: 'DATASET.TABLE' or 'PROJECT:DATASET.TABLE'. - dataset: The ID of the dataset containing this table or null if the table - reference is specified entirely by the table argument. - project: The ID of the project containing this table or null if the table - reference is specified entirely by the table argument. - schema: The schema to be used if the BigQuery table to write has to be - created. This can be either specified as a 'bigquery.TableSchema' object - or a single string of the form 'field1:type1,field2:type2,field3:type3' - that defines a comma separated list of fields. Here 'type' should - specify the BigQuery type of the field. Single string based schemas do - not support nested fields, repeated fields, or specifying a BigQuery - mode for fields (mode will always be set to 'NULLABLE'). - create_disposition: A string describing what happens if the table does not - exist. Possible values are: - - BigQueryDisposition.CREATE_IF_NEEDED: create if does not exist. - - BigQueryDisposition.CREATE_NEVER: fail the write if does not exist. - write_disposition: A string describing what happens if the table has - already some data. Possible values are: - - BigQueryDisposition.WRITE_TRUNCATE: delete existing rows. - - BigQueryDisposition.WRITE_APPEND: add to existing rows. - - BigQueryDisposition.WRITE_EMPTY: fail the write if table not empty. + table (str): The ID of the table. The ID must contain only letters + ``a-z``, ``A-Z``, numbers ``0-9``, or underscores ``_``. If dataset + argument is :data:`None` then the table argument must contain the + entire table reference specified as: ``'DATASET.TABLE'`` or + ``'PROJECT:DATASET.TABLE'``. + dataset (str): The ID of the dataset containing this table or + :data:`None` if the table reference is specified entirely by the table + argument. + project (str): The ID of the project containing this table or + :data:`None` if the table reference is specified entirely by the table + argument. + schema (str): The schema to be used if the BigQuery table to write has to + be created. This can be either specified as a + :class:`~apache_beam.io.gcp.internal.clients.bigquery.\ +bigquery_v2_messages.TableSchema` + object or a single string of the form + ``'field1:type1,field2:type2,field3:type3'`` that defines a comma + separated list of fields. Here ``'type'`` should specify the BigQuery + type of the field. Single string based schemas do not support nested + fields, repeated fields, or specifying a BigQuery mode for fields + (mode will always be set to ``'NULLABLE'``). + create_disposition (BigQueryDisposition): A string describing what + happens if the table does not exist. Possible values are: + + * :attr:`BigQueryDisposition.CREATE_IF_NEEDED`: create if does not + exist. + * :attr:`BigQueryDisposition.CREATE_NEVER`: fail the write if does not + exist. + + write_disposition (BigQueryDisposition): A string describing what happens + if the table has already some data. Possible values are: + + * :attr:`BigQueryDisposition.WRITE_TRUNCATE`: delete existing rows. + * :attr:`BigQueryDisposition.WRITE_APPEND`: add to existing rows. + * :attr:`BigQueryDisposition.WRITE_EMPTY`: fail the write if table not + empty. + For streaming pipelines WriteTruncate can not be used. - batch_size: Number of rows to be written to BQ per streaming API insert. + + batch_size (int): Number of rows to be written to BQ per streaming API + insert. test_client: Override the default bigquery client used for testing. """ self.table_reference = _parse_table_reference(table, dataset, project) @@ -1300,14 +1335,20 @@ def __init__(self, table, dataset=None, project=None, schema=None, @staticmethod def get_table_schema_from_string(schema): - """Transform the string table schema into a bigquery.TableSchema instance. + """Transform the string table schema into a + :class:`~apache_beam.io.gcp.internal.clients.bigquery.\ +bigquery_v2_messages.TableSchema` instance. Args: - schema: The sting schema to be used if the BigQuery table to write has - to be created. + schema (str): The sting schema to be used if the BigQuery table to write + has to be created. + Returns: - table_schema: The schema to be used if the BigQuery table to write has - to be created but in the bigquery.TableSchema format. + ~apache_beam.io.gcp.internal.clients.bigquery.\ +bigquery_v2_messages.TableSchema: + The schema to be used if the BigQuery table to write has to be created + but in the :class:`~apache_beam.io.gcp.internal.clients.bigquery.\ +bigquery_v2_messages.TableSchema` format. """ table_schema = bigquery.TableSchema() schema_list = [s.strip() for s in schema.split(',')] @@ -1349,12 +1390,14 @@ def get_dict_table_schema(schema): """Transform the table schema into a dictionary instance. Args: - schema: The schema to be used if the BigQuery table to write has to be - created. This can either be a dict or string or in the TableSchema - format. + schema (~apache_beam.io.gcp.internal.clients.bigquery.\ +bigquery_v2_messages.TableSchema): + The schema to be used if the BigQuery table to write has to be created. + This can either be a dict or string or in the TableSchema format. + Returns: - table_schema: The schema to be used if the BigQuery table to write has - to be created but in the dictionary format. + Dict[str, Any]: The schema to be used if the BigQuery table to write has + to be created but in the dictionary format. """ if isinstance(schema, dict): return schema diff --git a/sdks/python/apache_beam/io/gcp/gcsio.py b/sdks/python/apache_beam/io/gcp/gcsio.py index 643fbc75c0020..ae71a5fe43dbe 100644 --- a/sdks/python/apache_beam/io/gcp/gcsio.py +++ b/sdks/python/apache_beam/io/gcp/gcsio.py @@ -137,16 +137,16 @@ def open(self, """Open a GCS file path for reading or writing. Args: - filename: GCS file path in the form gs:///. - mode: 'r' for reading or 'w' for writing. - read_buffer_size: Buffer size to use during read operations. - mime_type: Mime type to set for write operations. + filename (str): GCS file path in the form ``gs:///``. + mode (str): ``'r'`` for reading or ``'w'`` for writing. + read_buffer_size (int): Buffer size to use during read operations. + mime_type (str): Mime type to set for write operations. Returns: - file object. + GCS file object. Raises: - ValueError: Invalid open file mode. + ~exceptions.ValueError: Invalid open file mode. """ if mode == 'r' or mode == 'rb': return GcsBufferedReader(self.client, filename, mode=mode, diff --git a/sdks/python/apache_beam/io/range_trackers.py b/sdks/python/apache_beam/io/range_trackers.py index 4bd19f82087f6..1339b911efc50 100644 --- a/sdks/python/apache_beam/io/range_trackers.py +++ b/sdks/python/apache_beam/io/range_trackers.py @@ -317,17 +317,19 @@ def fraction_to_position(self, fraction, start, end): class UnsplittableRangeTracker(iobase.RangeTracker): """A RangeTracker that always ignores split requests. - This can be used to make a given ``RangeTracker`` object unsplittable by - ignoring all calls to ``try_split()``. All other calls will be delegated to - the given ``RangeTracker``. + This can be used to make a given + :class:`~apache_beam.io.iobase.RangeTracker` object unsplittable by + ignoring all calls to :meth:`.try_split()`. All other calls will be delegated + to the given :class:`~apache_beam.io.iobase.RangeTracker`. """ def __init__(self, range_tracker): """Initializes UnsplittableRangeTracker. Args: - range_tracker: a ``RangeTracker`` to which all method calls expect calls - to ``try_split()`` will be delegated. + range_tracker (~apache_beam.io.iobase.RangeTracker): a + :class:`~apache_beam.io.iobase.RangeTracker` to which all method + calls expect calls to :meth:`.try_split()` will be delegated. """ assert isinstance(range_tracker, iobase.RangeTracker) self._range_tracker = range_tracker diff --git a/sdks/python/apache_beam/io/source_test_utils.py b/sdks/python/apache_beam/io/source_test_utils.py index a144a8a8e5e43..bea970850838f 100644 --- a/sdks/python/apache_beam/io/source_test_utils.py +++ b/sdks/python/apache_beam/io/source_test_utils.py @@ -80,12 +80,13 @@ def read_from_source(source, start_position=None, stop_position=None): Only reads elements within the given position range. Args: - source: ``iobase.BoundedSource`` implementation. - start_position: start position for reading. - stop_position: stop position for reading. + source (~apache_beam.io.iobase.BoundedSource): + :class:`~apache_beam.io.iobase.BoundedSource` implementation. + start_position (int): start position for reading. + stop_position (int): stop position for reading. Returns: - the set of values read from the sources. + List[str]: the set of values read from the sources. """ values = [] range_tracker = source.get_range_tracker(start_position, stop_position) @@ -108,21 +109,25 @@ def _ThreadPool(threads): def assert_sources_equal_reference_source(reference_source_info, sources_info): """Tests if a reference source is equal to a given set of sources. - Given a reference source (a ``BoundedSource`` and a position range) and a - list of sources, assert that the union of the records - read from the list of sources is equal to the records read from the + Given a reference source (a :class:`~apache_beam.io.iobase.BoundedSource` + and a position range) and a list of sources, assert that the union of the + records read from the list of sources is equal to the records read from the reference source. Args: - reference_source_info: a three-tuple that gives the reference - ``iobase.BoundedSource``, position to start reading - at, and position to stop reading at. - sources_info: a set of sources. Each source is a three-tuple that is of - the same format described above. + reference_source_info\ + (Tuple[~apache_beam.io.iobase.BoundedSource, int, int]): + a three-tuple that gives the reference + :class:`~apache_beam.io.iobase.BoundedSource`, position to start + reading at, and position to stop reading at. + sources_info\ + (Iterable[Tuple[~apache_beam.io.iobase.BoundedSource, int, int]]): + a set of sources. Each source is a three-tuple that is of the same + format described above. Raises: - ValueError: if the set of data produced by the reference source and the - given set of sources are not equivalent. + ~exceptions.ValueError: if the set of data produced by the reference source + and the given set of sources are not equivalent. """ @@ -172,18 +177,20 @@ def assert_sources_equal_reference_source(reference_source_info, sources_info): def assert_reentrant_reads_succeed(source_info): """Tests if a given source can be read in a reentrant manner. - Assume that given source produces the set of values {v1, v2, v3, ... vn}. For - i in range [1, n-1] this method performs a reentrant read after reading i - elements and verifies that both the original and reentrant read produce the - expected set of values. + Assume that given source produces the set of values ``{v1, v2, v3, ... vn}``. + For ``i`` in range ``[1, n-1]`` this method performs a reentrant read after + reading ``i`` elements and verifies that both the original and reentrant read + produce the expected set of values. Args: - source_info: a three-tuple that gives the reference - ``iobase.BoundedSource``, position to start reading at, and a - position to stop reading at. + source_info (Tuple[~apache_beam.io.iobase.BoundedSource, int, int]): + a three-tuple that gives the reference + :class:`~apache_beam.io.iobase.BoundedSource`, position to start reading + at, and a position to stop reading at. + Raises: - ValueError: if source is too trivial or reentrant read result in an - incorrect read. + ~exceptions.ValueError: if source is too trivial or reentrant read result + in an incorrect read. """ source, start_position, stop_position = source_info @@ -228,21 +235,25 @@ def assert_split_at_fraction_behavior(source, num_items_to_read_before_split, split_fraction, expected_outcome): """Verifies the behaviour of splitting a source at a given fraction. - Asserts that splitting a ``BoundedSource`` either fails after reading - ``num_items_to_read_before_split`` items, or succeeds in a way that is - consistent according to ``assertSplitAtFractionSucceedsAndConsistent()``. + Asserts that splitting a :class:`~apache_beam.io.iobase.BoundedSource` either + fails after reading **num_items_to_read_before_split** items, or succeeds in + a way that is consistent according to + :func:`assert_split_at_fraction_succeeds_and_consistent()`. Args: - source: the source to perform dynamic splitting on. - num_items_to_read_before_split: number of items to read before splitting. - split_fraction: fraction to split at. - expected_outcome: a value from 'ExpectedSplitOutcome'. + source (~apache_beam.io.iobase.BoundedSource): the source to perform + dynamic splitting on. + num_items_to_read_before_split (int): number of items to read before + splitting. + split_fraction (float): fraction to split at. + expected_outcome (int): a value from + :class:`~apache_beam.io.source_test_utils.ExpectedSplitOutcome`. Returns: - a tuple that gives the number of items produced by reading the two ranges - produced after dynamic splitting. If splitting did not occur, the first - value of the tuple will represent the full set of records read by the - source while the second value of the tuple will be '-1'. + Tuple[int, int]: a tuple that gives the number of items produced by reading + the two ranges produced after dynamic splitting. If splitting did not + occur, the first value of the tuple will represent the full set of records + read by the source while the second value of the tuple will be ``-1``. """ assert isinstance(source, iobase.BoundedSource) expected_items = read_from_source(source, None, None) @@ -503,12 +514,13 @@ def assert_split_at_fraction_exhaustive( Verifies multi threaded splitting as well. Args: - source: the source to perform dynamic splitting on. - perform_multi_threaded_test: if true performs a multi-threaded test - otherwise this test is skipped. + source (~apache_beam.io.iobase.BoundedSource): the source to perform + dynamic splitting on. + perform_multi_threaded_test (bool): if :data:`True` performs a + multi-threaded test, otherwise this test is skipped. Raises: - ValueError: if the exhaustive splitting test fails. + ~exceptions.ValueError: if the exhaustive splitting test fails. """ expected_items = read_from_source(source, start_position, stop_position) diff --git a/sdks/python/apache_beam/io/textio.py b/sdks/python/apache_beam/io/textio.py index 9c6532ee0e98c..9708df7c753ca 100644 --- a/sdks/python/apache_beam/io/textio.py +++ b/sdks/python/apache_beam/io/textio.py @@ -417,13 +417,15 @@ def expand(self, pvalue): class ReadFromText(PTransform): - """A ``PTransform`` for reading text files. + r"""A :class:`~apache_beam.transforms.ptransform.PTransform` for reading text + files. Parses a text file as newline-delimited elements, by default assuming - UTF-8 encoding. Supports newline delimiters '\\n' and '\\r\\n'. + ``UTF-8`` encoding. Supports newline delimiters ``\n`` and ``\r\n``. - This implementation only supports reading text encoded using UTF-8 or ASCII. - This does not support other encodings such as UTF-16 or UTF-32. + This implementation only supports reading text encoded using ``UTF-8`` or + ``ASCII``. + This does not support other encodings such as ``UTF-16`` or ``UTF-32``. """ def __init__( self, @@ -435,26 +437,28 @@ def __init__( validate=True, skip_header_lines=0, **kwargs): - """Initialize the ``ReadFromText`` transform. + """Initialize the :class:`ReadFromText` transform. Args: - file_pattern: The file path to read from as a local file path or a GCS - ``gs://`` path. The path can contain glob characters - ``(*, ?, and [...] sets)``. - min_bundle_size: Minimum size of bundles that should be generated when - splitting this source into bundles. See ``FileBasedSource`` for more + file_pattern (str): The file path to read from as a local file path or a + GCS ``gs://`` path. The path can contain glob characters + (``*``, ``?``, and ``[...]`` sets). + min_bundle_size (int): Minimum size of bundles that should be generated + when splitting this source into bundles. See + :class:`~apache_beam.io.filebasedsource.FileBasedSource` for more details. - compression_type: Used to handle compressed input files. Typical value - is ``CompressionTypes.AUTO``, in which case the underlying file_path's - extension will be used to detect the compression. - strip_trailing_newlines: Indicates whether this source should remove - the newline char in each line it reads before decoding that line. - validate: flag to verify that the files exist during the pipeline + compression_type (str): Used to handle compressed input files. + Typical value is :attr:`CompressionTypes.AUTO + `, in which case the + underlying file_path's extension will be used to detect the compression. + strip_trailing_newlines (bool): Indicates whether this source should + remove the newline char in each line it reads before decoding that line. + validate (bool): flag to verify that the files exist during the pipeline creation time. - skip_header_lines: Number of header lines to skip. Same number is skipped - from each source file. Must be 0 or higher. Large number of skipped - lines might impact performance. - coder: Coder used to decode each line. + skip_header_lines (int): Number of header lines to skip. Same number is + skipped from each source file. Must be 0 or higher. Large number of + skipped lines might impact performance. + coder (~apache_beam.coders.coders.Coder): Coder used to decode each line. """ super(ReadFromText, self).__init__(**kwargs) @@ -468,49 +472,54 @@ def expand(self, pvalue): class WriteToText(PTransform): - """A PTransform for writing to text files.""" + """A :class:`~apache_beam.transforms.ptransform.PTransform` for writing to + text files.""" - def __init__(self, - file_path_prefix, - file_name_suffix='', - append_trailing_newlines=True, - num_shards=0, - shard_name_template=None, - coder=coders.ToStringCoder(), - compression_type=CompressionTypes.AUTO, - header=None): - """Initialize a WriteToText PTransform. + def __init__( + self, + file_path_prefix, + file_name_suffix='', + append_trailing_newlines=True, + num_shards=0, + shard_name_template=None, + coder=coders.ToStringCoder(), + compression_type=CompressionTypes.AUTO, + header=None): + r"""Initialize a :class:`WriteToText` transform. Args: - file_path_prefix: The file path to write to. The files written will begin - with this prefix, followed by a shard identifier (see num_shards), and - end in a common extension, if given by file_name_suffix. In most cases, - only this argument is specified and num_shards, shard_name_template, and - file_name_suffix use default values. - file_name_suffix: Suffix for the files written. - append_trailing_newlines: indicate whether this sink should write an - additional newline char after writing each element. - num_shards: The number of files (shards) used for output. If not set, the - service will decide on the optimal number of shards. + file_path_prefix (str): The file path to write to. The files written will + begin with this prefix, followed by a shard identifier (see + **num_shards**), and end in a common extension, if given by + **file_name_suffix**. In most cases, only this argument is specified and + **num_shards**, **shard_name_template**, and **file_name_suffix** use + default values. + file_name_suffix (str): Suffix for the files written. + append_trailing_newlines (bool): indicate whether this sink should write + an additional newline char after writing each element. + num_shards (int): The number of files (shards) used for output. + If not set, the service will decide on the optimal number of shards. Constraining the number of shards is likely to reduce the performance of a pipeline. Setting this value is not recommended unless you require a specific number of output files. - shard_name_template: A template string containing placeholders for - the shard number and shard count. Currently only '' and - '-SSSSS-of-NNNNN' are patterns accepted by the service. + shard_name_template (str): A template string containing placeholders for + the shard number and shard count. Currently only ``''`` and + ``'-SSSSS-of-NNNNN'`` are patterns accepted by the service. When constructing a filename for a particular shard number, the - upper-case letters 'S' and 'N' are replaced with the 0-padded shard - number and shard count respectively. This argument can be '' in which - case it behaves as if num_shards was set to 1 and only one file will be - generated. The default pattern used is '-SSSSS-of-NNNNN'. - coder: Coder used to encode each line. - compression_type: Used to handle compressed output files. Typical value - is CompressionTypes.AUTO, in which case the final file path's - extension (as determined by file_path_prefix, file_name_suffix, - num_shards and shard_name_template) will be used to detect the - compression. - header: String to write at beginning of file as a header. If not None and - append_trailing_newlines is set, '\n' will be added. + upper-case letters ``S`` and ``N`` are replaced with the ``0``-padded + shard number and shard count respectively. This argument can be ``''`` + in which case it behaves as if num_shards was set to 1 and only one file + will be generated. The default pattern used is ``'-SSSSS-of-NNNNN'``. + coder (~apache_beam.coders.coders.Coder): Coder used to encode each line. + compression_type (str): Used to handle compressed output files. + Typical value is :class:`CompressionTypes.AUTO + `, in which case the + final file path's extension (as determined by **file_path_prefix**, + **file_name_suffix**, **num_shards** and **shard_name_template**) will + be used to detect the compression. + header (str): String to write at beginning of file as a header. + If not :data:`None` and **append_trailing_newlines** is set, ``\n`` will + be added. """ self._sink = _TextSink(file_path_prefix, file_name_suffix, diff --git a/sdks/python/apache_beam/pipeline.py b/sdks/python/apache_beam/pipeline.py index e7c23225d51cd..1ade6c067c5f8 100644 --- a/sdks/python/apache_beam/pipeline.py +++ b/sdks/python/apache_beam/pipeline.py @@ -15,17 +15,18 @@ # limitations under the License. # -"""Pipeline, the top-level Dataflow object. +"""Pipeline, the top-level Beam object. A pipeline holds a DAG of data transforms. Conceptually the nodes of the DAG -are transforms (PTransform objects) and the edges are values (mostly PCollection +are transforms (:class:`~apache_beam.transforms.ptransform.PTransform` objects) +and the edges are values (mostly :class:`~apache_beam.pvalue.PCollection` objects). The transforms take as inputs one or more PValues and output one or -more PValues. +more :class:`~apache_beam.pvalue.PValue` s. The pipeline offers functionality to traverse the graph. The actual operation to be executed for each node visited is specified through a runner object. -Typical usage: +Typical usage:: # Create a pipeline object using a local runner for execution. with beam.Pipeline('DirectRunner') as p: @@ -73,32 +74,40 @@ class Pipeline(object): - """A pipeline object that manages a DAG of PValues and their PTransforms. + """A pipeline object that manages a DAG of + :class:`~apache_beam.pvalue.PValue` s and their + :class:`~apache_beam.transforms.ptransform.PTransform` s. - Conceptually the PValues are the DAG's nodes and the PTransforms computing - the PValues are the edges. + Conceptually the :class:`~apache_beam.pvalue.PValue` s are the DAG's nodes and + the :class:`~apache_beam.transforms.ptransform.PTransform` s computing + the :class:`~apache_beam.pvalue.PValue` s are the edges. All the transforms applied to the pipeline must have distinct full labels. If same transform instance needs to be applied then the right shift operator - should be used to designate new names (e.g. `input | "label" >> my_tranform`). + should be used to designate new names + (e.g. ``input | "label" >> my_tranform``). """ def __init__(self, runner=None, options=None, argv=None): """Initialize a pipeline object. Args: - runner: An object of type 'PipelineRunner' that will be used to execute - the pipeline. For registered runners, the runner name can be specified, - otherwise a runner object must be supplied. - options: A configured 'PipelineOptions' object containing arguments - that should be used for running the Dataflow job. - argv: a list of arguments (such as sys.argv) to be used for building a - 'PipelineOptions' object. This will only be used if argument 'options' - is None. + runner (~apache_beam.runners.runner.PipelineRunner): An object of + type :class:`~apache_beam.runners.runner.PipelineRunner` that will be + used to execute the pipeline. For registered runners, the runner name + can be specified, otherwise a runner object must be supplied. + options (~apache_beam.options.pipeline_options.PipelineOptions): + A configured + :class:`~apache_beam.options.pipeline_options.PipelineOptions` object + containing arguments that should be used for running the Beam job. + argv (List[str]): a list of arguments (such as :data:`sys.argv`) + to be used for building a + :class:`~apache_beam.options.pipeline_options.PipelineOptions` object. + This will only be used if argument **options** is :data:`None`. Raises: - ValueError: if either the runner or options argument is not of the - expected type. + ~exceptions.ValueError: if either the runner or options argument is not + of the expected type. """ if options is not None: if isinstance(options, PipelineOptions): @@ -292,13 +301,15 @@ def visit_transform(self, transform_node): def replace_all(self, replacements): """ Dynamically replaces PTransforms in the currently populated hierarchy. - Currently this only works for replacements where input and output types - are exactly the same. - TODO: Update this to also work for transform overrides where input and - output types are different. + Currently this only works for replacements where input and output types + are exactly the same. + + TODO: Update this to also work for transform overrides where input and + output types are different. Args: - replacements a list of PTransformOverride objects. + replacements (List[~apache_beam.pipeline.PTransformOverride]): a list of + :class:`~apache_beam.pipeline.PTransformOverride` objects. """ for override in replacements: assert isinstance(override, PTransformOverride) @@ -341,13 +352,16 @@ def visit(self, visitor): Runner-internal implementation detail; no backwards-compatibility guarantees Args: - visitor: PipelineVisitor object whose callbacks will be called for each - node visited. See PipelineVisitor comments. + visitor (~apache_beam.pipeline.PipelineVisitor): + :class:`~apache_beam.pipeline.PipelineVisitor` object whose callbacks + will be called for each node visited. See + :class:`~apache_beam.pipeline.PipelineVisitor` comments. Raises: - TypeError: if node is specified and is not a PValue. - pipeline.PipelineError: if node is specified and does not belong to this - pipeline instance. + ~exceptions.TypeError: if node is specified and is not a + :class:`~apache_beam.pvalue.PValue`. + ~apache_beam.error.PipelineError: if node is specified and does not + belong to this pipeline instance. """ visited = set() @@ -357,15 +371,20 @@ def apply(self, transform, pvalueish=None, label=None): """Applies a custom transform using the pvalueish specified. Args: - transform: the PTranform to apply. - pvalueish: the input for the PTransform (typically a PCollection). - label: label of the PTransform. + transform (~apache_beam.transforms.ptransform.PTransform): the + :class:`~apache_beam.transforms.ptransform.PTransform` to apply. + pvalueish (~apache_beam.pvalue.PCollection): the input for the + :class:`~apache_beam.transforms.ptransform.PTransform` (typically a + :class:`~apache_beam.pvalue.PCollection`). + label (str): label of the + :class:`~apache_beam.transforms.ptransform.PTransform`. Raises: - TypeError: if the transform object extracted from the argument list is - not a PTransform. - RuntimeError: if the transform object was already applied to this pipeline - and needs to be cloned in order to apply again. + ~exceptions.TypeError: if the transform object extracted from the + argument list is not a + :class:`~apache_beam.transforms.ptransform.PTransform`. + ~exceptions.RuntimeError: if the transform object was already applied to + this pipeline and needs to be cloned in order to apply again. """ if isinstance(transform, ptransform._NamedPTransform): return self.apply(transform.transform, pvalueish, diff --git a/sdks/python/apache_beam/runners/runner.py b/sdks/python/apache_beam/runners/runner.py index 7ce9a03876f8a..a3c6b345f2099 100644 --- a/sdks/python/apache_beam/runners/runner.py +++ b/sdks/python/apache_beam/runners/runner.py @@ -283,10 +283,10 @@ def key(self, pobj): class PipelineState(object): - """State of the Pipeline, as returned by PipelineResult.state. + """State of the Pipeline, as returned by :attr:`PipelineResult.state`. This is meant to be the union of all the states any runner can put a - pipeline in. Currently, it represents the values of the dataflow + pipeline in. Currently, it represents the values of the dataflow API JobState enum. """ UNKNOWN = 'UNKNOWN' # not specified @@ -301,7 +301,7 @@ class PipelineState(object): class PipelineResult(object): - """A PipelineResult provides access to info about a pipeline.""" + """A :class:`PipelineResult` provides access to info about a pipeline.""" def __init__(self, state): self._state = state @@ -315,15 +315,18 @@ def wait_until_finish(self, duration=None): """Waits until the pipeline finishes and returns the final status. Args: - duration: The time to wait (in milliseconds) for job to finish. If it is - set to None, it will wait indefinitely until the job is finished. + duration (int): The time to wait (in milliseconds) for job to finish. + If it is set to :data:`None`, it will wait indefinitely until the job + is finished. Raises: - IOError: If there is a persistent problem getting job information. - NotImplementedError: If the runner does not support this operation. + ~exceptions.IOError: If there is a persistent problem getting job + information. + ~exceptions.NotImplementedError: If the runner does not support this + operation. Returns: - The final state of the pipeline, or None on timeout. + The final state of the pipeline, or :data:`None` on timeout. """ raise NotImplementedError @@ -331,8 +334,10 @@ def cancel(self): """Cancels the pipeline execution. Raises: - IOError: If there is a persistent problem getting job information. - NotImplementedError: If the runner does not support this operation. + ~exceptions.IOError: If there is a persistent problem getting job + information. + ~exceptions.NotImplementedError: If the runner does not support this + operation. Returns: The final state of the pipeline. @@ -340,10 +345,12 @@ def cancel(self): raise NotImplementedError def metrics(self): - """Returns MetricsResult object to query metrics from the runner. + """Returns :class:`~apache_beam.metrics.metric.MetricResults` object to + query metrics from the runner. Raises: - NotImplementedError: If the runner does not support this operation. + ~exceptions.NotImplementedError: If the runner does not support this + operation. """ raise NotImplementedError diff --git a/sdks/python/apache_beam/testing/test_pipeline.py b/sdks/python/apache_beam/testing/test_pipeline.py index 13b1639ea499f..83802423dbca4 100644 --- a/sdks/python/apache_beam/testing/test_pipeline.py +++ b/sdks/python/apache_beam/testing/test_pipeline.py @@ -33,23 +33,23 @@ class TestPipeline(Pipeline): - """TestPipeline class is used inside of Beam tests that can be configured to - run against pipeline runner. + """:class:`TestPipeline` class is used inside of Beam tests that can be + configured to run against pipeline runner. It has a functionality to parse arguments from command line and build pipeline options for tests who runs against a pipeline runner and utilizes resources of the pipeline runner. Those test functions are recommended to be tagged by - @attr("ValidatesRunner") annotation. + ``@attr("ValidatesRunner")`` annotation. In order to configure the test with customized pipeline options from command - line, system argument 'test-pipeline-options' can be used to obtains a list - of pipeline options. If no options specified, default value will be used. + line, system argument ``--test-pipeline-options`` can be used to obtains a + list of pipeline options. If no options specified, default value will be used. For example, use following command line to execute all ValidatesRunner tests:: - python setup.py nosetests -a ValidatesRunner \ - --test-pipeline-options="--runner=DirectRunner \ - --job_name=myJobName \ + python setup.py nosetests -a ValidatesRunner \\ + --test-pipeline-options="--runner=DirectRunner \\ + --job_name=myJobName \\ --num_workers=1" For example, use assert_that for test validation:: @@ -69,21 +69,27 @@ def __init__(self, """Initialize a pipeline object for test. Args: - runner: An object of type 'PipelineRunner' that will be used to execute - the pipeline. For registered runners, the runner name can be specified, - otherwise a runner object must be supplied. - options: A configured 'PipelineOptions' object containing arguments - that should be used for running the pipeline job. - argv: A list of arguments (such as sys.argv) to be used for building a - 'PipelineOptions' object. This will only be used if argument 'options' - is None. - is_integration_test: True if the test is an integration test, False - otherwise. - blocking: Run method will wait until pipeline execution is completed. + runner (~apache_beam.runners.runner.PipelineRunner): An object of type + :class:`~apache_beam.runners.runner.PipelineRunner` that will be used + to execute the pipeline. For registered runners, the runner name can be + specified, otherwise a runner object must be supplied. + options (~apache_beam.options.pipeline_options.PipelineOptions): + A configured + :class:`~apache_beam.options.pipeline_options.PipelineOptions` + object containing arguments that should be used for running the + pipeline job. + argv (List[str]): A list of arguments (such as :data:`sys.argv`) to be + used for building a + :class:`~apache_beam.options.pipeline_options.PipelineOptions` object. + This will only be used if argument **options** is :data:`None`. + is_integration_test (bool): :data:`True` if the test is an integration + test, :data:`False` otherwise. + blocking (bool): Run method will wait until pipeline execution is + completed. Raises: - ValueError: if either the runner or options argument is not of the - expected type. + ~exceptions.ValueError: if either the runner or options argument is not + of the expected type. """ self.is_integration_test = is_integration_test self.options_list = self._parse_test_option_args(argv) diff --git a/sdks/python/apache_beam/transforms/core.py b/sdks/python/apache_beam/transforms/core.py index 9018a496cdd99..d6f56d2d0cf05 100644 --- a/sdks/python/apache_beam/transforms/core.py +++ b/sdks/python/apache_beam/transforms/core.py @@ -601,31 +601,35 @@ def partition_for(self, element, num_partitions, *args, **kwargs): class ParDo(PTransformWithSideInputs): - """A ParDo transform. + """A :class:`ParDo` transform. - Processes an input PCollection by applying a DoFn to each element and - returning the accumulated results into an output PCollection. The type of the - elements is not fixed as long as the DoFn can deal with it. In reality - the type is restrained to some extent because the elements sometimes must be - persisted to external storage. See the expand() method comments for a detailed - description of all possible arguments. + Processes an input :class:`~apache_beam.pvalue.PCollection` by applying a + :class:`DoFn` to each element and returning the accumulated results into an + output :class:`~apache_beam.pvalue.PCollection`. The type of the elements is + not fixed as long as the :class:`DoFn` can deal with it. In reality the type + is restrained to some extent because the elements sometimes must be persisted + to external storage. See the :meth:`.expand()` method comments for a + detailed description of all possible arguments. - Note that the DoFn must return an iterable for each element of the input - PCollection. An easy way to do this is to use the yield keyword in the - process method. + Note that the :class:`DoFn` must return an iterable for each element of the + input :class:`~apache_beam.pvalue.PCollection`. An easy way to do this is to + use the ``yield`` keyword in the process method. Args: - pcoll: a PCollection to be processed. - fn: a DoFn object to be applied to each element of pcoll argument. - *args: positional arguments passed to the dofn object. - **kwargs: keyword arguments passed to the dofn object. + pcoll (~apache_beam.pvalue.PCollection): + a :class:`~apache_beam.pvalue.PCollection` to be processed. + fn (DoFn): a :class:`DoFn` object to be applied to each element + of **pcoll** argument. + *args: positional arguments passed to the :class:`DoFn` object. + **kwargs: keyword arguments passed to the :class:`DoFn` object. Note that the positional and keyword arguments will be processed in order - to detect PCollections that will be computed as side inputs to the - transform. During pipeline execution whenever the DoFn object gets executed - (its apply() method gets called) the PCollection arguments will be replaced - by values from the PCollection in the exact positions where they appear in - the argument lists. + to detect :class:`~apache_beam.pvalue.PCollection` s that will be computed as + side inputs to the transform. During pipeline execution whenever the + :class:`DoFn` object gets executed (its :meth:`DoFn.process()` method gets + called) the :class:`~apache_beam.pvalue.PCollection` arguments will be + replaced by values from the :class:`~apache_beam.pvalue.PCollection` in the + exact positions where they appear in the argument lists. """ def __init__(self, fn, *args, **kwargs): @@ -665,27 +669,34 @@ def expand(self, pcoll): return pvalue.PCollection(pcoll.pipeline) def with_outputs(self, *tags, **main_kw): - """Returns a tagged tuple allowing access to the outputs of a ParDo. + """Returns a tagged tuple allowing access to the outputs of a + :class:`ParDo`. The resulting object supports access to the - PCollection associated with a tag (e.g., o.tag, o[tag]) and iterating over - the available tags (e.g., for tag in o: ...). + :class:`~apache_beam.pvalue.PCollection` associated with a tag + (e.g. ``o.tag``, ``o[tag]``) and iterating over the available tags + (e.g. ``for tag in o: ...``). Args: *tags: if non-empty, list of valid tags. If a list of valid tags is given, it will be an error to use an undeclared tag later in the pipeline. - **main_kw: dictionary empty or with one key 'main' defining the tag to be - used for the main output (which will not have a tag associated with it). + **main_kw: dictionary empty or with one key ``'main'`` defining the tag to + be used for the main output (which will not have a tag associated with + it). Returns: - An object of type DoOutputsTuple that bundles together all the outputs - of a ParDo transform and allows accessing the individual - PCollections for each output using an object.tag syntax. + ~apache_beam.pvalue.DoOutputsTuple: An object of type + :class:`~apache_beam.pvalue.DoOutputsTuple` that bundles together all + the outputs of a :class:`ParDo` transform and allows accessing the + individual :class:`~apache_beam.pvalue.PCollection` s for each output + using an ``object.tag`` syntax. Raises: - TypeError: if the self object is not a PCollection that is the result of - a ParDo transform. - ValueError: if main_kw contains any key other than 'main'. + ~exceptions.TypeError: if the **self** object is not a + :class:`~apache_beam.pvalue.PCollection` that is the result of a + :class:`ParDo` transform. + ~exceptions.ValueError: if **main_kw** contains any key other than + ``'main'``. """ main_tag = main_kw.pop('main', None) if main_kw: @@ -739,24 +750,27 @@ def expand(self, pcoll): def FlatMap(fn, *args, **kwargs): # pylint: disable=invalid-name - """FlatMap is like ParDo except it takes a callable to specify the - transformation. + """:func:`FlatMap` is like :class:`ParDo` except it takes a callable to + specify the transformation. The callable must return an iterable for each element of the input - PCollection. The elements of these iterables will be flattened into - the output PCollection. + :class:`~apache_beam.pvalue.PCollection`. The elements of these iterables will + be flattened into the output :class:`~apache_beam.pvalue.PCollection`. Args: - fn: a callable object. + fn (callable): a callable object. *args: positional arguments passed to the transform callable. **kwargs: keyword arguments passed to the transform callable. Returns: - A PCollection containing the Map outputs. + ~apache_beam.pvalue.PCollection: + A :class:`~apache_beam.pvalue.PCollection` containing the + :func:`FlatMap` outputs. Raises: - TypeError: If the fn passed as argument is not a callable. Typical error - is to pass a DoFn instance which is supported only for ParDo. + ~exceptions.TypeError: If the **fn** passed as argument is not a callable. + Typical error is to pass a :class:`DoFn` instance which is supported only + for :class:`ParDo`. """ label = 'FlatMap(%s)' % ptransform.label_from_callable(fn) if not callable(fn): @@ -770,19 +784,23 @@ def FlatMap(fn, *args, **kwargs): # pylint: disable=invalid-name def Map(fn, *args, **kwargs): # pylint: disable=invalid-name - """Map is like FlatMap except its callable returns only a single element. + """:func:`Map` is like :func:`FlatMap` except its callable returns only a + single element. Args: - fn: a callable object. + fn (callable): a callable object. *args: positional arguments passed to the transform callable. **kwargs: keyword arguments passed to the transform callable. Returns: - A PCollection containing the Map outputs. + ~apache_beam.pvalue.PCollection: + A :class:`~apache_beam.pvalue.PCollection` containing the + :func:`Map` outputs. Raises: - TypeError: If the fn passed as argument is not a callable. Typical error - is to pass a DoFn instance which is supported only for ParDo. + ~exceptions.TypeError: If the **fn** passed as argument is not a callable. + Typical error is to pass a :class:`DoFn` instance which is supported only + for :class:`ParDo`. """ if not callable(fn): raise TypeError( @@ -815,19 +833,23 @@ def Map(fn, *args, **kwargs): # pylint: disable=invalid-name def Filter(fn, *args, **kwargs): # pylint: disable=invalid-name - """Filter is a FlatMap with its callable filtering out elements. + """:func:`Filter` is a :func:`FlatMap` with its callable filtering out + elements. Args: - fn: a callable object. + fn (callable): a callable object. *args: positional arguments passed to the transform callable. **kwargs: keyword arguments passed to the transform callable. Returns: - A PCollection containing the Filter outputs. + ~apache_beam.pvalue.PCollection: + A :class:`~apache_beam.pvalue.PCollection` containing the + :func:`Filter` outputs. Raises: - TypeError: If the fn passed as argument is not a callable. Typical error - is to pass a DoFn instance which is supported only for FlatMap. + ~exceptions.TypeError: If the **fn** passed as argument is not a callable. + Typical error is to pass a :class:`DoFn` instance which is supported only + for :class:`ParDo`. """ if not callable(fn): raise TypeError( @@ -867,35 +889,42 @@ def _combine_payload(combine_fn, context): class CombineGlobally(PTransform): - """A CombineGlobally transform. + """A :class:`CombineGlobally` transform. - Reduces a PCollection to a single value by progressively applying a CombineFn - to portions of the PCollection (and to intermediate values created thereby). - See documentation in CombineFn for details on the specifics on how CombineFns - are applied. + Reduces a :class:`~apache_beam.pvalue.PCollection` to a single value by + progressively applying a :class:`CombineFn` to portions of the + :class:`~apache_beam.pvalue.PCollection` (and to intermediate values created + thereby). See documentation in :class:`CombineFn` for details on the specifics + on how :class:`CombineFn` s are applied. Args: - pcoll: a PCollection to be reduced into a single value. - fn: a CombineFn object that will be called to progressively reduce the - PCollection into single values, or a callable suitable for wrapping - by CallableWrapperCombineFn. - *args: positional arguments passed to the CombineFn object. - **kwargs: keyword arguments passed to the CombineFn object. + pcoll (~apache_beam.pvalue.PCollection): + a :class:`~apache_beam.pvalue.PCollection` to be reduced into a single + value. + fn (callable): a :class:`CombineFn` object that will be called to + progressively reduce the :class:`~apache_beam.pvalue.PCollection` into + single values, or a callable suitable for wrapping by + :class:`~apache_beam.transforms.core.CallableWrapperCombineFn`. + *args: positional arguments passed to the :class:`CombineFn` object. + **kwargs: keyword arguments passed to the :class:`CombineFn` object. Raises: - TypeError: If the output type of the input PCollection is not compatible - with Iterable[A]. + ~exceptions.TypeError: If the output type of the input + :class:`~apache_beam.pvalue.PCollection` is not compatible + with ``Iterable[A]``. Returns: - A single-element PCollection containing the main output of the Combine - transform. + ~apache_beam.pvalue.PCollection: A single-element + :class:`~apache_beam.pvalue.PCollection` containing the main output of + the :class:`CombineGlobally` transform. Note that the positional and keyword arguments will be processed in order - to detect PObjects that will be computed as side inputs to the transform. - During pipeline execution whenever the CombineFn object gets executed (i.e., - any of the CombineFn methods get called), the PObject arguments will be - replaced by their actual value in the exact position where they appear in - the argument lists. + to detect :class:`~apache_beam.pvalue.PValue` s that will be computed as side + inputs to the transform. + During pipeline execution whenever the :class:`CombineFn` object gets executed + (i.e. any of the :class:`CombineFn` methods get called), the + :class:`~apache_beam.pvalue.PValue` arguments will be replaced by their + actual value in the exact position where they appear in the argument lists. """ has_defaults = True as_view = False diff --git a/sdks/python/apache_beam/transforms/display.py b/sdks/python/apache_beam/transforms/display.py index 152f16e2d9fb0..88a1feef75e1f 100644 --- a/sdks/python/apache_beam/transforms/display.py +++ b/sdks/python/apache_beam/transforms/display.py @@ -16,21 +16,24 @@ # """ -DisplayData, its classes, interfaces and methods. +:class:`DisplayData`, its classes, interfaces and methods. The classes in this module allow users and transform developers to define -static display data to be displayed when a pipeline runs. PTransforms, DoFns -and other pipeline components are subclasses of the HasDisplayData mixin. To -add static display data to a component, you can override the display_data -method of the HasDisplayData class. +static display data to be displayed when a pipeline runs. +:class:`~apache_beam.transforms.ptransform.PTransform` s, +:class:`~apache_beam.transforms.core.DoFn` s +and other pipeline components are subclasses of the :class:`HasDisplayData` +mixin. To add static display data to a component, you can override the +:meth:`HasDisplayData.display_data()` method. Available classes: -- HasDisplayData - Components that inherit from this class can have static - display data shown in the UI. -- DisplayDataItem - This class represents static display data elements. -- DisplayData - Internal class that is used to create display data and - communicate it to the API. +* :class:`HasDisplayData` - Components that inherit from this class can have + static display data shown in the UI. +* :class:`DisplayDataItem` - This class represents static display data + elements. +* :class:`DisplayData` - Internal class that is used to create display data + and communicate it to the API. """ from __future__ import absolute_import @@ -57,17 +60,19 @@ def display_data(self): static display data. Returns: - A dictionary containing key:value pairs. The value might be an - integer, float or string value; a DisplayDataItem for values that - have more data (e.g. short value, label, url); or a HasDisplayData - instance that has more display data that should be picked up. For - example: - - { 'key1': 'string_value', - 'key2': 1234, - 'key3': 3.14159265, - 'key4': DisplayDataItem('apache.org', url='http://apache.org'), - 'key5': subComponent } + Dict[str, Any]: A dictionary containing ``key:value`` pairs. + The value might be an integer, float or string value; a + :class:`DisplayDataItem` for values that have more data + (e.g. short value, label, url); or a :class:`HasDisplayData` instance + that has more display data that should be picked up. For example:: + + { + 'key1': 'string_value', + 'key2': 1234, + 'key3': 3.14159265, + 'key4': DisplayDataItem('apache.org', url='http://apache.org'), + 'key5': subComponent + } """ return {} @@ -111,18 +116,19 @@ def _populate_items(self, display_data_dict): @classmethod def create_from_options(cls, pipeline_options): - """ Creates DisplayData from a PipelineOptions instance. + """ Creates :class:`DisplayData` from a + :class:`~apache_beam.options.pipeline_options.PipelineOptions` instance. - When creating DisplayData, this method will convert the value of any - item of a non-supported type to its string representation. - The normal DisplayData.create_from method rejects those items. + When creating :class:`DisplayData`, this method will convert the value of + any item of a non-supported type to its string representation. + The normal :meth:`.create_from()` method rejects those items. Returns: - A DisplayData instance with populated items. + DisplayData: A :class:`DisplayData` instance with populated items. Raises: - ValueError: If the has_display_data argument is not an instance of - HasDisplayData. + ~exceptions.ValueError: If the **has_display_data** argument is + not an instance of :class:`HasDisplayData`. """ from apache_beam.options.pipeline_options import PipelineOptions if not isinstance(pipeline_options, PipelineOptions): @@ -138,14 +144,14 @@ def create_from_options(cls, pipeline_options): @classmethod def create_from(cls, has_display_data): - """ Creates DisplayData from a HasDisplayData instance. + """ Creates :class:`DisplayData` from a :class:`HasDisplayData` instance. Returns: - A DisplayData instance with populated items. + DisplayData: A :class:`DisplayData` instance with populated items. Raises: - ValueError: If the has_display_data argument is not an instance of - HasDisplayData. + ~exceptions.ValueError: If the **has_display_data** argument is + not an instance of :class:`HasDisplayData`. """ if not isinstance(has_display_data, HasDisplayData): raise ValueError('Element of class {}.{} does not subclass HasDisplayData' @@ -214,11 +220,13 @@ def should_drop(self): return False def is_valid(self): - """ Checks that all the necessary fields of the DisplayDataItem are - filled in. It checks that neither key, namespace, value or type are None. + """ Checks that all the necessary fields of the :class:`DisplayDataItem` + are filled in. It checks that neither key, namespace, value or type are + :data:`None`. Raises: - ValueError: If the item does not have a key, namespace, value or type. + ~exceptions.ValueError: If the item does not have a key, namespace, + value or type. """ if self.key is None: raise ValueError('Invalid DisplayDataItem. Key must not be None') @@ -247,14 +255,15 @@ def _get_dict(self): return res def get_dict(self): - """ Returns the internal-API dictionary representing the DisplayDataItem. + """ Returns the internal-API dictionary representing the + :class:`DisplayDataItem`. Returns: - A dictionary. The internal-API dictionary representing the - DisplayDataItem + Dict[str, Any]: A dictionary. The internal-API dictionary representing + the :class:`DisplayDataItem`. Raises: - ValueError: if the item is not valid. + ~exceptions.ValueError: if the item is not valid. """ self.is_valid() return self._get_dict() diff --git a/sdks/python/apache_beam/transforms/ptransform.py b/sdks/python/apache_beam/transforms/ptransform.py index a798fa1b3b250..f6e08ca9c0cee 100644 --- a/sdks/python/apache_beam/transforms/ptransform.py +++ b/sdks/python/apache_beam/transforms/ptransform.py @@ -214,38 +214,44 @@ def default_label(self): return self.__class__.__name__ def with_input_types(self, input_type_hint): - """Annotates the input type of a PTransform with a type-hint. + """Annotates the input type of a :class:`PTransform` with a type-hint. Args: - input_type_hint: An instance of an allowed built-in type, a custom class, - or an instance of a typehints.TypeConstraint. + input_type_hint (type): An instance of an allowed built-in type, a custom + class, or an instance of a + :class:`~apache_beam.typehints.typehints.TypeConstraint`. Raises: - TypeError: If 'type_hint' is not a valid type-hint. See - typehints.validate_composite_type_param for further details. + ~exceptions.TypeError: If **input_type_hint** is not a valid type-hint. + See + :obj:`apache_beam.typehints.typehints.validate_composite_type_param()` + for further details. Returns: - A reference to the instance of this particular PTransform object. This - allows chaining type-hinting related methods. + PTransform: A reference to the instance of this particular + :class:`PTransform` object. This allows chaining type-hinting related + methods. """ validate_composite_type_param(input_type_hint, 'Type hints for a PTransform') return super(PTransform, self).with_input_types(input_type_hint) def with_output_types(self, type_hint): - """Annotates the output type of a PTransform with a type-hint. + """Annotates the output type of a :class:`PTransform` with a type-hint. Args: - type_hint: An instance of an allowed built-in type, a custom class, or a - typehints.TypeConstraint. + type_hint (type): An instance of an allowed built-in type, a custom class, + or a :class:`~apache_beam.typehints.typehints.TypeConstraint`. Raises: - TypeError: If 'type_hint' is not a valid type-hint. See - typehints.validate_composite_type_param for further details. + ~exceptions.TypeError: If **type_hint** is not a valid type-hint. See + :obj:`~apache_beam.typehints.typehints.validate_composite_type_param()` + for further details. Returns: - A reference to the instance of this particular PTransform object. This - allows chaining type-hinting related methods. + PTransform: A reference to the instance of this particular + :class:`PTransform` object. This allows chaining type-hinting related + methods. """ validate_composite_type_param(type_hint, 'Type hints for a PTransform') return super(PTransform, self).with_output_types(type_hint) @@ -491,13 +497,16 @@ def expand(self, pval): class PTransformWithSideInputs(PTransform): - """A superclass for any PTransform (e.g. FlatMap or Combine) + """A superclass for any :class:`PTransform` (e.g. + :func:`~apache_beam.transforms.core.FlatMap` or + :class:`~apache_beam.transforms.core.CombineFn`) invoking user code. - PTransforms like FlatMap invoke user-supplied code in some kind of - package (e.g. a DoFn) and optionally provide arguments and side inputs - to that code. This internal-use-only class contains common functionality - for PTransforms that fit this model. + :class:`PTransform` s like :func:`~apache_beam.transforms.core.FlatMap` + invoke user-supplied code in some kind of package (e.g. a + :class:`~apache_beam.transforms.core.DoFn`) and optionally provide arguments + and side inputs to that code. This internal-use-only class contains common + functionality for :class:`PTransform` s that fit this model. """ def __init__(self, fn, *args, **kwargs): @@ -543,16 +552,20 @@ def with_input_types( of an allowed built-in type, a custom class, or a typehints.TypeConstraint. - Example of annotating the types of side-inputs: + Example of annotating the types of side-inputs:: + FlatMap().with_input_types(int, int, bool) Raises: - TypeError: If 'type_hint' is not a valid type-hint. See - typehints.validate_composite_type_param for further details. + :class:`~exceptions.TypeError`: If **type_hint** is not a valid type-hint. + See + :func:`~apache_beam.typehints.typehints.validate_composite_type_param` + for further details. Returns: - A reference to the instance of this particular PTransform object. This - allows chaining type-hinting related methods. + :class:`PTransform`: A reference to the instance of this particular + :class:`PTransform` object. This allows chaining type-hinting related + methods. """ super(PTransformWithSideInputs, self).with_input_types(input_type_hint) diff --git a/sdks/python/apache_beam/typehints/decorators.py b/sdks/python/apache_beam/typehints/decorators.py index d5954e21af497..3f5b4c9c479bf 100644 --- a/sdks/python/apache_beam/typehints/decorators.py +++ b/sdks/python/apache_beam/typehints/decorators.py @@ -310,38 +310,54 @@ def with_input_types(*positional_hints, **keyword_hints): be type-hinted in totality if even one parameter is type-hinted. Once fully decorated, if the arguments passed to the resulting function - violate the type-hint constraints defined, a TypeCheckError detailing the - error will be raised. + violate the type-hint constraints defined, a :class:`TypeCheckError` + detailing the error will be raised. - To be used as:: + To be used as: - * @with_input_types(s=str) # just @with_input_types(str) will work too. - def upper(s): - return s.upper() + .. testcode:: - Or:: + from apache_beam.typehints import with_input_types - * @with_input_types(ls=List[Tuple[int, int]) - def increment(ls): - [(i + 1, j + 1) for (i,j) in ls] + @with_input_types(str) + def upper(s): + return s.upper() + + Or: + + .. testcode:: + + from apache_beam.typehints import with_input_types + from apache_beam.typehints import List + from apache_beam.typehints import Tuple + + @with_input_types(ls=List[Tuple[int, int]]) + def increment(ls): + [(i + 1, j + 1) for (i,j) in ls] Args: *positional_hints: Positional type-hints having identical order as the function's formal arguments. Values for this argument must either be a - built-in Python type or an instance of a TypeContraint created by - 'indexing' a CompositeTypeHint instance with a type parameter. + built-in Python type or an instance of a + :class:`~apache_beam.typehints.typehints.TypeConstraint` created by + 'indexing' a + :class:`~apache_beam.typehints.typehints.CompositeTypeHint` instance + with a type parameter. **keyword_hints: Keyword arguments mirroring the names of the parameters to the decorated functions. The value of each keyword argument must either be one of the allowed built-in Python types, a custom class, or an - instance of a TypeContraint created by 'indexing' a CompositeTypeHint - instance with a type parameter. + instance of a :class:`~apache_beam.typehints.typehints.TypeConstraint` + created by 'indexing' a + :class:`~apache_beam.typehints.typehints.CompositeTypeHint` instance + with a type parameter. Raises: - ValueError: If not all function arguments have corresponding type-hints - specified. Or if the inner wrapper function isn't passed a function - object. - TypeCheckError: If the any of the passed type-hint constraints are not a - type or TypeContraint instance. + :class:`~exceptions.ValueError`: If not all function arguments have + corresponding type-hints specified. Or if the inner wrapper function isn't + passed a function object. + :class:`TypeCheckError`: If the any of the passed type-hint + constraints are not a type or + :class:`~apache_beam.typehints.typehints.TypeConstraint` instance. Returns: The original function decorated such that it enforces type-hint constraints @@ -375,37 +391,53 @@ def with_output_types(*return_type_hint, **kwargs): Only a single type-hint is accepted to specify the return type of the return value. If the function to be decorated has multiple return values, then one - should use: 'Tuple[type_1, type_2]' to annotate the types of the return + should use: ``Tuple[type_1, type_2]`` to annotate the types of the return values. If the ultimate return value for the function violates the specified type-hint - a TypeCheckError will be raised detailing the type-constraint violation. + a :class:`TypeCheckError` will be raised detailing the type-constraint + violation. + + This decorator is intended to be used like: + + .. testcode:: + + from apache_beam.typehints import with_output_types + from apache_beam.typehints import Set + + class Coordinate: + def __init__(self, x, y): + self.x = x + self.y = y + + @with_output_types(Set[Coordinate]) + def parse_ints(ints): + return {Coordinate(i, i) for i in ints} - This decorator is intended to be used like:: + Or with a simple type-hint: - * @with_output_types(Set[Coordinate]) - def parse_ints(ints): - .... - return [Coordinate.from_int(i) for i in ints] + .. testcode:: - Or with a simple type-hint:: + from apache_beam.typehints import with_output_types - * @with_output_types(bool) - def negate(p): - return not p if p else p + @with_output_types(bool) + def negate(p): + return not p if p else p Args: *return_type_hint: A type-hint specifying the proper return type of the function. This argument should either be a built-in Python type or an - instance of a 'TypeConstraint' created by 'indexing' a - 'CompositeTypeHint'. + instance of a :class:`~apache_beam.typehints.typehints.TypeConstraint` + created by 'indexing' a + :class:`~apache_beam.typehints.typehints.CompositeTypeHint`. **kwargs: Not used. Raises: - ValueError: If any kwarg parameters are passed in, or the length of - 'return_type_hint' is greater than 1. Or if the inner wrapper function - isn't passed a function object. - TypeCheckError: If the 'return_type_hint' object is in invalid type-hint. + :class:`~exceptions.ValueError`: If any kwarg parameters are passed in, + or the length of **return_type_hint** is greater than ``1``. Or if the + inner wrapper function isn't passed a function object. + :class:`TypeCheckError`: If the **return_type_hint** object is + in invalid type-hint. Returns: The original function decorated such that it enforces type-hint constraints diff --git a/sdks/python/apache_beam/typehints/native_type_compatibility.py b/sdks/python/apache_beam/typehints/native_type_compatibility.py index d88f93308dd11..26c584e380d2b 100644 --- a/sdks/python/apache_beam/typehints/native_type_compatibility.py +++ b/sdks/python/apache_beam/typehints/native_type_compatibility.py @@ -82,13 +82,14 @@ def convert_to_beam_type(typ): """Convert a given typing type to a Beam type. Args: - typ: typing type. + typ (type): typing type. Returns: - The given type converted to a Beam type as far as we can do the conversion. + type: The given type converted to a Beam type as far as we can do the + conversion. Raises: - ValueError: The type was malformed. + ~exceptions.ValueError: The type was malformed. """ type_map = [ diff --git a/sdks/python/apache_beam/typehints/typehints.py b/sdks/python/apache_beam/typehints/typehints.py index 6039e0e7e20a0..98d399b82f3b8 100644 --- a/sdks/python/apache_beam/typehints/typehints.py +++ b/sdks/python/apache_beam/typehints/typehints.py @@ -73,7 +73,6 @@ 'Union', 'Optional', 'Tuple', - 'Tuple', 'List', 'KV', 'Dict', @@ -109,9 +108,10 @@ class TypeConstraint(object): """The base-class for all created type-constraints defined below. - A TypeConstraint is the result of parameterizing a CompositeTypeHint with - with one of the allowed Python types or another CompositeTypeHint. It - binds and enforces a specific version of a generalized TypeHint. + A :class:`TypeConstraint` is the result of parameterizing a + :class:`CompositeTypeHint` with with one of the allowed Python types or + another :class:`CompositeTypeHint`. It binds and enforces a specific + version of a generalized TypeHint. """ def _consistent_with_check_(self, sub): @@ -135,12 +135,14 @@ def type_check(self, instance): instance: An instance of a Python object. Raises: - TypeError: The passed 'instance' doesn't satisfy this TypeConstraint. - Subclasses of TypeConstraint are free to raise any of the subclasses of - TypeError defined above, depending on the manner of the type hint error. - - All TypeConstraint sub-classes must define this method in other for the - class object to be created. + :class:`~exceptions.TypeError`: The passed **instance** doesn't satisfy + this :class:`TypeConstraint`. Subclasses of + :class:`TypeConstraint` are free to raise any of the subclasses of + :class:`~exceptions.TypeError` defined above, depending on + the manner of the type hint error. + + All :class:`TypeConstraint` sub-classes must define this method in other + for the class object to be created. """ raise NotImplementedError @@ -296,19 +298,21 @@ def __getitem___(self, py_type): def validate_composite_type_param(type_param, error_msg_prefix): - """Determines if an object is a valid type parameter to a CompositeTypeHint. + """Determines if an object is a valid type parameter to a + :class:`CompositeTypeHint`. + + Implements sanity checking to disallow things like:: - Implements sanity checking to disallow things like: - * List[1, 2, 3] or Dict[5]. + List[1, 2, 3] or Dict[5]. Args: type_param: An object instance. - error_msg_prefix: A string prefix used to format an error message in the - case of an exception. + error_msg_prefix (:class:`str`): A string prefix used to format an error + message in the case of an exception. Raises: - TypeError: If the passed 'type_param' is not a valid type parameter for a - CompositeTypeHint. + ~exceptions.TypeError: If the passed **type_param** is not a valid type + parameter for a :class:`CompositeTypeHint`. """ # Must either be a TypeConstraint instance or a basic Python type. is_not_type_constraint = ( diff --git a/sdks/python/generate_pydoc.sh b/sdks/python/generate_pydoc.sh index 1fea6f12a1c5a..662bd098e1099 100755 --- a/sdks/python/generate_pydoc.sh +++ b/sdks/python/generate_pydoc.sh @@ -31,43 +31,132 @@ rm -rf target/docs/* mkdir -p target/docs/source -# Exclude internal/experimental files from the documentation. -excluded_internal_code=( +# Sphinx apidoc autodoc options +export SPHINX_APIDOC_OPTIONS=\ +members,\ +undoc-members,\ +show-inheritance + +# Exclude internal, test, and Cython paths/patterns from the documentation. +excluded_patterns=( + apache_beam/coders/stream.* + apache_beam/coders/coder_impl.* apache_beam/examples/ apache_beam/internal/clients/ - apache_beam/io/gcp/internal/clients/ + apache_beam/io/gcp/internal/ + apache_beam/io/gcp/tests/ + apache_beam/metrics/execution.* + apache_beam/runners/common.* apache_beam/runners/api/ apache_beam/runners/test/ + apache_beam/runners/dataflow/internal/ apache_beam/runners/portability/ apache_beam/runners/worker/ - apache_beam/runners/dataflow/internal/clients/ - apache_beam/testing/data/) - -python $(type -p sphinx-apidoc) -f -o target/docs/source apache_beam \ - "${excluded_internal_code[@]}" "*_test.py" + apache_beam/transforms/cy_combiners.* + apache_beam/utils/counters.* + apache_beam/utils/windowed_value.* + *_pb2.py + *_test.py + *_test_common.py +) -# Remove Cython modules from doc template; they won't load -sed -i -e '/.. automodule:: apache_beam.coders.stream/d' \ - target/docs/source/apache_beam.coders.rst +python $(type -p sphinx-apidoc) -fMeT -o target/docs/source apache_beam \ + "${excluded_patterns[@]}" # Create the configuration and index files +#=== conf.py ===# cat > target/docs/source/conf.py <<'EOF' import os import sys +import sphinx_rtd_theme + sys.path.insert(0, os.path.abspath('../../..')) +exclude_patterns = [ + '_build', + 'target/docs/source/apache_beam.rst', +] + extensions = [ 'sphinx.ext.autodoc', + 'sphinx.ext.doctest', + 'sphinx.ext.intersphinx', 'sphinx.ext.napoleon', 'sphinx.ext.viewcode', ] master_doc = 'index' -html_theme = 'sphinxdoc' +html_theme = 'sphinx_rtd_theme' +html_theme_path = [sphinx_rtd_theme.get_html_theme_path()] project = 'Apache Beam' + +autoclass_content = 'both' +autodoc_member_order = 'bysource' + +doctest_global_setup = ''' +import apache_beam as beam +''' + +intersphinx_mapping = { + 'python': ('https://docs.python.org/2', None), + 'hamcrest': ('https://pyhamcrest.readthedocs.io/en/latest/', None), +} + +# Since private classes are skipped by sphinx, if there is any cross reference +# to them, it will be broken. This can happen if a class inherits from a +# private class. +ignore_identifiers = [ + # Ignore "custom" builtin types + '', + 'Any', + 'Dict', + 'Iterable', + 'List', + 'Set', + 'Tuple', + + # Ignore private classes + 'apache_beam.coders.coders._PickleCoderBase', + 'apache_beam.coders.coders.FastCoder', + 'apache_beam.io._AvroSource', + 'apache_beam.io.gcp.bigquery.RowAsDictJsonCoder', + 'apache_beam.io.gcp.datastore.v1.datastoreio._Mutate', + 'apache_beam.io.gcp.internal.clients.bigquery.' + 'bigquery_v2_messages.TableSchema', + 'apache_beam.io.iobase.SourceBase', + 'apache_beam.io.source_test_utils.ExpectedSplitOutcome', + 'apache_beam.metrics.metric.MetricResults', + 'apache_beam.pipeline.PipelineVisitor', + 'apache_beam.pipeline.PTransformOverride', + 'apache_beam.pvalue.AsSideInput', + 'apache_beam.pvalue.DoOutputsTuple', + 'apache_beam.pvalue.PValue', + 'apache_beam.runners.direct.executor.CallableTask', + 'apache_beam.transforms.core.CallableWrapperCombineFn', + 'apache_beam.transforms.ptransform.PTransformWithSideInputs', + 'apache_beam.transforms.trigger._ParallelTriggerFn', + 'apache_beam.transforms.trigger.InMemoryUnmergedState', + 'apache_beam.typehints.typehints.AnyTypeConstraint', + 'apache_beam.typehints.typehints.CompositeTypeHint', + 'apache_beam.typehints.typehints.TypeConstraint', + 'apache_beam.typehints.typehints.validate_composite_type_param()', + + # Private classes which are used within the same module + 'WindowedTypeConstraint', # apache_beam.typehints.typehints +] + +# When inferring a base class it will use ':py:class'; if inferring a function +# argument type or return type, it will use ':py:obj'. We'll generate both. +nitpicky = True +nitpick_ignore = [] +nitpick_ignore += [('py:class', iden) for iden in ignore_identifiers] +nitpick_ignore += [('py:obj', iden) for iden in ignore_identifiers] EOF + +#=== index.rst ===# cat > target/docs/source/index.rst <<'EOF' -.. include:: ./modules.rst +.. include:: ./apache_beam.rst + :start-line: 2 EOF # Build the documentation using sphinx @@ -76,10 +165,21 @@ python $(type -p sphinx-build) -v -a -E -q target/docs/source \ target/docs/_build -c target/docs/source \ -w "target/docs/sphinx-build.warnings.log" +# Fail if there are errors or warnings in docs +! grep -q "ERROR:" target/docs/sphinx-build.warnings.log || exit 1 +! grep -q "WARNING:" target/docs/sphinx-build.warnings.log || exit 1 + +# Run tests for code samples, these can be: +# - Code blocks using '.. testsetup::', '.. testcode::' and '.. testoutput::' +# - Interactive code starting with '>>>' +python -msphinx -M doctest target/docs/source \ + target/docs/_build -c target/docs/source \ + -w "target/docs/sphinx-doctest.warnings.log" + +# Fail if there are errors or warnings in docs +! grep -q "ERROR:" target/docs/sphinx-doctest.warnings.log || exit 1 +! grep -q "WARNING:" target/docs/sphinx-doctest.warnings.log || exit 1 + # Message is useful only when this script is run locally. In a remote # test environment, this path will be removed when the test completes. echo "Browse to file://$PWD/target/docs/_build/index.html" - -# Fail if there are errors or warnings in docs -! grep -q "ERROR:" target/docs/sphinx-build.warnings.log -! grep -q "WARNING:" target/docs/sphinx-build.warnings.log diff --git a/sdks/python/tox.ini b/sdks/python/tox.ini index eff91fece9862..fea3854fd34aa 100644 --- a/sdks/python/tox.ini +++ b/sdks/python/tox.ini @@ -99,6 +99,7 @@ deps= nose==1.3.7 grpcio-tools==1.3.5 Sphinx==1.5.5 + sphinx_rtd_theme==0.2.4 commands = pip install -e .[test,gcp,docs] {toxinidir}/generate_pydoc.sh From 1f2ddddabf541b88f01b17aa9a549081a8607bb9 Mon Sep 17 00:00:00 2001 From: Maria Garcia Herrero Date: Thu, 3 Aug 2017 00:16:54 -0700 Subject: [PATCH 267/346] Add initial bundle retry code --- .../apache_beam/options/pipeline_options.py | 7 ++ sdks/python/apache_beam/pipeline_test.py | 30 ++++++ .../apache_beam/runners/direct/executor.py | 100 +++++++++++------- 3 files changed, 100 insertions(+), 37 deletions(-) diff --git a/sdks/python/apache_beam/options/pipeline_options.py b/sdks/python/apache_beam/options/pipeline_options.py index ea996a3d9fb56..db65b3c658370 100644 --- a/sdks/python/apache_beam/options/pipeline_options.py +++ b/sdks/python/apache_beam/options/pipeline_options.py @@ -314,6 +314,13 @@ def _add_argparse_args(cls, parser): help='DirectRunner uses stacked WindowedValues within a Bundle for ' 'memory optimization. Set --no_direct_runner_use_stacked_bundle to ' 'avoid it.') + parser.add_argument( + '--direct_runner_bundle_retry', + action='store_true', + default=False, + help= + ('Whether to allow bundle retries. If True the maximum' + 'number of attempts to process a bundle is 4. ')) class GoogleCloudOptions(PipelineOptions): diff --git a/sdks/python/apache_beam/pipeline_test.py b/sdks/python/apache_beam/pipeline_test.py index aad01435fd9ea..b3ac100780fe2 100644 --- a/sdks/python/apache_beam/pipeline_test.py +++ b/sdks/python/apache_beam/pipeline_test.py @@ -499,6 +499,36 @@ def __reduce__(self): self.assertEqual(MyPTransform.pickle_count[0], 20) +class DirectRunnerRetryTests(unittest.TestCase): + + def test_retry_fork_graph(self): + pipeline_options = PipelineOptions(['--direct_runner_bundle_retry']) + p = beam.Pipeline(options=pipeline_options) + + # TODO(mariagh): Remove the use of globals from the test. + global count_b, count_c # pylint: disable=global-variable-undefined + count_b, count_c = 0, 0 + + def f_b(x): + global count_b # pylint: disable=global-variable-undefined + count_b += 1 + raise Exception('exception in f_b') + + def f_c(x): + global count_c # pylint: disable=global-variable-undefined + count_c += 1 + raise Exception('exception in f_c') + + names = p | 'CreateNodeA' >> beam.Create(['Ann', 'Joe']) + + fork_b = names | 'SendToB' >> beam.Map(f_b) # pylint: disable=unused-variable + fork_c = names | 'SendToC' >> beam.Map(f_c) # pylint: disable=unused-variable + + with self.assertRaises(Exception): + p.run().wait_until_finish() + assert count_b == count_c == 4 + + if __name__ == '__main__': logging.getLogger().setLevel(logging.DEBUG) unittest.main() diff --git a/sdks/python/apache_beam/runners/direct/executor.py b/sdks/python/apache_beam/runners/direct/executor.py index e70e326978842..2e46978575532 100644 --- a/sdks/python/apache_beam/runners/direct/executor.py +++ b/sdks/python/apache_beam/runners/direct/executor.py @@ -25,10 +25,12 @@ import Queue import sys import threading +import traceback from weakref import WeakValueDictionary from apache_beam.metrics.execution import MetricsContainer from apache_beam.metrics.execution import ScopedMetricsContainer +from apache_beam.options.pipeline_options import DirectOptions class _ExecutorService(object): @@ -271,6 +273,15 @@ def __init__(self, transform_evaluator_registry, evaluation_context, self._side_input_values = {} self.blocked = False self._call_count = 0 + self._retry_count = 0 + # Switch to turn on/off the retry of bundles. + pipeline_options = self._evaluation_context.pipeline_options + if not pipeline_options.view_as(DirectOptions).direct_runner_bundle_retry: + self._max_retries_per_bundle = 1 + else: + self._max_retries_per_bundle = 4 + # TODO(mariagh): make _max_retries_per_bundle a constant + # once "bundle retry" is no longer experimental. def call(self): self._call_count += 1 @@ -288,47 +299,62 @@ def call(self): # available. return self._side_input_values[side_input] = value - side_input_values = [self._side_input_values[side_input] for side_input in self._applied_ptransform.side_inputs] - try: - evaluator = self._transform_evaluator_registry.get_evaluator( - self._applied_ptransform, self._input_bundle, - side_input_values, scoped_metrics_container) - - if self._fired_timers: - for timer_firing in self._fired_timers: - evaluator.process_timer_wrapper(timer_firing) - - if self._input_bundle: - for value in self._input_bundle.get_elements_iterable(): - evaluator.process_element(value) - - with scoped_metrics_container: - result = evaluator.finish_bundle() - result.logical_metric_updates = metrics_container.get_cumulative() - - if self._evaluation_context.has_cache: - for uncommitted_bundle in result.uncommitted_output_bundles: + while self._retry_count < self._max_retries_per_bundle: + try: + self.attempt_call(metrics_container, + scoped_metrics_container, + side_input_values) + break + except Exception as e: + self._retry_count += 1 + logging.info( + 'Exception at bundle %r, due to an exception: %s', + self._input_bundle, traceback.format_exc()) + if self._retry_count == self._max_retries_per_bundle: + logging.error('Giving up after %s attempts.', + self._max_retries_per_bundle) + self._completion_callback.handle_exception(self, e) + + self._evaluation_context.metrics().commit_physical( + self._input_bundle, + metrics_container.get_cumulative()) + self._transform_evaluation_state.complete(self) + + def attempt_call(self, metrics_container, + scoped_metrics_container, + side_input_values): + evaluator = self._transform_evaluator_registry.get_evaluator( + self._applied_ptransform, self._input_bundle, + side_input_values, scoped_metrics_container) + + if self._fired_timers: + for timer_firing in self._fired_timers: + evaluator.process_timer_wrapper(timer_firing) + + if self._input_bundle: + for value in self._input_bundle.get_elements_iterable(): + evaluator.process_element(value) + + with scoped_metrics_container: + result = evaluator.finish_bundle() + result.logical_metric_updates = metrics_container.get_cumulative() + + if self._evaluation_context.has_cache: + for uncommitted_bundle in result.uncommitted_output_bundles: + self._evaluation_context.append_to_cache( + self._applied_ptransform, uncommitted_bundle.tag, + uncommitted_bundle.get_elements_iterable()) + undeclared_tag_values = result.undeclared_tag_values + if undeclared_tag_values: + for tag, value in undeclared_tag_values.iteritems(): self._evaluation_context.append_to_cache( - self._applied_ptransform, uncommitted_bundle.tag, - uncommitted_bundle.get_elements_iterable()) - undeclared_tag_values = result.undeclared_tag_values - if undeclared_tag_values: - for tag, value in undeclared_tag_values.iteritems(): - self._evaluation_context.append_to_cache( - self._applied_ptransform, tag, value) - - self._completion_callback.handle_result(self, self._input_bundle, result) - return result - except Exception as e: # pylint: disable=broad-except - self._completion_callback.handle_exception(self, e) - finally: - self._evaluation_context.metrics().commit_physical( - self._input_bundle, - metrics_container.get_cumulative()) - self._transform_evaluation_state.complete(self) + self._applied_ptransform, tag, value) + + self._completion_callback.handle_result(self, self._input_bundle, result) + return result class Executor(object): From e73ae399fcda37ad33dd0e1c04cf0eb3c0548473 Mon Sep 17 00:00:00 2001 From: Kamil Szewczyk Date: Mon, 17 Jul 2017 13:54:10 +0200 Subject: [PATCH 268/346] Throw an Exception if no files are found to stage We should always stage the user's JAR. If we don't find any files and none were specified, then the pipeline will not execute, and this should fail early rather than later. --- .../beam/runners/dataflow/DataflowRunner.java | 14 +++++++++----- 1 file changed, 9 insertions(+), 5 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 6999616f667a8..496681eb59810 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 @@ -243,11 +243,15 @@ public static DataflowRunner fromOptions(PipelineOptions options) { if (dataflowOptions.getFilesToStage() == null) { dataflowOptions.setFilesToStage(detectClassPathResourcesToStage( DataflowRunner.class.getClassLoader())); - LOG.info("PipelineOptions.filesToStage was not specified. " - + "Defaulting to files from the classpath: will stage {} files. " - + "Enable logging at DEBUG level to see which files will be staged.", - dataflowOptions.getFilesToStage().size()); - LOG.debug("Classpath elements: {}", dataflowOptions.getFilesToStage()); + if (dataflowOptions.getFilesToStage().isEmpty()) { + throw new IllegalArgumentException("No files to stage has been found."); + } else { + LOG.info("PipelineOptions.filesToStage was not specified. " + + "Defaulting to files from the classpath: will stage {} files. " + + "Enable logging at DEBUG level to see which files will be staged.", + dataflowOptions.getFilesToStage().size()); + LOG.debug("Classpath elements: {}", dataflowOptions.getFilesToStage()); + } } // Verify jobName according to service requirements, truncating converting to lowercase if From 075d4d45a9cd398f3b4023b6efd495cc58eb9bdd Mon Sep 17 00:00:00 2001 From: Reuven Lax Date: Sun, 30 Jul 2017 11:17:39 -0700 Subject: [PATCH 269/346] Allow users to choose the BigQuery insertion method. If choosing file load jobs on an unbounded PCollection, a triggering frequency must be specified to control how often load jobs are generated. --- .../beam/sdk/io/gcp/bigquery/BatchLoads.java | 447 +++++++++++++----- .../beam/sdk/io/gcp/bigquery/BigQueryIO.java | 168 ++++++- .../sdk/io/gcp/bigquery/ReifyAsIterable.java | 51 ++ .../io/gcp/bigquery/WriteBundlesToFiles.java | 15 +- .../sdk/io/gcp/bigquery/WritePartition.java | 13 +- .../beam/sdk/io/gcp/bigquery/WriteRename.java | 111 +++-- .../beam/sdk/io/gcp/bigquery/WriteTables.java | 38 +- .../sdk/io/gcp/bigquery/BigQueryIOTest.java | 269 ++++++----- 8 files changed, 770 insertions(+), 342 deletions(-) create mode 100644 sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/ReifyAsIterable.java diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BatchLoads.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BatchLoads.java index e46b1d3f945be..0a1306dfc1151 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BatchLoads.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BatchLoads.java @@ -19,6 +19,7 @@ package org.apache.beam.sdk.io.gcp.bigquery; import static com.google.common.base.Preconditions.checkArgument; +import static com.google.common.base.Preconditions.checkState; import static org.apache.beam.sdk.io.gcp.bigquery.BigQueryHelpers.resolveTempLocation; import com.google.api.services.bigquery.model.TableRow; @@ -26,9 +27,10 @@ import com.google.common.base.Strings; import com.google.common.collect.Lists; 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.Coder; +import org.apache.beam.sdk.coders.IterableCoder; import org.apache.beam.sdk.coders.KvCoder; import org.apache.beam.sdk.coders.ListCoder; import org.apache.beam.sdk.coders.NullableCoder; @@ -48,9 +50,15 @@ import org.apache.beam.sdk.transforms.ParDo; import org.apache.beam.sdk.transforms.Reshuffle; import org.apache.beam.sdk.transforms.SimpleFunction; +import org.apache.beam.sdk.transforms.Values; import org.apache.beam.sdk.transforms.View; +import org.apache.beam.sdk.transforms.WithKeys; +import org.apache.beam.sdk.transforms.windowing.AfterFirst; +import org.apache.beam.sdk.transforms.windowing.AfterPane; +import org.apache.beam.sdk.transforms.windowing.AfterProcessingTime; import org.apache.beam.sdk.transforms.windowing.DefaultTrigger; import org.apache.beam.sdk.transforms.windowing.GlobalWindows; +import org.apache.beam.sdk.transforms.windowing.Repeatedly; import org.apache.beam.sdk.transforms.windowing.Window; import org.apache.beam.sdk.util.gcsfs.GcsPath; import org.apache.beam.sdk.values.KV; @@ -62,6 +70,7 @@ import org.apache.beam.sdk.values.TupleTag; import org.apache.beam.sdk.values.TupleTagList; import org.apache.beam.sdk.values.TypeDescriptor; +import org.joda.time.Duration; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -93,6 +102,12 @@ class BatchLoads // The maximum size of a single file - 4TiB, just under the 5 TiB limit. static final long DEFAULT_MAX_FILE_SIZE = 4 * (1L << 40); + static final int DEFAULT_NUM_FILE_SHARDS = 0; + + // If user triggering is supplied, we will trigger the file write after this many records are + // written. + static final int FILE_TRIGGERING_RECORD_COUNT = 500000; + // The maximum number of retries to poll the status of a job. // 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; @@ -110,6 +125,8 @@ class BatchLoads private final Coder destinationCoder; private int maxNumWritersPerBundle; private long maxFileSize; + private int numFileShards; + private Duration triggeringFrequency; BatchLoads(WriteDisposition writeDisposition, CreateDisposition createDisposition, boolean singletonTable, @@ -123,6 +140,8 @@ class BatchLoads this.destinationCoder = destinationCoder; this.maxNumWritersPerBundle = DEFAULT_MAX_NUM_WRITERS_PER_BUNDLE; this.maxFileSize = DEFAULT_MAX_FILE_SIZE; + this.numFileShards = DEFAULT_NUM_FILE_SHARDS; + this.triggeringFrequency = null; } void setTestServices(BigQueryServices bigQueryServices) { @@ -139,6 +158,14 @@ public void setMaxNumWritersPerBundle(int maxNumWritersPerBundle) { this.maxNumWritersPerBundle = maxNumWritersPerBundle; } + public void setTriggeringFrequency(Duration triggeringFrequency) { + this.triggeringFrequency = triggeringFrequency; + } + + public void setNumFileShards(int numFileShards) { + this.numFileShards = numFileShards; + } + @VisibleForTesting void setMaxFileSize(long maxFileSize) { this.maxFileSize = maxFileSize; @@ -164,171 +191,323 @@ public void validate(PipelineOptions options) { } } - @Override - public WriteResult expand(PCollection> input) { + // Expand the pipeline when the user has requested periodically-triggered file writes. + private WriteResult expandTriggered(PCollection> input) { + checkArgument(numFileShards > 0); Pipeline p = input.getPipeline(); + final PCollectionView jobIdTokenView = createJobIdView(p); + final PCollectionView tempFilePrefixView = createTempFilePrefixView(jobIdTokenView); + // The user-supplied triggeringDuration is often chosen to to control how many BigQuery load + // jobs are generated, to prevent going over BigQuery's daily quota for load jobs. If this + // is set to a large value, currently we have to buffer all the data unti the trigger fires. + // Instead we ensure that the files are written if a threshold number of records are ready. + // We use only the user-supplied trigger on the actual BigQuery load. This allows us to + // offload the data to the filesystem. + PCollection> inputInGlobalWindow = + input.apply( + "rewindowIntoGlobal", + Window.>into(new GlobalWindows()) + .triggering( + Repeatedly.forever( + AfterFirst.of( + AfterProcessingTime.pastFirstElementInPane() + .plusDelayOf(triggeringFrequency), + AfterPane.elementCountAtLeast(FILE_TRIGGERING_RECORD_COUNT)))) + .discardingFiredPanes()); + PCollection> results = + writeShardedFiles(inputInGlobalWindow, tempFilePrefixView); + + // Apply the user's trigger before we start generating BigQuery load jobs. + results = + results.apply( + "applyUserTrigger", + Window.>into(new GlobalWindows()) + .triggering( + Repeatedly.forever( + AfterProcessingTime.pastFirstElementInPane() + .plusDelayOf(triggeringFrequency))) + .discardingFiredPanes()); - // Create a singleton job ID token at execution time. This will be used as the base for all - // load jobs issued from this instance of the transform. - final PCollection jobIdToken = - p.apply("TriggerIdCreation", Create.of("ignored")) - .apply( - "CreateJobId", - MapElements.via( - new SimpleFunction() { - @Override - public String apply(String input) { - return BigQueryHelpers.randomUUIDString(); - } - })); - final PCollectionView jobIdTokenView = jobIdToken.apply(View.asSingleton()); + TupleTag, List>> multiPartitionsTag = + new TupleTag, List>>("multiPartitionsTag"); + TupleTag, List>> singlePartitionTag = + new TupleTag, List>>("singlePartitionTag"); - PCollectionView tempFilePrefix = jobIdToken + // If we have non-default triggered output, we can't use the side-input technique used in + // expandUntriggered . Instead make the result list a main input. Apply a GroupByKey first for + // determinism. + PCollectionTuple partitions = + results + .apply( + "AttachSingletonKey", + WithKeys.>of((Void) null)) + .setCoder( + KvCoder.of(VoidCoder.of(), WriteBundlesToFiles.ResultCoder.of(destinationCoder))) + .apply("GroupOntoSingleton", GroupByKey.>create()) + .apply("ExtractResultValues", Values.>>create()) .apply( - "GetTempFilePrefix", + "WritePartitionTriggered", ParDo.of( - new DoFn() { - @ProcessElement - public void getTempFilePrefix(ProcessContext c) { - String tempLocation = resolveTempLocation( - c.getPipelineOptions().getTempLocation(), - "BigQueryWriteTemp", c.element()); - LOG.info("Writing BigQuery temporary files to {} before loading them.", - tempLocation); - c.output(tempLocation); - } - })) - .apply("TempFilePrefixView", View.asSingleton()); + new WritePartition<>( + singletonTable, + dynamicDestinations, + tempFilePrefixView, + multiPartitionsTag, + singlePartitionTag)) + .withSideInputs(tempFilePrefixView) + .withOutputTags(multiPartitionsTag, TupleTagList.of(singlePartitionTag))); + PCollection> tempTables = + writeTempTables(partitions.get(multiPartitionsTag), jobIdTokenView); + tempTables + // Now that the load job has happened, we want the rename to happen immediately. + .apply( + Window.>into(new GlobalWindows()) + .triggering(Repeatedly.forever(AfterPane.elementCountAtLeast(1)))) + .apply(WithKeys.>of((Void) null)) + .setCoder( + KvCoder.of( + VoidCoder.of(), KvCoder.of(TableDestinationCoder.of(), StringUtf8Coder.of()))) + .apply(GroupByKey.>create()) + .apply(Values.>>create()) + .apply( + "WriteRenameTriggered", + ParDo.of( + new WriteRename( + bigQueryServices, jobIdTokenView, writeDisposition, createDisposition)) + .withSideInputs(jobIdTokenView)); + writeSinglePartition(partitions.get(singlePartitionTag), jobIdTokenView); + return writeResult(p); + } + // Expand the pipeline when the user has not requested periodically-triggered file writes. + public WriteResult expandUntriggered(PCollection> input) { + Pipeline p = input.getPipeline(); + final PCollectionView jobIdTokenView = createJobIdView(p); + final PCollectionView tempFilePrefixView = createTempFilePrefixView(jobIdTokenView); PCollection> inputInGlobalWindow = input.apply( "rewindowIntoGlobal", Window.>into(new GlobalWindows()) .triggering(DefaultTrigger.of()) .discardingFiredPanes()); - PCollectionView> schemasView = - inputInGlobalWindow.apply(new CalculateSchemas<>(dynamicDestinations)); + PCollection> results = + (numFileShards == 0) + ? writeDynamicallyShardedFiles(inputInGlobalWindow, tempFilePrefixView) + : writeShardedFiles(inputInGlobalWindow, tempFilePrefixView); + + TupleTag, List>> multiPartitionsTag = + new TupleTag, List>>("multiPartitionsTag") {}; + TupleTag, List>> singlePartitionTag = + new TupleTag, List>>("singlePartitionTag") {}; + + // This transform will look at the set of files written for each table, and if any table has + // too many files or bytes, will partition that table's files into multiple partitions for + // loading. + PCollectionTuple partitions = + results + .apply("ReifyResults", new ReifyAsIterable>()) + .setCoder(IterableCoder.of(WriteBundlesToFiles.ResultCoder.of(destinationCoder))) + .apply( + "WritePartitionUntriggered", + ParDo.of( + new WritePartition<>( + singletonTable, + dynamicDestinations, + tempFilePrefixView, + multiPartitionsTag, + singlePartitionTag)) + .withSideInputs(tempFilePrefixView) + .withOutputTags(multiPartitionsTag, TupleTagList.of(singlePartitionTag))); + PCollection> tempTables = + writeTempTables(partitions.get(multiPartitionsTag), jobIdTokenView); + + tempTables + .apply("ReifyRenameInput", new ReifyAsIterable>()) + .setCoder(IterableCoder.of(KvCoder.of(TableDestinationCoder.of(), StringUtf8Coder.of()))) + .apply( + "WriteRenameUntriggered", + ParDo.of( + new WriteRename( + bigQueryServices, jobIdTokenView, writeDisposition, createDisposition)) + .withSideInputs(jobIdTokenView)); + writeSinglePartition(partitions.get(singlePartitionTag), jobIdTokenView); + return writeResult(p); + } + // Generate the base job id string. + private PCollectionView createJobIdView(Pipeline p) { + // Create a singleton job ID token at execution time. This will be used as the base for all + // load jobs issued from this instance of the transform. + return p.apply("JobIdCreationRoot", Create.of((Void) null)) + .apply( + "CreateJobId", + MapElements.via( + new SimpleFunction() { + @Override + public String apply(Void input) { + return BigQueryHelpers.randomUUIDString(); + } + })) + .apply(View.asSingleton()); + } + + // Generate the temporary-file prefix. + private PCollectionView createTempFilePrefixView(PCollectionView jobIdView) { + return ((PCollection) jobIdView.getPCollection()) + .apply( + "GetTempFilePrefix", + ParDo.of( + new DoFn() { + @ProcessElement + public void getTempFilePrefix(ProcessContext c) { + String tempLocation = + resolveTempLocation( + c.getPipelineOptions().getTempLocation(), + "BigQueryWriteTemp", + c.element()); + LOG.info( + "Writing BigQuery temporary files to {} before loading them.", + tempLocation); + c.output(tempLocation); + } + })) + .apply("TempFilePrefixView", View.asSingleton()); + } + + // Writes input data to dynamically-sharded, per-bundle files. Returns a PCollection of filename, + // file byte size, and table destination. + PCollection> writeDynamicallyShardedFiles( + PCollection> input, PCollectionView tempFilePrefix) { TupleTag> writtenFilesTag = - new TupleTag>("writtenFiles"){}; + new TupleTag>("writtenFiles") {}; TupleTag, TableRow>> unwrittedRecordsTag = new TupleTag, TableRow>>("unwrittenRecords") {}; - PCollectionTuple writeBundlesTuple = inputInGlobalWindow - .apply("WriteBundlesToFiles", - ParDo.of(new WriteBundlesToFiles<>(tempFilePrefix, unwrittedRecordsTag, - maxNumWritersPerBundle, maxFileSize)) - .withSideInputs(tempFilePrefix) - .withOutputTags(writtenFilesTag, TupleTagList.of(unwrittedRecordsTag))); + PCollectionTuple writeBundlesTuple = + input.apply( + "WriteBundlesToFiles", + ParDo.of( + new WriteBundlesToFiles<>( + tempFilePrefix, unwrittedRecordsTag, maxNumWritersPerBundle, maxFileSize)) + .withSideInputs(tempFilePrefix) + .withOutputTags(writtenFilesTag, TupleTagList.of(unwrittedRecordsTag))); PCollection> writtenFiles = - writeBundlesTuple.get(writtenFilesTag) - .setCoder(WriteBundlesToFiles.ResultCoder.of(destinationCoder)); + writeBundlesTuple + .get(writtenFilesTag) + .setCoder(WriteBundlesToFiles.ResultCoder.of(destinationCoder)); + PCollection, TableRow>> unwrittenRecords = + writeBundlesTuple + .get(unwrittedRecordsTag) + .setCoder(KvCoder.of(ShardedKeyCoder.of(destinationCoder), TableRowJsonCoder.of())); // If the bundles contain too many output tables to be written inline to files (due to memory // limits), any unwritten records will be spilled to the unwrittenRecordsTag PCollection. // Group these records by key, and write the files after grouping. Since the record is grouped // by key, we can ensure that only one file is open at a time in each bundle. PCollection> writtenFilesGrouped = - writeBundlesTuple - .get(unwrittedRecordsTag) - .setCoder(KvCoder.of(ShardedKeyCoder.of(destinationCoder), TableRowJsonCoder.of())) - .apply(GroupByKey., TableRow>create()) - .apply( - ParDo.of(new WriteGroupedRecordsToFiles(tempFilePrefix, maxFileSize)) - .withSideInputs(tempFilePrefix)) - .setCoder(WriteBundlesToFiles.ResultCoder.of(destinationCoder)); + writeShardedRecords(unwrittenRecords, tempFilePrefix); // PCollection of filename, file byte size, and table destination. - PCollection> results = - PCollectionList.of(writtenFiles).and(writtenFilesGrouped) - .apply(Flatten.>pCollections()); + return PCollectionList.of(writtenFiles) + .and(writtenFilesGrouped) + .apply("FlattenFiles", Flatten.>pCollections()) + .setCoder(WriteBundlesToFiles.ResultCoder.of(destinationCoder)); + } - TupleTag, List>> multiPartitionsTag = - new TupleTag, List>>("multiPartitionsTag") {}; - TupleTag, List>> singlePartitionTag = - new TupleTag, List>>("singlePartitionTag") {}; + // Writes input data to statically-sharded files. Returns a PCollection of filename, + // file byte size, and table destination. + PCollection> writeShardedFiles( + PCollection> input, PCollectionView tempFilePrefix) { + checkState(numFileShards > 0); + PCollection, TableRow>> shardedRecords = + input + .apply( + "AddShard", + ParDo.of( + new DoFn, KV, TableRow>>() { + int shardNumber; - // Turn the list of files and record counts in a PCollectionView that can be used as a - // side input. - PCollectionView>> resultsView = - results.apply("ResultsView", - View.>asIterable()); - // This transform will look at the set of files written for each table, and if any table has - // too many files or bytes, will partition that table's files into multiple partitions for - // loading. - PCollection singleton = p.apply("singleton", - Create.of((Void) null).withCoder(VoidCoder.of())); - PCollectionTuple partitions = - singleton.apply( - "WritePartition", - ParDo.of( - new WritePartition<>( - singletonTable, - dynamicDestinations, - tempFilePrefix, - resultsView, - multiPartitionsTag, - singlePartitionTag)) - .withSideInputs(tempFilePrefix, resultsView) - .withOutputTags(multiPartitionsTag, TupleTagList.of(singlePartitionTag))); - - List> writeTablesSideInputs = - Lists.newArrayList(jobIdTokenView, schemasView); - writeTablesSideInputs.addAll(dynamicDestinations.getSideInputs()); + @Setup + public void setup() { + shardNumber = ThreadLocalRandom.current().nextInt(numFileShards); + } + + @ProcessElement + public void processElement(ProcessContext c) { + DestinationT destination = c.element().getKey(); + TableRow tableRow = c.element().getValue(); + c.output( + KV.of( + ShardedKey.of(destination, ++shardNumber % numFileShards), + tableRow)); + } + })) + .setCoder(KvCoder.of(ShardedKeyCoder.of(destinationCoder), TableRowJsonCoder.of())); + + return writeShardedRecords(shardedRecords, tempFilePrefix); + } + + private PCollection> writeShardedRecords( + PCollection, TableRow>> shardedRecords, + PCollectionView tempFilePrefix) { + return shardedRecords + .apply("GroupByDestination", GroupByKey., TableRow>create()) + .apply( + "WriteGroupedRecords", + ParDo.of(new WriteGroupedRecordsToFiles(tempFilePrefix, maxFileSize)) + .withSideInputs(tempFilePrefix)) + .setCoder(WriteBundlesToFiles.ResultCoder.of(destinationCoder)); + } + + // Take in a list of files and write them to temporary tables. + private PCollection> writeTempTables( + PCollection, List>> input, + PCollectionView jobIdTokenView) { + List> sideInputs = Lists.>newArrayList(jobIdTokenView); + sideInputs.addAll(dynamicDestinations.getSideInputs()); Coder, List>> partitionsCoder = - KvCoder.of( - ShardedKeyCoder.of(NullableCoder.of(destinationCoder)), - ListCoder.of(StringUtf8Coder.of())); + KvCoder.of( + ShardedKeyCoder.of(NullableCoder.of(destinationCoder)), + ListCoder.of(StringUtf8Coder.of())); // If WriteBundlesToFiles produced more than MAX_NUM_FILES files or MAX_SIZE_BYTES bytes, then // the import needs to be split into multiple partitions, and those partitions will be // specified in multiPartitionsTag. - PCollection> tempTables = - partitions - .get(multiPartitionsTag) - .setCoder(partitionsCoder) - // Reshuffle will distribute this among multiple workers, and also guard against - // reexecution of the WritePartitions step once WriteTables has begun. - .apply( - "MultiPartitionsReshuffle", - Reshuffle., List>of()) - .apply( - "MultiPartitionsWriteTables", - ParDo.of( - new WriteTables<>( - false, - bigQueryServices, - jobIdTokenView, - schemasView, - WriteDisposition.WRITE_EMPTY, - CreateDisposition.CREATE_IF_NEEDED, - dynamicDestinations)) - .withSideInputs(writeTablesSideInputs)); - - // This view maps each final table destination to the set of temporary partitioned tables - // the PCollection was loaded into. - PCollectionView>> tempTablesView = - tempTables.apply("TempTablesView", View.asMultimap()); - - singleton.apply( - "WriteRename", - ParDo.of( - new WriteRename( - bigQueryServices, - jobIdTokenView, - writeDisposition, - createDisposition, - tempTablesView)) - .withSideInputs(tempTablesView, jobIdTokenView)); + return input + .setCoder(partitionsCoder) + // Reshuffle will distribute this among multiple workers, and also guard against + // reexecution of the WritePartitions step once WriteTables has begun. + .apply("MultiPartitionsReshuffle", Reshuffle., List>of()) + .apply( + "MultiPartitionsWriteTables", + ParDo.of( + new WriteTables<>( + false, + bigQueryServices, + jobIdTokenView, + WriteDisposition.WRITE_EMPTY, + CreateDisposition.CREATE_IF_NEEDED, + dynamicDestinations)) + .withSideInputs(sideInputs)); + } + // In the case where the files fit into a single load job, there's no need to write temporary + // tables and rename. We can load these files directly into the target BigQuery table. + void writeSinglePartition( + PCollection, List>> input, + PCollectionView jobIdTokenView) { + List> sideInputs = Lists.>newArrayList(jobIdTokenView); + sideInputs.addAll(dynamicDestinations.getSideInputs()); + Coder, List>> partitionsCoder = + KvCoder.of( + ShardedKeyCoder.of(NullableCoder.of(destinationCoder)), + ListCoder.of(StringUtf8Coder.of())); // Write single partition to final table - partitions - .get(singlePartitionTag) + input .setCoder(partitionsCoder) // Reshuffle will distribute this among multiple workers, and also guard against // reexecution of the WritePartitions step once WriteTables has begun. - .apply( - "SinglePartitionsReshuffle", Reshuffle., List>of()) + .apply("SinglePartitionsReshuffle", Reshuffle., List>of()) .apply( "SinglePartitionWriteTables", ParDo.of( @@ -336,14 +515,20 @@ public void getTempFilePrefix(ProcessContext c) { true, bigQueryServices, jobIdTokenView, - schemasView, writeDisposition, createDisposition, dynamicDestinations)) - .withSideInputs(writeTablesSideInputs)); + .withSideInputs(sideInputs)); + } + private WriteResult writeResult(Pipeline p) { PCollection empty = p.apply("CreateEmptyFailedInserts", Create.empty(TypeDescriptor.of(TableRow.class))); - return WriteResult.in(input.getPipeline(), new TupleTag("failedInserts"), empty); + return WriteResult.in(p, new TupleTag("failedInserts"), empty); + } + + @Override + public WriteResult expand(PCollection> input) { + return (triggeringFrequency != null) ? expandTriggered(input) : expandUntriggered(input); } } 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 6edbd06c0ed06..feb085db6b52f 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 @@ -92,19 +92,20 @@ import org.apache.beam.sdk.values.TupleTag; import org.apache.beam.sdk.values.TupleTagList; import org.apache.beam.sdk.values.ValueInSingleWindow; +import org.joda.time.Duration; import org.slf4j.Logger; import org.slf4j.LoggerFactory; /** - * {@link PTransform}s for reading and writing - * BigQuery tables. + * {@link PTransform}s for reading and writing BigQuery tables. * *

              Table References

              * *

              A fully-qualified BigQuery table name consists of three components: + * *

                - *
              • {@code projectId}: the Cloud project id (defaults to - * {@link GcpOptions#getProject()}). + *
              • {@code projectId}: the Cloud project id (defaults to {@link GcpOptions#getProject()}). *
              • {@code datasetId}: the BigQuery dataset id, unique within a project. *
              • {@code tableId}: a table id, unique within a dataset. *
              @@ -122,8 +123,8 @@ * *

              Reading

              * - *

              To read from a BigQuery table, apply a {@link BigQueryIO.Read} transformation. - * This produces a {@link PCollection} of {@link TableRow TableRows} as output: + *

              To read from a BigQuery table, apply a {@link BigQueryIO.Read} transformation. This produces a + * {@link PCollection} of {@link TableRow TableRows} as output: * *

              {@code
                * PCollection weatherData = pipeline.apply(
              @@ -146,12 +147,11 @@
                *
                * 

              Writing

              * - *

              To write to a BigQuery table, apply a {@link BigQueryIO.Write} transformation. - * This consumes either a {@link PCollection} of {@link TableRow TableRows} as input when using - * {@link BigQueryIO#writeTableRows()} or of a user-defined type when using - * {@link BigQueryIO#write()}. When using a user-defined type, a function must be provided to - * turn this type into a {@link TableRow} using - * {@link BigQueryIO.Write#withFormatFunction(SerializableFunction)}. + *

              To write to a BigQuery table, apply a {@link BigQueryIO.Write} transformation. This consumes + * either a {@link PCollection} of {@link TableRow TableRows} as input when using {@link + * BigQueryIO#writeTableRows()} or of a user-defined type when using {@link BigQueryIO#write()}. + * When using a user-defined type, a function must be provided to turn this type into a {@link + * TableRow} using {@link BigQueryIO.Write#withFormatFunction(SerializableFunction)}. * *

              {@code
                * PCollection quotes = ...
              @@ -216,8 +216,8 @@
                * can also be useful when writing to a single table, as it allows a previous stage to calculate the
                * schema (possibly based on the full collection of records being written to BigQuery).
                *
              - * 

              For the most general form of dynamic table destinations and schemas, look at - * {@link BigQueryIO.Write#to(DynamicDestinations)}. + *

              For the most general form of dynamic table destinations and schemas, look at {@link + * BigQueryIO.Write#to(DynamicDestinations)}. * *

              Permissions

              * @@ -227,6 +227,15 @@ * *

              Please see BigQuery Access Control * for security and permission related information specific to BigQuery. + * + *

              Insertion Method

              + * + * {@link BigQueryIO.Write} supports two methods of inserting data into BigQuery specified using + * {@link BigQueryIO.Write#withMethod}. If no method is supplied, then a default method will be + * chosen based on the input PCollection. See {@link BigQueryIO.Write.Method} for more information + * about the methods. The different insertion methods provide different tradeoffs of cost, quota, + * and data consistency; please see BigQuery documentation for more information about these + * tradeoffs. */ public class BigQueryIO { private static final Logger LOG = LoggerFactory.getLogger(BigQueryIO.class); @@ -757,6 +766,8 @@ public static Write write() { .setBigQueryServices(new BigQueryServicesImpl()) .setCreateDisposition(Write.CreateDisposition.CREATE_IF_NEEDED) .setWriteDisposition(Write.WriteDisposition.WRITE_EMPTY) + .setNumFileShards(0) + .setMethod(Write.Method.DEFAULT) .build(); } @@ -771,6 +782,41 @@ public static Write writeTableRows() { /** Implementation of {@link #write}. */ @AutoValue public abstract static class Write extends PTransform, WriteResult> { + /** Determines the method used to insert data in BigQuery. */ + public enum Method { + /** + * The default behavior if no method is explicitly set. If the input is bounded, then file + * loads will be used. If the input is unbounded, then streaming inserts will be used. + */ + DEFAULT, + + /** + * Use BigQuery load jobs to insert data. Records will first be written to files, and these + * files will be loaded into BigQuery. This is the default method when the input is bounded. + * This method can be chosen for unbounded inputs as well, as long as a triggering frequency + * is also set using {@link #withTriggeringFrequency}. BigQuery has daily quotas on the number + * of load jobs allowed per day, so be careful not to set the triggering frequency too + * frequent. For more information, see Loading Data from + * Cloud Storage. + */ + FILE_LOADS, + + /** + * Use the BigQuery streaming insert API to insert data. This provides the lowest-latency + * insert path into BigQuery, and therefore is the default method when the input is unbounded. + * BigQuery will make a strong effort to ensure no duplicates when using this path, however + * there are some scenarios in which BigQuery is unable to make this guarantee (see + * https://cloud.google.com/bigquery/streaming-data-into-bigquery). A query can be run over + * the output table to periodically clean these rare duplicates. Alternatively, using the + * {@link #FILE_LOADS} insert method does guarantee no duplicates, though the latency for the + * insert into BigQuery will be much higher. For more information, see Streaming Data into + * BigQuery. + */ + STREAMING_INSERTS + } + @Nullable abstract ValueProvider getJsonTableRef(); @Nullable abstract SerializableFunction, TableDestination> getTableFunction(); @@ -787,6 +833,14 @@ public abstract static class Write extends PTransform, WriteRe abstract BigQueryServices getBigQueryServices(); @Nullable abstract Integer getMaxFilesPerBundle(); @Nullable abstract Long getMaxFileSize(); + + abstract int getNumFileShards(); + + @Nullable + abstract Duration getTriggeringFrequency(); + + abstract Method getMethod(); + @Nullable abstract InsertRetryPolicy getFailedInsertRetryPolicy(); abstract Builder toBuilder(); @@ -807,6 +861,13 @@ abstract Builder setTableFunction( abstract Builder setBigQueryServices(BigQueryServices bigQueryServices); abstract Builder setMaxFilesPerBundle(Integer maxFilesPerBundle); abstract Builder setMaxFileSize(Long maxFileSize); + + abstract Builder setNumFileShards(int numFileShards); + + abstract Builder setTriggeringFrequency(Duration triggeringFrequency); + + abstract Builder setMethod(Method method); + abstract Builder setFailedInsertRetryPolicy(InsertRetryPolicy retryPolicy); abstract Write build(); @@ -992,6 +1053,40 @@ public Write withoutValidation() { return toBuilder().setValidate(false).build(); } + /** + * Choose the method used to write data to BigQuery. See the Javadoc on {@link Method} for + * information and restrictions of the different methods. + */ + public Write withMethod(Method method) { + return toBuilder().setMethod(method).build(); + } + + /** + * Choose the frequency at which file writes are triggered. + * + *

              This is only applicable when the write method is set to {@link Method#FILE_LOADS}, and + * only when writing a bounded {@link PCollection}. + * + *

              Every triggeringFrequency duration, a BigQuery load job will be generated for all the data + * written since the last load job. BigQuery has limits on how many load jobs can be triggered + * per day, so be careful not to set this duration too low, or you may exceed daily quota. Often + * this is set to 5 or 10 minutes to ensure that the project stays well under the BigQuery + * quota. See Quota Policy for more + * information about BigQuery quotas. + */ + public Write withTriggeringFrequency(Duration triggeringFrequency) { + return toBuilder().setTriggeringFrequency(triggeringFrequency).build(); + } + + /** + * Control how many file shards are written when using BigQuery load jobs. Applicable only when + * also setting {@link #withTriggeringFrequency}. The default value is 1000. + */ + @Experimental + public Write withNumFileShards(int numFileShards) { + return toBuilder().setNumFileShards(numFileShards).build(); + } + @VisibleForTesting Write withTestServices(BigQueryServices testServices) { return toBuilder().setBigQueryServices(testServices).build(); @@ -1029,6 +1124,17 @@ public void validate(PipelineOptions pipelineOptions) { } } + private Method resolveMethod(PCollection input) { + if (getMethod() != Method.DEFAULT) { + return getMethod(); + } + // By default, when writing an Unbounded PCollection, we use StreamingInserts and + // BigQuery's streaming import API. + return (input.isBounded() == IsBounded.UNBOUNDED) + ? Method.STREAMING_INSERTS + : Method.FILE_LOADS; + } + @Override public WriteResult expand(PCollection input) { // We must have a destination to write to! @@ -1048,6 +1154,7 @@ public WriteResult expand(PCollection input) { || getSchemaFromView() != null, "CreateDisposition is CREATE_IF_NEEDED, however no schema was provided."); + List allToArgs = Lists.newArrayList(getJsonTableRef(), getTableFunction(), getDynamicDestinations()); checkArgument(1 @@ -1061,7 +1168,21 @@ public WriteResult expand(PCollection input) { "No more than one of jsonSchema, schemaFromView, or dynamicDestinations may " + "be set"); - + Method method = resolveMethod(input); + if (input.isBounded() == IsBounded.UNBOUNDED && method == Method.FILE_LOADS) { + checkArgument( + getTriggeringFrequency() != null, + "When writing an unbounded PCollection via FILE_LOADS, " + + "triggering frequency must be specified"); + } else { + checkArgument( + getTriggeringFrequency() == null && getNumFileShards() == 0, + "Triggering frequency or number of file shards can be specified only when writing " + + "an unbounded PCollection via FILE_LOADS, but: the collection was %s " + + "and the method was %s", + input.isBounded(), + method); + } DynamicDestinations dynamicDestinations = getDynamicDestinations(); if (dynamicDestinations == null) { if (getJsonTableRef() != null) { @@ -1069,17 +1190,20 @@ public WriteResult expand(PCollection input) { DynamicDestinationsHelpers.ConstantTableDestinations.fromJsonTableRef( getJsonTableRef(), getTableDescription()); } else if (getTableFunction() != null) { - dynamicDestinations = new TableFunctionDestinations(getTableFunction()); + dynamicDestinations = new TableFunctionDestinations<>(getTableFunction()); } // Wrap with a DynamicDestinations class that will provide a schema. There might be no // schema provided if the create disposition is CREATE_NEVER. if (getJsonSchema() != null) { dynamicDestinations = - new ConstantSchemaDestinations(dynamicDestinations, getJsonSchema()); + new ConstantSchemaDestinations<>( + (DynamicDestinations) dynamicDestinations, getJsonSchema()); } else if (getSchemaFromView() != null) { dynamicDestinations = - new SchemaFromViewDestinations(dynamicDestinations, getSchemaFromView()); + new SchemaFromViewDestinations<>( + (DynamicDestinations) dynamicDestinations, + getSchemaFromView()); } } return expandTyped(input, dynamicDestinations); @@ -1100,9 +1224,9 @@ private WriteResult expandTyped( .apply("PrepareWrite", new PrepareWrite<>(dynamicDestinations, getFormatFunction())) .setCoder(KvCoder.of(destinationCoder, TableRowJsonCoder.of())); - // When writing an Unbounded PCollection, we use StreamingInserts and BigQuery's streaming - // import API. - if (input.isBounded() == IsBounded.UNBOUNDED) { + Method method = resolveMethod(input); + + if (method == Method.STREAMING_INSERTS) { checkArgument( getWriteDisposition() != WriteDisposition.WRITE_TRUNCATE, "WriteDisposition.WRITE_TRUNCATE is not supported for an unbounded" @@ -1129,6 +1253,8 @@ private WriteResult expandTyped( if (getMaxFileSize() != null) { batchLoads.setMaxFileSize(getMaxFileSize()); } + batchLoads.setTriggeringFrequency(getTriggeringFrequency()); + batchLoads.setNumFileShards(getNumFileShards()); return rowsWithDestination.apply(batchLoads); } } diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/ReifyAsIterable.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/ReifyAsIterable.java new file mode 100644 index 0000000000000..18a359cda7cd0 --- /dev/null +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/ReifyAsIterable.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.bigquery; + +import org.apache.beam.sdk.coders.VoidCoder; +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.View; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.PCollectionView; + +/** + * This transforms turns a side input into a singleton PCollection that can be used as the main + * input for another transform. + */ +public class ReifyAsIterable extends PTransform, PCollection>> { + @Override + public PCollection> expand(PCollection input) { + final PCollectionView> view = input.apply(View.asIterable()); + return input + .getPipeline() + .apply(Create.of((Void) null).withCoder(VoidCoder.of())) + .apply( + ParDo.of( + new DoFn>() { + @ProcessElement + public void processElement(ProcessContext c) { + c.output(c.sideInput(view)); + } + }) + .withSideInputs(view)); + } +} diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/WriteBundlesToFiles.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/WriteBundlesToFiles.java index e1ed746b40382..e337f94aab9bb 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/WriteBundlesToFiles.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/WriteBundlesToFiles.java @@ -66,9 +66,10 @@ class WriteBundlesToFiles private transient Map writers; private transient Map writerWindows; private final PCollectionView tempFilePrefixView; - private final TupleTag, TableRow>> unwrittedRecordsTag; + private final TupleTag, TableRow>> unwrittenRecordsTag; private int maxNumWritersPerBundle; private long maxFileSize; + private int spilledShardNumber; /** * The result of the {@link WriteBundlesToFiles} transform. Corresponds to a single output file, @@ -133,11 +134,11 @@ public void verifyDeterministic() {} WriteBundlesToFiles( PCollectionView tempFilePrefixView, - TupleTag, TableRow>> unwrittedRecordsTag, + TupleTag, TableRow>> unwrittenRecordsTag, int maxNumWritersPerBundle, long maxFileSize) { this.tempFilePrefixView = tempFilePrefixView; - this.unwrittedRecordsTag = unwrittedRecordsTag; + this.unwrittenRecordsTag = unwrittenRecordsTag; this.maxNumWritersPerBundle = maxNumWritersPerBundle; this.maxFileSize = maxFileSize; } @@ -148,6 +149,7 @@ public void startBundle() { // bundles. this.writers = Maps.newHashMap(); this.writerWindows = Maps.newHashMap(); + this.spilledShardNumber = ThreadLocalRandom.current().nextInt(SPILLED_RECORD_SHARDING_FACTOR); } TableRowWriter createAndInsertWriter(DestinationT destination, String tempFilePrefix, @@ -174,9 +176,10 @@ public void processElement(ProcessContext c, BoundedWindow window) throws Except } else { // This means that we already had too many writers open in this bundle. "spill" this record // into the output. It will be grouped and written to a file in a subsequent stage. - c.output(unwrittedRecordsTag, - KV.of(ShardedKey.of(destination, - ThreadLocalRandom.current().nextInt(SPILLED_RECORD_SHARDING_FACTOR)), + c.output( + unwrittenRecordsTag, + KV.of( + ShardedKey.of(destination, (++spilledShardNumber) % SPILLED_RECORD_SHARDING_FACTOR), c.element().getValue())); return; } diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/WritePartition.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/WritePartition.java index 451d1bddd606a..934f1aecb1e45 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/WritePartition.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/WritePartition.java @@ -22,6 +22,7 @@ import com.google.common.collect.Maps; import java.util.List; import java.util.Map; +import javax.annotation.Nullable; import org.apache.beam.sdk.io.gcp.bigquery.WriteBundlesToFiles.Result; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.values.KV; @@ -34,12 +35,13 @@ * tablespec and the list of files corresponding to each partition of that table. */ class WritePartition - extends DoFn, List>> { + extends DoFn< + Iterable>, + KV, List>> { private final boolean singletonTable; private final DynamicDestinations dynamicDestinations; private final PCollectionView tempFilePrefix; - private final PCollectionView>> results; - private TupleTag, List>> multiPartitionsTag; + @Nullable private TupleTag, List>> multiPartitionsTag; private TupleTag, List>> singlePartitionTag; private static class PartitionData { @@ -104,12 +106,10 @@ void addPartition(PartitionData partition) { boolean singletonTable, DynamicDestinations dynamicDestinations, PCollectionView tempFilePrefix, - PCollectionView>> results, TupleTag, List>> multiPartitionsTag, TupleTag, List>> singlePartitionTag) { this.singletonTable = singletonTable; this.dynamicDestinations = dynamicDestinations; - this.results = results; this.tempFilePrefix = tempFilePrefix; this.multiPartitionsTag = multiPartitionsTag; this.singlePartitionTag = singlePartitionTag; @@ -117,8 +117,7 @@ void addPartition(PartitionData partition) { @ProcessElement public void processElement(ProcessContext c) throws Exception { - List> results = - Lists.newArrayList(c.sideInput(this.results)); + List> results = Lists.newArrayList(c.element()); // If there are no elements to write _and_ the user specified a constant output table, then // generate an empty table of that name. diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/WriteRename.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/WriteRename.java index f641b327db78b..eb1da5f7ffec8 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/WriteRename.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/WriteRename.java @@ -22,9 +22,11 @@ import com.google.api.services.bigquery.model.JobConfigurationTableCopy; import com.google.api.services.bigquery.model.JobReference; import com.google.api.services.bigquery.model.TableReference; +import com.google.common.collect.ArrayListMultimap; import com.google.common.collect.Lists; -import com.google.common.collect.Maps; +import com.google.common.collect.Multimap; import java.io.IOException; +import java.util.Collection; import java.util.List; import java.util.Map; import javax.annotation.Nullable; @@ -35,74 +37,85 @@ import org.apache.beam.sdk.io.gcp.bigquery.BigQueryServices.JobService; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.display.DisplayData; +import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollectionView; import org.slf4j.Logger; import org.slf4j.LoggerFactory; /** - * Copies temporary tables to destination table. + * Copies temporary tables to destination table. The input element is an {@link Iterable} that + * provides the list of all temporary tables created for a given {@link TableDestination}. */ -class WriteRename extends DoFn { +class WriteRename extends DoFn>, Void> { private static final Logger LOG = LoggerFactory.getLogger(WriteRename.class); private final BigQueryServices bqServices; private final PCollectionView jobIdToken; - private final WriteDisposition writeDisposition; - private final CreateDisposition createDisposition; - // Map from final destination to a list of temporary tables that need to be copied into it. - private final PCollectionView>> tempTablesView; + // In the triggered scenario, the user-supplied create and write dispositions only apply to the + // first trigger pane, as that's when when the table is created. Subsequent loads should always + // append to the table, and so use CREATE_NEVER and WRITE_APPEND dispositions respectively. + private final WriteDisposition firstPaneWriteDisposition; + private final CreateDisposition firstPaneCreateDisposition; public WriteRename( BigQueryServices bqServices, PCollectionView jobIdToken, WriteDisposition writeDisposition, - CreateDisposition createDisposition, - PCollectionView>> tempTablesView) { + CreateDisposition createDisposition) { this.bqServices = bqServices; this.jobIdToken = jobIdToken; - this.writeDisposition = writeDisposition; - this.createDisposition = createDisposition; - this.tempTablesView = tempTablesView; + this.firstPaneWriteDisposition = writeDisposition; + this.firstPaneCreateDisposition = createDisposition; } @ProcessElement public void processElement(ProcessContext c) throws Exception { - Map> tempTablesMap = - Maps.newHashMap(c.sideInput(tempTablesView)); - - // Process each destination table. - for (Map.Entry> entry : tempTablesMap.entrySet()) { - TableDestination finalTableDestination = entry.getKey(); - List tempTablesJson = Lists.newArrayList(entry.getValue()); - // Do not copy if no temp tables are provided - if (tempTablesJson.size() == 0) { - return; - } + Multimap tempTables = ArrayListMultimap.create(); + for (KV entry : c.element()) { + tempTables.put(entry.getKey(), entry.getValue()); + } + for (Map.Entry> entry : tempTables.asMap().entrySet()) { + // Process each destination table. + writeRename(entry.getKey(), entry.getValue(), c); + } + } - List tempTables = Lists.newArrayList(); - for (String table : tempTablesJson) { - tempTables.add(BigQueryHelpers.fromJsonString(table, TableReference.class)); - } + private void writeRename( + TableDestination finalTableDestination, Iterable tempTableNames, ProcessContext c) + throws Exception { + WriteDisposition writeDisposition = + (c.pane().getIndex() == 0) ? firstPaneWriteDisposition : WriteDisposition.WRITE_APPEND; + CreateDisposition createDisposition = + (c.pane().getIndex() == 0) ? firstPaneCreateDisposition : CreateDisposition.CREATE_NEVER; + List tempTablesJson = Lists.newArrayList(tempTableNames); + // Do not copy if no temp tables are provided + if (tempTablesJson.size() == 0) { + return; + } - // Make sure each destination table gets a unique job id. - String jobIdPrefix = BigQueryHelpers.createJobId( - c.sideInput(jobIdToken), finalTableDestination, -1); - - copy( - bqServices.getJobService(c.getPipelineOptions().as(BigQueryOptions.class)), - bqServices.getDatasetService(c.getPipelineOptions().as(BigQueryOptions.class)), - jobIdPrefix, - finalTableDestination.getTableReference(), - tempTables, - writeDisposition, - createDisposition, - finalTableDestination.getTableDescription()); - - DatasetService tableService = - bqServices.getDatasetService(c.getPipelineOptions().as(BigQueryOptions.class)); - removeTemporaryTables(tableService, tempTables); + List tempTables = Lists.newArrayList(); + for (String table : tempTablesJson) { + tempTables.add(BigQueryHelpers.fromJsonString(table, TableReference.class)); } + + // Make sure each destination table gets a unique job id. + String jobIdPrefix = + BigQueryHelpers.createJobId(c.sideInput(jobIdToken), finalTableDestination, -1); + + copy( + bqServices.getJobService(c.getPipelineOptions().as(BigQueryOptions.class)), + bqServices.getDatasetService(c.getPipelineOptions().as(BigQueryOptions.class)), + jobIdPrefix, + finalTableDestination.getTableReference(), + tempTables, + writeDisposition, + createDisposition, + finalTableDestination.getTableDescription()); + + DatasetService tableService = + bqServices.getDatasetService(c.getPipelineOptions().as(BigQueryOptions.class)); + removeTemporaryTables(tableService, tempTables); } private void copy( @@ -174,9 +187,11 @@ public void populateDisplayData(DisplayData.Builder builder) { super.populateDisplayData(builder); builder - .add(DisplayData.item("writeDisposition", writeDisposition.toString()) - .withLabel("Write Disposition")) - .add(DisplayData.item("createDisposition", createDisposition.toString()) - .withLabel("Create Disposition")); + .add( + DisplayData.item("firstPaneWriteDisposition", firstPaneWriteDisposition.toString()) + .withLabel("Write Disposition")) + .add( + DisplayData.item("firstPaneCreateDisposition", firstPaneCreateDisposition.toString()) + .withLabel("Create Disposition")); } } diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/WriteTables.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/WriteTables.java index 9ed2916b36a48..24911a7ccc125 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/WriteTables.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/WriteTables.java @@ -26,6 +26,7 @@ import com.google.common.base.Strings; import com.google.common.collect.ImmutableList; import com.google.common.collect.Lists; +import com.google.common.collect.Maps; import java.io.IOException; import java.util.Collection; import java.util.List; @@ -65,35 +66,48 @@ class WriteTables private final boolean singlePartition; private final BigQueryServices bqServices; private final PCollectionView jobIdToken; - private final PCollectionView> schemasView; - private final WriteDisposition writeDisposition; - private final CreateDisposition createDisposition; + private final WriteDisposition firstPaneWriteDisposition; + private final CreateDisposition firstPaneCreateDisposition; private final DynamicDestinations dynamicDestinations; + private Map jsonSchemas = Maps.newHashMap(); public WriteTables( boolean singlePartition, BigQueryServices bqServices, PCollectionView jobIdToken, - PCollectionView> schemasView, WriteDisposition writeDisposition, CreateDisposition createDisposition, DynamicDestinations dynamicDestinations) { this.singlePartition = singlePartition; this.bqServices = bqServices; this.jobIdToken = jobIdToken; - this.schemasView = schemasView; - this.writeDisposition = writeDisposition; - this.createDisposition = createDisposition; + this.firstPaneWriteDisposition = writeDisposition; + this.firstPaneCreateDisposition = createDisposition; this.dynamicDestinations = dynamicDestinations; } + @StartBundle + public void startBundle(StartBundleContext c) { + // Clear the map on each bundle so we can notice side-input updates. + // (alternative is to use a cache with a TTL). + jsonSchemas.clear(); + } + @ProcessElement public void processElement(ProcessContext c) throws Exception { dynamicDestinations.setSideInputAccessorFromProcessContext(c); DestinationT destination = c.element().getKey().getKey(); - TableSchema tableSchema = - BigQueryHelpers.fromJsonString( - c.sideInput(schemasView).get(destination), TableSchema.class); + TableSchema tableSchema; + String jsonSchema = jsonSchemas.get(destination); + if (jsonSchema != null) { + tableSchema = BigQueryHelpers.fromJsonString(jsonSchema, TableSchema.class); + } else { + tableSchema = dynamicDestinations.getSchema(destination); + if (tableSchema != null) { + jsonSchemas.put(destination, BigQueryHelpers.toJsonString(tableSchema)); + } + } + TableDestination tableDestination = dynamicDestinations.getTable(destination); TableReference tableReference = tableDestination.getTableReference(); if (Strings.isNullOrEmpty(tableReference.getProjectId())) { @@ -112,6 +126,10 @@ public void processElement(ProcessContext c) throws Exception { tableReference.setTableId(jobIdPrefix); } + WriteDisposition writeDisposition = + (c.pane().getIndex() == 0) ? firstPaneWriteDisposition : WriteDisposition.WRITE_APPEND; + CreateDisposition createDisposition = + (c.pane().getIndex() == 0) ? firstPaneCreateDisposition : CreateDisposition.CREATE_NEVER; load( bqServices.getJobService(c.getPipelineOptions().as(BigQueryOptions.class)), bqServices.getDatasetService(c.getPipelineOptions().as(BigQueryOptions.class)), 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 8db4e9407ec11..3d53b7e39ed63 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 @@ -47,12 +47,14 @@ import com.google.api.services.bigquery.model.TableReference; import com.google.api.services.bigquery.model.TableRow; import com.google.api.services.bigquery.model.TableSchema; +import com.google.common.collect.ArrayListMultimap; import com.google.common.collect.HashBasedTable; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Iterables; import com.google.common.collect.Lists; import com.google.common.collect.Maps; +import com.google.common.collect.Multimap; import java.io.File; import java.io.FileFilter; import java.io.IOException; @@ -80,7 +82,6 @@ import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.Coder.Context; import org.apache.beam.sdk.coders.CoderException; -import org.apache.beam.sdk.coders.IterableCoder; import org.apache.beam.sdk.coders.KvCoder; import org.apache.beam.sdk.coders.ShardedKeyCoder; import org.apache.beam.sdk.coders.StringUtf8Coder; @@ -91,6 +92,7 @@ import org.apache.beam.sdk.io.fs.ResourceId; import org.apache.beam.sdk.io.gcp.bigquery.BigQueryHelpers.JsonSchemaToTableSchema; import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.Write.CreateDisposition; +import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.Write.Method; import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO.Write.WriteDisposition; import org.apache.beam.sdk.io.gcp.bigquery.PassThroughThenCleanup.CleanupOperation; import org.apache.beam.sdk.io.gcp.bigquery.WriteBundlesToFiles.Result; @@ -105,6 +107,9 @@ import org.apache.beam.sdk.testing.SourceTestUtils; import org.apache.beam.sdk.testing.SourceTestUtils.ExpectedSplitOutcome; import org.apache.beam.sdk.testing.TestPipeline; +import org.apache.beam.sdk.testing.TestStream; +import org.apache.beam.sdk.testing.UsesTestStream; +import org.apache.beam.sdk.testing.ValidatesRunner; import org.apache.beam.sdk.transforms.Create; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.DoFnTester; @@ -130,20 +135,19 @@ 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.PCollectionViews; import org.apache.beam.sdk.values.ShardedKey; import org.apache.beam.sdk.values.TupleTag; -import org.apache.beam.sdk.values.TypeDescriptor; import org.apache.beam.sdk.values.ValueInSingleWindow; -import org.apache.beam.sdk.values.WindowingStrategy; import org.hamcrest.CoreMatchers; import org.hamcrest.Matchers; +import org.joda.time.Duration; import org.joda.time.Instant; 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.junit.rules.ExpectedException; import org.junit.rules.TemporaryFolder; import org.junit.runner.RunWith; @@ -540,65 +544,73 @@ public String apply(String arg) { if (streaming) { users = users.setIsBoundedInternal(PCollection.IsBounded.UNBOUNDED); } - users.apply("WriteBigQuery", BigQueryIO.write() + users.apply( + "WriteBigQuery", + BigQueryIO.write() .withTestServices(fakeBqServices) .withMaxFilesPerBundle(5) .withMaxFileSize(10) .withCreateDisposition(CreateDisposition.CREATE_IF_NEEDED) - .withFormatFunction(new SerializableFunction() { - @Override - public TableRow apply(String user) { - Matcher matcher = userPattern.matcher(user); - if (matcher.matches()) { - return new TableRow().set("name", matcher.group(1)) - .set("id", Integer.valueOf(matcher.group(2))); - } - throw new RuntimeException("Unmatching element " + user); - } - }) - .to(new StringIntegerDestinations() { - @Override - public Integer getDestination(ValueInSingleWindow element) { - assertThat(element.getWindow(), Matchers.instanceOf(PartitionedGlobalWindow.class)); - Matcher matcher = userPattern.matcher(element.getValue()); - if (matcher.matches()) { - // Since we name tables by userid, we can simply store an Integer to represent - // a table. - return Integer.valueOf(matcher.group(2)); - } - throw new RuntimeException("Unmatching destination " + element.getValue()); - } + .withFormatFunction( + new SerializableFunction() { + @Override + public TableRow apply(String user) { + Matcher matcher = userPattern.matcher(user); + if (matcher.matches()) { + return new TableRow() + .set("name", matcher.group(1)) + .set("id", Integer.valueOf(matcher.group(2))); + } + throw new RuntimeException("Unmatching element " + user); + } + }) + .to( + new StringIntegerDestinations() { + @Override + public Integer getDestination(ValueInSingleWindow element) { + assertThat( + element.getWindow(), Matchers.instanceOf(PartitionedGlobalWindow.class)); + Matcher matcher = userPattern.matcher(element.getValue()); + if (matcher.matches()) { + // Since we name tables by userid, we can simply store an Integer to represent + // a table. + return Integer.valueOf(matcher.group(2)); + } + throw new RuntimeException("Unmatching destination " + element.getValue()); + } - @Override - public TableDestination getTable(Integer userId) { - verifySideInputs(); - // Each user in it's own table. - return new TableDestination("dataset-id.userid-" + userId, - "table for userid " + userId); - } + @Override + public TableDestination getTable(Integer userId) { + verifySideInputs(); + // Each user in it's own table. + return new TableDestination( + "dataset-id.userid-" + userId, "table for userid " + userId); + } - @Override - public TableSchema getSchema(Integer userId) { - verifySideInputs(); - return new TableSchema().setFields( - ImmutableList.of( - new TableFieldSchema().setName("name").setType("STRING"), - new TableFieldSchema().setName("id").setType("INTEGER"))); - } + @Override + public TableSchema getSchema(Integer userId) { + verifySideInputs(); + return new TableSchema() + .setFields( + ImmutableList.of( + new TableFieldSchema().setName("name").setType("STRING"), + new TableFieldSchema().setName("id").setType("INTEGER"))); + } - @Override - public List> getSideInputs() { - return ImmutableList.of(sideInput1, sideInput2); - } + @Override + public List> getSideInputs() { + return ImmutableList.of(sideInput1, sideInput2); + } - private void verifySideInputs() { - assertThat(sideInput(sideInput1), containsInAnyOrder("a", "b", "c")); - Map mapSideInput = sideInput(sideInput2); - assertEquals(3, mapSideInput.size()); - assertThat(mapSideInput, - allOf(hasEntry("a", "a"), hasEntry("b", "b"), hasEntry("c", "c"))); - } - }) + private void verifySideInputs() { + assertThat(sideInput(sideInput1), containsInAnyOrder("a", "b", "c")); + Map mapSideInput = sideInput(sideInput2); + assertEquals(3, mapSideInput.size()); + assertThat( + mapSideInput, + allOf(hasEntry("a", "a"), hasEntry("b", "b"), hasEntry("c", "c"))); + } + }) .withoutValidation()); p.run(); @@ -625,6 +637,59 @@ private void verifySideInputs() { } } + @Test + @Category({ValidatesRunner.class, UsesTestStream.class}) + public void testTriggeredFileLoads() throws Exception { + BigQueryOptions bqOptions = TestPipeline.testingPipelineOptions().as(BigQueryOptions.class); + bqOptions.setProject("project-id"); + bqOptions.setTempLocation(testFolder.newFolder("BigQueryIOTest").getAbsolutePath()); + + FakeDatasetService datasetService = new FakeDatasetService(); + FakeBigQueryServices fakeBqServices = + new FakeBigQueryServices() + .withJobService(new FakeJobService()) + .withDatasetService(datasetService); + + List elements = Lists.newArrayList(); + for (int i = 0; i < 30; ++i) { + elements.add(new TableRow().set("number", i)); + } + + datasetService.createDataset("project-id", "dataset-id", "", ""); + TestStream testStream = + TestStream.create(TableRowJsonCoder.of()) + .addElements( + elements.get(0), Iterables.toArray(elements.subList(1, 10), TableRow.class)) + .advanceProcessingTime(Duration.standardMinutes(1)) + .addElements( + elements.get(10), Iterables.toArray(elements.subList(11, 20), TableRow.class)) + .advanceProcessingTime(Duration.standardMinutes(1)) + .addElements( + elements.get(20), Iterables.toArray(elements.subList(21, 30), TableRow.class)) + .advanceWatermarkToInfinity(); + + Pipeline p = TestPipeline.create(bqOptions); + p.apply(testStream) + .apply( + BigQueryIO.writeTableRows() + .to("project-id:dataset-id.table-id") + .withSchema( + new TableSchema() + .setFields( + ImmutableList.of( + new TableFieldSchema().setName("number").setType("INTEGER")))) + .withTestServices(fakeBqServices) + .withTriggeringFrequency(Duration.standardSeconds(30)) + .withNumFileShards(2) + .withMethod(Method.FILE_LOADS) + .withoutValidation()); + p.run(); + + assertThat( + datasetService.getAllRows("project-id", "dataset-id", "table-id"), + containsInAnyOrder(Iterables.toArray(elements, TableRow.class))); + } + @Test public void testRetryPolicy() throws Exception { BigQueryOptions bqOptions = TestPipeline.testingPipelineOptions().as(BigQueryOptions.class); @@ -1796,25 +1861,24 @@ private void testWritePartition(long numTables, long numFilesPerTable, long file TupleTag, List>> singlePartitionTag = new TupleTag, List>>("singlePartitionTag") {}; - PCollectionView>> resultsView = - p.apply( - Create.of(files) - .withCoder(WriteBundlesToFiles.ResultCoder.of(TableDestinationCoder.of()))) - .apply(View.>asIterable()); - String tempFilePrefix = testFolder.newFolder("BigQueryIOTest").getAbsolutePath(); PCollectionView tempFilePrefixView = p.apply(Create.of(tempFilePrefix)).apply(View.asSingleton()); WritePartition writePartition = - new WritePartition<>(isSingleton, dynamicDestinations, tempFilePrefixView, - resultsView, multiPartitionsTag, singlePartitionTag); - - DoFnTester, List>> tester = - DoFnTester.of(writePartition); - tester.setSideInput(resultsView, GlobalWindow.INSTANCE, files); + new WritePartition<>( + isSingleton, + dynamicDestinations, + tempFilePrefixView, + multiPartitionsTag, + singlePartitionTag); + + DoFnTester< + Iterable>, + KV, List>> + tester = DoFnTester.of(writePartition); tester.setSideInput(tempFilePrefixView, GlobalWindow.INSTANCE, tempFilePrefix); - tester.processElement(null); + tester.processElement(files); List, List>> partitions; if (expectedNumPartitionsPerTable > 1) { @@ -1864,7 +1928,7 @@ public TableDestination getTable(String destination) { @Override public TableSchema getSchema(String destination) { - throw new UnsupportedOperationException("getSchema not expected in this test."); + return null; } } @@ -1926,16 +1990,11 @@ public void testWriteTables() throws Exception { .apply("CreateJobId", Create.of("jobId")) .apply(View.asSingleton()); - PCollectionView> schemaMapView = - p.apply("CreateEmptySchema", - Create.empty(new TypeDescriptor>() {})) - .apply(View.asMap()); WriteTables writeTables = new WriteTables<>( false, fakeBqServices, jobIdTokenView, - schemaMapView, WriteDisposition.WRITE_EMPTY, CreateDisposition.CREATE_IF_NEEDED, new IdentityDynamicTables()); @@ -1943,7 +2002,6 @@ public void testWriteTables() throws Exception { DoFnTester, List>, KV> tester = DoFnTester.of(writeTables); tester.setSideInput(jobIdTokenView, GlobalWindow.INSTANCE, jobIdToken); - tester.setSideInput(schemaMapView, GlobalWindow.INSTANCE, ImmutableMap.of()); tester.getPipelineOptions().setTempLocation("tempLocation"); for (KV, List> partition : partitions) { tester.processElement(partition); @@ -1999,21 +2057,14 @@ public void testWriteRename() throws Exception { final int numTempTablesPerFinalTable = 3; final int numRecordsPerTempTable = 10; - Map> expectedRowsPerTable = Maps.newHashMap(); + Multimap expectedRowsPerTable = ArrayListMultimap.create(); String jobIdToken = "jobIdToken"; - Map> tempTables = Maps.newHashMap(); + Multimap tempTables = ArrayListMultimap.create(); + List> tempTablesElement = Lists.newArrayList(); for (int i = 0; i < numFinalTables; ++i) { String tableName = "project-id:dataset-id.table_" + i; TableDestination tableDestination = new TableDestination( tableName, "table_" + i + "_desc"); - List tables = Lists.newArrayList(); - tempTables.put(tableDestination, tables); - - List expectedRows = expectedRowsPerTable.get(tableDestination); - if (expectedRows == null) { - expectedRows = Lists.newArrayList(); - expectedRowsPerTable.put(tableDestination, expectedRows); - } for (int j = 0; i < numTempTablesPerFinalTable; ++i) { TableReference tempTable = new TableReference() .setProjectId("project-id") @@ -2026,56 +2077,36 @@ public void testWriteRename() throws Exception { rows.add(new TableRow().set("number", j * numTempTablesPerFinalTable + k)); } datasetService.insertAll(tempTable, rows, null); - expectedRows.addAll(rows); - tables.add(BigQueryHelpers.toJsonString(tempTable)); + expectedRowsPerTable.putAll(tableDestination, rows); + String tableJson = BigQueryHelpers.toJsonString(tempTable); + tempTables.put(tableDestination, tableJson); + tempTablesElement.add(KV.of(tableDestination, tableJson)); } } - PCollection> tempTablesPCollection = - p.apply(Create.of(tempTables) - .withCoder(KvCoder.of(TableDestinationCoder.of(), - IterableCoder.of(StringUtf8Coder.of())))) - .apply(ParDo.of(new DoFn>, - KV>() { - @ProcessElement - public void processElement(ProcessContext c) { - TableDestination tableDestination = c.element().getKey(); - for (String tempTable : c.element().getValue()) { - c.output(KV.of(tableDestination, tempTable)); - } - } - })); - - PCollectionView>> tempTablesView = - PCollectionViews.multimapView( - tempTablesPCollection, - WindowingStrategy.globalDefault(), - KvCoder.of(TableDestinationCoder.of(), - StringUtf8Coder.of())); PCollectionView jobIdTokenView = p .apply("CreateJobId", Create.of("jobId")) .apply(View.asSingleton()); - WriteRename writeRename = new WriteRename( - fakeBqServices, - jobIdTokenView, - WriteDisposition.WRITE_EMPTY, - CreateDisposition.CREATE_IF_NEEDED, - tempTablesView); + WriteRename writeRename = + new WriteRename( + fakeBqServices, + jobIdTokenView, + WriteDisposition.WRITE_EMPTY, + CreateDisposition.CREATE_IF_NEEDED); - DoFnTester tester = DoFnTester.of(writeRename); - tester.setSideInput(tempTablesView, GlobalWindow.INSTANCE, tempTables); + DoFnTester>, Void> tester = DoFnTester.of(writeRename); tester.setSideInput(jobIdTokenView, GlobalWindow.INSTANCE, jobIdToken); - tester.processElement(null); + tester.processElement(tempTablesElement); - for (Map.Entry> entry : tempTables.entrySet()) { + for (Map.Entry> entry : tempTables.asMap().entrySet()) { TableDestination tableDestination = entry.getKey(); TableReference tableReference = tableDestination.getTableReference(); Table table = checkNotNull(datasetService.getTable(tableReference)); assertEquals(tableReference.getTableId() + "_desc", tableDestination.getTableDescription()); - List expectedRows = expectedRowsPerTable.get(tableDestination); + Collection expectedRows = expectedRowsPerTable.get(tableDestination); assertThat(datasetService.getAllRows(tableReference.getProjectId(), tableReference.getDatasetId(), tableReference.getTableId()), containsInAnyOrder(Iterables.toArray(expectedRows, TableRow.class))); From 9ff3ac3dcd61a310ed52d9bbc9f133a5ae5fb445 Mon Sep 17 00:00:00 2001 From: Thomas Groh Date: Tue, 15 Aug 2017 13:26:02 -0700 Subject: [PATCH 270/346] Fix Javadoc for waitUntilFinish --- .../java/org/apache/beam/sdk/PipelineResult.java | 14 +++++++------- 1 file changed, 7 insertions(+), 7 deletions(-) diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/PipelineResult.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/PipelineResult.java index b60de63678e56..4a9c30a5f7258 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/PipelineResult.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/PipelineResult.java @@ -46,14 +46,14 @@ public interface PipelineResult { State cancel() throws IOException; /** - * Waits until the pipeline finishes and returns the final status. - * It times out after the given duration. - * - * @param duration The time to wait for the pipeline to finish. - * Provide a value less than 1 ms for an infinite wait. + * Waits until the pipeline finishes and returns the final status. It times out after the given + * duration. * + * @param duration The time to wait for the pipeline to finish. Provide a value less than 1 ms for + * an infinite wait. * @return The final state of the pipeline or null on timeout. - * @throws UnsupportedOperationException if the runner does not support cancellation. + * @throws UnsupportedOperationException if the runner does not support waiting to finish with a + * timeout. */ State waitUntilFinish(Duration duration); @@ -61,7 +61,7 @@ public interface PipelineResult { * Waits until the pipeline finishes and returns the final status. * * @return The final state of the pipeline. - * @throws UnsupportedOperationException if the runner does not support cancellation. + * @throws UnsupportedOperationException if the runner does not support waiting to finish. */ State waitUntilFinish(); From ceec7ce5ba287ab40ee1f7c87129b72d4db1c1c7 Mon Sep 17 00:00:00 2001 From: JingsongLi Date: Thu, 15 Jun 2017 17:48:59 +0800 Subject: [PATCH 271/346] [BEAM-1612] Support real Bundle in Flink runner --- .../runners/flink/FlinkPipelineOptions.java | 11 + .../FlinkStreamingTransformTranslators.java | 77 +++- .../wrappers/streaming/DoFnOperator.java | 412 +++++++++++++----- .../streaming/SplittableDoFnOperator.java | 4 +- .../streaming/WindowDoFnOperator.java | 4 +- .../state/FlinkSplitStateInternals.java | 8 +- .../runners/flink/PipelineOptionsTest.java | 21 +- .../flink/streaming/DoFnOperatorTest.java | 161 +++++-- 8 files changed, 535 insertions(+), 163 deletions(-) diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkPipelineOptions.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkPipelineOptions.java index c255672626b08..2432394f4644f 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkPipelineOptions.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkPipelineOptions.java @@ -127,4 +127,15 @@ public interface FlinkPipelineOptions @Default.Boolean(false) Boolean getRetainExternalizedCheckpointsOnCancellation(); void setRetainExternalizedCheckpointsOnCancellation(Boolean retainOnCancellation); + + @Description("The maximum number of elements in a bundle.") + @Default.Long(1000) + Long getMaxBundleSize(); + void setMaxBundleSize(Long size); + + @Description("The maximum time to wait before finalising a bundle (in milliseconds).") + @Default.Long(1000) + Long getMaxBundleTimeMills(); + void setMaxBundleTimeMills(Long time); + } diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkStreamingTransformTranslators.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkStreamingTransformTranslators.java index 3d7e81f0584f9..058e1959168db 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkStreamingTransformTranslators.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkStreamingTransformTranslators.java @@ -339,7 +339,9 @@ DoFnOperator createDoFnOperator( List> additionalOutputTags, FlinkStreamingTranslationContext context, WindowingStrategy windowingStrategy, - Map, OutputTag>> tagsToLabels, + Map, OutputTag>> tagsToOutputTags, + Map, Coder>> tagsToCoders, + Map, Integer> tagsToIds, Coder> inputCoder, Coder keyCoder, Map> transformedSideInputs); @@ -360,15 +362,27 @@ static void translateParDo( WindowingStrategy windowingStrategy = input.getWindowingStrategy(); Map, OutputTag>> tagsToOutputTags = Maps.newHashMap(); + Map, Coder>> tagsToCoders = Maps.newHashMap(); + + // We associate output tags with ids, the Integer is easier to serialize than TupleTag. + // The return map of AppliedPTransform.getOutputs() is an ImmutableMap, its implementation is + // RegularImmutableMap, its entrySet order is the same with the order of insertion. + // So we can use the original AppliedPTransform.getOutputs() to produce deterministic ids. + Map, Integer> tagsToIds = Maps.newHashMap(); + int idCount = 0; + tagsToIds.put(mainOutputTag, idCount++); for (Map.Entry, PValue> entry : outputs.entrySet()) { if (!tagsToOutputTags.containsKey(entry.getKey())) { tagsToOutputTags.put( entry.getKey(), - new OutputTag>( + new OutputTag<>( entry.getKey().getId(), (TypeInformation) context.getTypeInfo((PCollection) entry.getValue()) ) ); + tagsToCoders.put(entry.getKey(), + (Coder) context.getCoder((PCollection) entry.getValue())); + tagsToIds.put(entry.getKey(), idCount++); } } @@ -409,6 +423,8 @@ static void translateParDo( context, windowingStrategy, tagsToOutputTags, + tagsToCoders, + tagsToIds, inputCoder, keyCoder, new HashMap>() /* side-input mapping */); @@ -430,6 +446,8 @@ static void translateParDo( context, windowingStrategy, tagsToOutputTags, + tagsToCoders, + tagsToIds, inputCoder, keyCoder, transformedSideInputs.f0); @@ -506,6 +524,8 @@ public DoFnOperator createDoFnOperator( FlinkStreamingTranslationContext context, WindowingStrategy windowingStrategy, Map, OutputTag>> tagsToOutputTags, + Map, Coder>> tagsToCoders, + Map, Integer> tagsToIds, Coder> inputCoder, Coder keyCoder, Map> transformedSideInputs) { @@ -515,7 +535,8 @@ public DoFnOperator createDoFnOperator( inputCoder, mainOutputTag, additionalOutputTags, - new DoFnOperator.MultiOutputOutputManagerFactory(mainOutputTag, tagsToOutputTags), + new DoFnOperator.MultiOutputOutputManagerFactory<>( + mainOutputTag, tagsToOutputTags, tagsToCoders, tagsToIds), windowingStrategy, transformedSideInputs, sideInputs, @@ -551,25 +572,28 @@ public void translateNode( @Override public DoFnOperator>, OutputT> createDoFnOperator( - DoFn>, OutputT> doFn, - String stepName, - List> sideInputs, - TupleTag mainOutputTag, - List> additionalOutputTags, - FlinkStreamingTranslationContext context, - WindowingStrategy windowingStrategy, - Map, OutputTag>> tagsToOutputTags, - Coder>>> - inputCoder, - Coder keyCoder, - Map> transformedSideInputs) { + DoFn>, OutputT> doFn, + String stepName, + List> sideInputs, + TupleTag mainOutputTag, + List> additionalOutputTags, + FlinkStreamingTranslationContext context, + WindowingStrategy windowingStrategy, + Map, OutputTag>> tagsToOutputTags, + Map, Coder>> tagsToCoders, + Map, Integer> tagsToIds, + Coder>>> + inputCoder, + Coder keyCoder, + Map> transformedSideInputs) { return new SplittableDoFnOperator<>( doFn, stepName, inputCoder, mainOutputTag, additionalOutputTags, - new DoFnOperator.MultiOutputOutputManagerFactory(mainOutputTag, tagsToOutputTags), + new DoFnOperator.MultiOutputOutputManagerFactory<>( + mainOutputTag, tagsToOutputTags, tagsToCoders, tagsToIds), windowingStrategy, transformedSideInputs, sideInputs, @@ -693,20 +717,21 @@ public void translateNode( SystemReduceFn, Iterable, BoundedWindow> reduceFn = SystemReduceFn.buffering(inputKvCoder.getValueCoder()); + Coder>>> outputCoder = + context.getCoder(context.getOutput(transform)); TypeInformation>>> outputTypeInfo = context.getTypeInfo(context.getOutput(transform)); - DoFnOperator.DefaultOutputManagerFactory>> outputManagerFactory = - new DoFnOperator.DefaultOutputManagerFactory<>(); + TupleTag>> mainTag = new TupleTag<>("main output"); WindowDoFnOperator> doFnOperator = new WindowDoFnOperator<>( reduceFn, context.getCurrentTransform().getFullName(), (Coder) windowedWorkItemCoder, - new TupleTag>>("main output"), + mainTag, Collections.>emptyList(), - outputManagerFactory, + new DoFnOperator.MultiOutputOutputManagerFactory<>(mainTag, outputCoder), windowingStrategy, new HashMap>(), /* side-input mapping */ Collections.>emptyList(), /* side inputs */ @@ -793,6 +818,8 @@ public void translateNode( AppliedCombineFn.withInputCoder( transform.getFn(), input.getPipeline().getCoderRegistry(), inputKvCoder)); + Coder>> outputCoder = + context.getCoder(context.getOutput(transform)); TypeInformation>> outputTypeInfo = context.getTypeInfo(context.getOutput(transform)); @@ -800,14 +827,15 @@ public void translateNode( if (sideInputs.isEmpty()) { + TupleTag> mainTag = new TupleTag<>("main output"); WindowDoFnOperator doFnOperator = new WindowDoFnOperator<>( reduceFn, context.getCurrentTransform().getFullName(), (Coder) windowedWorkItemCoder, - new TupleTag>("main output"), + mainTag, Collections.>emptyList(), - new DoFnOperator.DefaultOutputManagerFactory>(), + new DoFnOperator.MultiOutputOutputManagerFactory<>(mainTag, outputCoder), windowingStrategy, new HashMap>(), /* side-input mapping */ Collections.>emptyList(), /* side inputs */ @@ -826,14 +854,15 @@ public void translateNode( Tuple2>, DataStream> transformSideInputs = transformSideInputs(sideInputs, context); + TupleTag> mainTag = new TupleTag<>("main output"); WindowDoFnOperator doFnOperator = new WindowDoFnOperator<>( reduceFn, context.getCurrentTransform().getFullName(), (Coder) windowedWorkItemCoder, - new TupleTag>("main output"), + mainTag, Collections.>emptyList(), - new DoFnOperator.DefaultOutputManagerFactory>(), + new DoFnOperator.MultiOutputOutputManagerFactory<>(mainTag, outputCoder), windowingStrategy, transformSideInputs.f0, sideInputs, diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DoFnOperator.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DoFnOperator.java index 62de42313b357..0bf860a207b06 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DoFnOperator.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DoFnOperator.java @@ -21,14 +21,20 @@ import com.google.common.base.Joiner; import com.google.common.base.Optional; +import com.google.common.collect.ImmutableMap; import com.google.common.collect.Iterables; import java.io.DataInputStream; import java.io.DataOutputStream; +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; import java.io.Serializable; import java.util.ArrayList; import java.util.Collection; +import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.concurrent.ScheduledFuture; import javax.annotation.Nullable; import org.apache.beam.runners.core.DoFnRunner; import org.apache.beam.runners.core.DoFnRunners; @@ -45,6 +51,7 @@ import org.apache.beam.runners.core.StateTag; import org.apache.beam.runners.core.StateTags; import org.apache.beam.runners.core.StatefulDoFnRunner; +import org.apache.beam.runners.core.StepContext; import org.apache.beam.runners.core.TimerInternals; import org.apache.beam.runners.core.TimerInternals.TimerData; import org.apache.beam.runners.core.construction.SerializablePipelineOptions; @@ -57,6 +64,8 @@ import org.apache.beam.runners.flink.translation.wrappers.streaming.state.FlinkStateInternals; import org.apache.beam.runners.flink.translation.wrappers.streaming.state.KeyGroupCheckpointedOperator; import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.coders.StructuredCoder; +import org.apache.beam.sdk.coders.VarIntCoder; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.state.BagState; import org.apache.beam.sdk.state.TimeDomain; @@ -66,6 +75,7 @@ import org.apache.beam.sdk.transforms.reflect.DoFnInvokers; 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.PCollectionView; import org.apache.beam.sdk.values.TupleTag; import org.apache.beam.sdk.values.WindowingStrategy; @@ -87,6 +97,7 @@ import org.apache.flink.streaming.api.operators.TwoInputStreamOperator; import org.apache.flink.streaming.api.watermark.Watermark; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; +import org.apache.flink.streaming.runtime.tasks.ProcessingTimeCallback; import org.apache.flink.util.OutputTag; import org.joda.time.Instant; @@ -95,8 +106,6 @@ * * @param the input type of the {@link DoFn} * @param the output type of the {@link DoFn} - * @param the output type of the operator, this can be different from the fn output - * type when we have additional tagged outputs */ public class DoFnOperator extends AbstractStreamOperator> @@ -125,7 +134,7 @@ public class DoFnOperator protected transient SideInputReader sideInputReader; - protected transient DoFnRunners.OutputManager outputManager; + protected transient BufferedOutputManager outputManager; private transient DoFnInvoker doFnInvoker; @@ -137,7 +146,7 @@ public class DoFnOperator private transient StateTag>> pushedBackTag; - protected transient FlinkStateInternals stateInternals; + protected transient FlinkStateInternals keyedStateInternals; private final String stepName; @@ -147,14 +156,24 @@ public class DoFnOperator private final TimerInternals.TimerDataCoder timerCoder; + private final long maxBundleSize; + + private final long maxBundleTimeMills; + protected transient HeapInternalTimerService timerService; protected transient FlinkTimerInternals timerInternals; - private transient StateInternals pushbackStateInternals; + private transient StateInternals nonKeyedStateInternals; private transient Optional pushedBackWatermark; + // bundle control + private transient boolean bundleStarted = false; + private transient long elementCount; + private transient long lastFinishBundleTime; + private transient ScheduledFuture checkFinishBundleTimer; + public DoFnOperator( DoFn doFn, String stepName, @@ -184,10 +203,11 @@ public DoFnOperator( this.timerCoder = TimerInternals.TimerDataCoder.of(windowingStrategy.getWindowFn().windowCoder()); - } - private org.apache.beam.runners.core.StepContext createStepContext() { - return new StepContext(); + FlinkPipelineOptions flinkOptions = options.as(FlinkPipelineOptions.class); + + this.maxBundleSize = flinkOptions.getMaxBundleSize(); + this.maxBundleTimeMills = flinkOptions.getMaxBundleTimeMills(); } // allow overriding this in WindowDoFnOperator because this one dynamically creates @@ -204,8 +224,21 @@ public void open() throws Exception { setCurrentSideInputWatermark(BoundedWindow.TIMESTAMP_MIN_VALUE.getMillis()); setCurrentOutputWatermark(BoundedWindow.TIMESTAMP_MIN_VALUE.getMillis()); + FlinkPipelineOptions options = + serializedOptions.get().as(FlinkPipelineOptions.class); sideInputReader = NullSideInputReader.of(sideInputs); + // maybe init by initializeState + if (nonKeyedStateInternals == null) { + if (keyCoder != null) { + nonKeyedStateInternals = new FlinkKeyGroupStateInternals<>(keyCoder, + getKeyedStateBackend()); + } else { + nonKeyedStateInternals = + new FlinkSplitStateInternals<>(getOperatorStateBackend()); + } + } + if (!sideInputs.isEmpty()) { pushedBackTag = StateTags.bag("pushed-back-values", inputCoder); @@ -217,26 +250,14 @@ public void open() throws Exception { sideInputHandler = new SideInputHandler(sideInputs, sideInputStateInternals); sideInputReader = sideInputHandler; - // maybe init by initializeState - if (pushbackStateInternals == null) { - if (keyCoder != null) { - pushbackStateInternals = new FlinkKeyGroupStateInternals<>(keyCoder, - getKeyedStateBackend()); - } else { - pushbackStateInternals = - new FlinkSplitStateInternals(getOperatorStateBackend()); - } - } - pushedBackWatermark = Optional.absent(); - } - outputManager = outputManagerFactory.create(output); + outputManager = outputManagerFactory.create(output, nonKeyedStateInternals); // StatefulPardo or WindowDoFn if (keyCoder != null) { - stateInternals = new FlinkStateInternals<>((KeyedStateBackend) getKeyedStateBackend(), + keyedStateInternals = new FlinkStateInternals<>((KeyedStateBackend) getKeyedStateBackend(), keyCoder); timerService = (HeapInternalTimerService) @@ -253,10 +274,10 @@ public void open() throws Exception { doFnInvoker.invokeSetup(); - org.apache.beam.runners.core.StepContext stepContext = createStepContext(); + StepContext stepContext = new FlinkStepContext(); doFnRunner = DoFnRunners.simpleRunner( - serializedOptions.get(), + options, doFn, sideInputReader, outputManager, @@ -301,11 +322,24 @@ public void open() throws Exception { stateCleaner); } - if ((serializedOptions.get().as(FlinkPipelineOptions.class)) - .getEnableMetrics()) { + if (options.getEnableMetrics()) { doFnRunner = new DoFnRunnerWithMetricsUpdate<>(stepName, doFnRunner, getRuntimeContext()); } + elementCount = 0L; + lastFinishBundleTime = getProcessingTimeService().getCurrentProcessingTime(); + + // Schedule timer to check timeout of finish bundle. + long bundleCheckPeriod = (maxBundleTimeMills + 1) / 2; + checkFinishBundleTimer = getProcessingTimeService().scheduleAtFixedRate( + new ProcessingTimeCallback() { + @Override + public void onProcessingTime(long timestamp) throws Exception { + checkInvokeFinishBundleByTime(); + } + }, + bundleCheckPeriod, bundleCheckPeriod); + pushbackDoFnRunner = SimplePushbackSideInputDoFnRunner.create(doFnRunner, sideInputs, sideInputHandler); } @@ -315,9 +349,9 @@ public void close() throws Exception { super.close(); // sanity check: these should have been flushed out by +Inf watermarks - if (pushbackStateInternals != null) { + if (!sideInputs.isEmpty() && nonKeyedStateInternals != null) { BagState> pushedBack = - pushbackStateInternals.state(StateNamespaces.global(), pushedBackTag); + nonKeyedStateInternals.state(StateNamespaces.global(), pushedBackTag); Iterable> pushedBackContents = pushedBack.read(); if (pushedBackContents != null) { @@ -328,10 +362,11 @@ public void close() throws Exception { } } } + checkFinishBundleTimer.cancel(true); doFnInvoker.invokeTeardown(); } - protected final long getPushbackWatermarkHold() { + private long getPushbackWatermarkHold() { // if we don't have side inputs we never hold the watermark if (sideInputs.isEmpty()) { return Long.MAX_VALUE; @@ -351,7 +386,7 @@ private void checkInitPushedBackWatermark() { if (!pushedBackWatermark.isPresent()) { BagState> pushedBack = - pushbackStateInternals.state(StateNamespaces.global(), pushedBackTag); + nonKeyedStateInternals.state(StateNamespaces.global(), pushedBackTag); long min = Long.MAX_VALUE; for (WindowedValue value : pushedBack.read()) { @@ -364,9 +399,9 @@ private void checkInitPushedBackWatermark() { @Override public final void processElement( StreamRecord> streamRecord) throws Exception { - doFnRunner.startBundle(); + checkInvokeStartBundle(); doFnRunner.processElement(streamRecord.getValue()); - doFnRunner.finishBundle(); + checkInvokeFinishBundleByCount(); } private void setPushedBackWatermark(long watermark) { @@ -376,12 +411,12 @@ private void setPushedBackWatermark(long watermark) { @Override public final void processElement1( StreamRecord> streamRecord) throws Exception { - pushbackDoFnRunner.startBundle(); + checkInvokeStartBundle(); Iterable> justPushedBack = pushbackDoFnRunner.processElementInReadyWindows(streamRecord.getValue()); BagState> pushedBack = - pushbackStateInternals.state(StateNamespaces.global(), pushedBackTag); + nonKeyedStateInternals.state(StateNamespaces.global(), pushedBackTag); checkInitPushedBackWatermark(); @@ -391,13 +426,13 @@ public final void processElement1( pushedBack.add(pushedBackValue); } setPushedBackWatermark(min); - pushbackDoFnRunner.finishBundle(); + checkInvokeFinishBundleByCount(); } @Override public final void processElement2( StreamRecord streamRecord) throws Exception { - pushbackDoFnRunner.startBundle(); + checkInvokeStartBundle(); @SuppressWarnings("unchecked") WindowedValue> value = @@ -407,7 +442,7 @@ public final void processElement2( sideInputHandler.addSideInputValue(sideInput, value); BagState> pushedBack = - pushbackStateInternals.state(StateNamespaces.global(), pushedBackTag); + nonKeyedStateInternals.state(StateNamespaces.global(), pushedBackTag); List> newPushedBack = new ArrayList<>(); @@ -433,7 +468,7 @@ public final void processElement2( } setPushedBackWatermark(min); - pushbackDoFnRunner.finishBundle(); + checkInvokeFinishBundleByCount(); // maybe output a new watermark processWatermark1(new Watermark(currentInputWatermark)); @@ -446,6 +481,9 @@ public void processWatermark(Watermark mark) throws Exception { @Override public void processWatermark1(Watermark mark) throws Exception { + + checkInvokeStartBundle(); + // We do the check here because we are guaranteed to at least get the +Inf watermark on the // main input when the job finishes. if (currentSideInputWatermark >= BoundedWindow.TIMESTAMP_MAX_VALUE.getMillis()) { @@ -461,12 +499,9 @@ public void processWatermark1(Watermark mark) throws Exception { Math.min(getPushbackWatermarkHold(), currentInputWatermark); if (potentialOutputWatermark > currentOutputWatermark) { setCurrentOutputWatermark(potentialOutputWatermark); - output.emitWatermark(new Watermark(currentOutputWatermark)); + emitWatermark(currentOutputWatermark); } } else { - // fireTimers, so we need startBundle. - pushbackDoFnRunner.startBundle(); - setCurrentInputWatermark(mark.getTimestamp()); // hold back by the pushed back values waiting for side inputs @@ -474,7 +509,7 @@ public void processWatermark1(Watermark mark) throws Exception { timerService.advanceWatermark(toFlinkRuntimeWatermark(pushedBackInputWatermark)); - Instant watermarkHold = stateInternals.watermarkHold(); + Instant watermarkHold = keyedStateInternals.watermarkHold(); long combinedWatermarkHold = Math.min(watermarkHold.getMillis(), getPushbackWatermarkHold()); @@ -482,14 +517,23 @@ public void processWatermark1(Watermark mark) throws Exception { if (potentialOutputWatermark > currentOutputWatermark) { setCurrentOutputWatermark(potentialOutputWatermark); - output.emitWatermark(new Watermark(currentOutputWatermark)); + emitWatermark(currentOutputWatermark); } - pushbackDoFnRunner.finishBundle(); } } + private void emitWatermark(long watermark) { + // Must invoke finishBatch before emit the +Inf watermark otherwise there are some late events. + if (watermark >= BoundedWindow.TIMESTAMP_MAX_VALUE.getMillis()) { + invokeFinishBundle(); + } + output.emitWatermark(new Watermark(watermark)); + } + @Override public void processWatermark2(Watermark mark) throws Exception { + checkInvokeStartBundle(); + setCurrentSideInputWatermark(mark.getTimestamp()); if (mark.getTimestamp() >= BoundedWindow.TIMESTAMP_MAX_VALUE.getMillis()) { // this means we will never see any more side input @@ -498,6 +542,7 @@ public void processWatermark2(Watermark mark) throws Exception { // maybe output a new watermark processWatermark1(new Watermark(currentInputWatermark)); } + } /** @@ -516,10 +561,9 @@ private static long toFlinkRuntimeWatermark(long beamWatermark) { * any future side input, i.e. that there is no point in waiting. */ private void emitAllPushedBackData() throws Exception { - pushbackDoFnRunner.startBundle(); BagState> pushedBack = - pushbackStateInternals.state(StateNamespaces.global(), pushedBackTag); + nonKeyedStateInternals.state(StateNamespaces.global(), pushedBackTag); Iterable> pushedBackContents = pushedBack.read(); if (pushedBackContents != null) { @@ -537,11 +581,65 @@ private void emitAllPushedBackData() throws Exception { setPushedBackWatermark(Long.MAX_VALUE); - pushbackDoFnRunner.finishBundle(); + } + + /** + * Check whether invoke startBundle, if it is, need to output elements that were + * buffered as part of finishing a bundle in snapshot() first. + * + *

              In order to avoid having {@link DoFnRunner#processElement(WindowedValue)} or + * {@link DoFnRunner#onTimer(String, BoundedWindow, Instant, TimeDomain)} not between + * StartBundle and FinishBundle, this method needs to be called in each processElement + * and each processWatermark and onProcessingTime. Do not need to call in onEventTime, + * because it has been guaranteed in the processWatermark. + */ + private void checkInvokeStartBundle() { + if (!bundleStarted) { + outputManager.flushBuffer(); + pushbackDoFnRunner.startBundle(); + bundleStarted = true; + } + } + + /** + * Check whether invoke finishBundle by elements count. Called in processElement. + */ + private void checkInvokeFinishBundleByCount() { + elementCount++; + if (elementCount >= maxBundleSize) { + invokeFinishBundle(); + } + } + + /** + * Check whether invoke finishBundle by timeout. + */ + private void checkInvokeFinishBundleByTime() { + long now = getProcessingTimeService().getCurrentProcessingTime(); + if (now - lastFinishBundleTime >= maxBundleTimeMills) { + invokeFinishBundle(); + } + } + + private void invokeFinishBundle() { + if (bundleStarted) { + pushbackDoFnRunner.finishBundle(); + bundleStarted = false; + elementCount = 0L; + lastFinishBundleTime = getProcessingTimeService().getCurrentProcessingTime(); + } } @Override public void snapshotState(StateSnapshotContext context) throws Exception { + + // Forced finish a bundle in checkpoint barrier otherwise may lose data. + // Careful, it use OperatorState or KeyGroupState to store outputs, So it + // must be called before their snapshot. + outputManager.openBuffer(); + invokeFinishBundle(); + outputManager.closeBuffer(); + // copy from AbstractStreamOperator if (getKeyedStateBackend() != null) { KeyedStateCheckpointOutputStream out; @@ -587,8 +685,8 @@ public void snapshotState(StateSnapshotContext context) throws Exception { @Override public void snapshotKeyGroupState(int keyGroupIndex, DataOutputStream out) throws Exception { - if (!sideInputs.isEmpty() && keyCoder != null) { - ((FlinkKeyGroupStateInternals) pushbackStateInternals).snapshotKeyGroupState( + if (keyCoder != null) { + ((FlinkKeyGroupStateInternals) nonKeyedStateInternals).snapshotKeyGroupState( keyGroupIndex, out); } } @@ -626,23 +724,26 @@ public void initializeState(StateInitializationContext context) throws Exception @Override public void restoreKeyGroupState(int keyGroupIndex, DataInputStream in) throws Exception { - if (!sideInputs.isEmpty() && keyCoder != null) { - if (pushbackStateInternals == null) { - pushbackStateInternals = new FlinkKeyGroupStateInternals<>(keyCoder, + if (keyCoder != null) { + if (nonKeyedStateInternals == null) { + nonKeyedStateInternals = new FlinkKeyGroupStateInternals<>(keyCoder, getKeyedStateBackend()); } - ((FlinkKeyGroupStateInternals) pushbackStateInternals) + ((FlinkKeyGroupStateInternals) nonKeyedStateInternals) .restoreKeyGroupState(keyGroupIndex, in, getUserCodeClassloader()); } } @Override public void onEventTime(InternalTimer timer) throws Exception { + // We don't have to cal checkInvokeStartBundle() because it's already called in + // processWatermark*(). fireTimer(timer); } @Override public void onProcessingTime(InternalTimer timer) throws Exception { + checkInvokeStartBundle(); fireTimer(timer); } @@ -670,71 +771,186 @@ private void setCurrentOutputWatermark(long currentOutputWatermark) { } /** - * Factory for creating an {@link DoFnRunners.OutputManager} from + * Factory for creating an {@link BufferedOutputManager} from * a Flink {@link Output}. */ interface OutputManagerFactory extends Serializable { - DoFnRunners.OutputManager create(Output>> output); + BufferedOutputManager create( + Output>> output, + StateInternals stateInternals); } /** - * Default implementation of {@link OutputManagerFactory} that creates an - * {@link DoFnRunners.OutputManager} that only writes to - * a single logical output. + * A {@link DoFnRunners.OutputManager} that can buffer its outputs. + * Use {@link FlinkSplitStateInternals} or {@link FlinkKeyGroupStateInternals} + * to keep buffer data. */ - public static class DefaultOutputManagerFactory - implements OutputManagerFactory { + public static class BufferedOutputManager implements + DoFnRunners.OutputManager { + + private TupleTag mainTag; + private Map, OutputTag>> tagsToOutputTags; + private Map, Integer> tagsToIds; + private Map> idsToTags; + protected Output>> output; + + private boolean openBuffer = false; + private BagState>> bufferState; + + BufferedOutputManager( + Output>> output, + TupleTag mainTag, + Map, OutputTag>> tagsToOutputTags, + final Map, Coder>> tagsToCoders, + Map, Integer> tagsToIds, + StateInternals stateInternals) { + this.output = output; + this.mainTag = mainTag; + this.tagsToOutputTags = tagsToOutputTags; + this.tagsToIds = tagsToIds; + this.idsToTags = new HashMap<>(); + for (Map.Entry, Integer> entry : tagsToIds.entrySet()) { + idsToTags.put(entry.getValue(), entry.getKey()); + } + + ImmutableMap.Builder>> idsToCodersBuilder = + ImmutableMap.builder(); + for (Map.Entry, Integer> entry : tagsToIds.entrySet()) { + idsToCodersBuilder.put(entry.getValue(), tagsToCoders.get(entry.getKey())); + } + + StateTag>>> bufferTag = + StateTags.bag("bundle-buffer-tag", + new TaggedKvCoder(idsToCodersBuilder.build())); + bufferState = stateInternals.state(StateNamespaces.global(), bufferTag); + } + + void openBuffer() { + this.openBuffer = true; + } + + void closeBuffer() { + this.openBuffer = false; + } + @Override - public DoFnRunners.OutputManager create( - final Output>> output) { - return new DoFnRunners.OutputManager() { - @Override - public void output(TupleTag tag, WindowedValue value) { - // with tagged outputs we can't get around this because we don't - // know our own output type... - @SuppressWarnings("unchecked") - WindowedValue castValue = (WindowedValue) value; - output.collect(new StreamRecord<>(castValue)); - } - }; + public void output(TupleTag tag, WindowedValue value) { + if (!openBuffer) { + emit(tag, value); + } else { + bufferState.add(KV.>of(tagsToIds.get(tag), value)); + } + } + + /** + * Flush elements of bufferState to Flink Output. This method can't be invoke in + * {@link #snapshotState(StateSnapshotContext)} + */ + void flushBuffer() { + for (KV> taggedElem : bufferState.read()) { + emit(idsToTags.get(taggedElem.getKey()), (WindowedValue) taggedElem.getValue()); + } + bufferState.clear(); + } + + private void emit(TupleTag tag, WindowedValue value) { + if (tag.equals(mainTag)) { + // with tagged outputs we can't get around this because we don't + // know our own output type... + @SuppressWarnings("unchecked") + WindowedValue castValue = (WindowedValue) value; + output.collect(new StreamRecord<>(castValue)); + } else { + @SuppressWarnings("unchecked") + OutputTag> outputTag = (OutputTag) tagsToOutputTags.get(tag); + output.collect(outputTag, new StreamRecord<>(value)); + } + } + } + + /** + * Coder for KV of id and value. It will be serialized in Flink checkpoint. + */ + private static class TaggedKvCoder extends StructuredCoder>> { + + private Map>> idsToCoders; + + TaggedKvCoder(Map>> idsToCoders) { + this.idsToCoders = idsToCoders; + } + + @Override + public void encode(KV> kv, OutputStream out) + throws IOException { + Coder> coder = idsToCoders.get(kv.getKey()); + VarIntCoder.of().encode(kv.getKey(), out); + coder.encode(kv.getValue(), out); + } + + @Override + public KV> decode(InputStream in) + throws IOException { + Integer id = VarIntCoder.of().decode(in); + Coder> coder = idsToCoders.get(id); + WindowedValue value = coder.decode(in); + return KV.>of(id, value); + } + + @Override + public List> getCoderArguments() { + return new ArrayList<>(idsToCoders.values()); + } + + @Override + public void verifyDeterministic() throws NonDeterministicException { + for (Coder coder : idsToCoders.values()) { + verifyDeterministic(this, "Coder must be deterministic", coder); + } } } /** * Implementation of {@link OutputManagerFactory} that creates an - * {@link DoFnRunners.OutputManager} that can write to multiple logical - * outputs by unioning them in a {@link RawUnionValue}. + * {@link BufferedOutputManager} that can write to multiple logical + * outputs by Flink side output. */ public static class MultiOutputOutputManagerFactory implements OutputManagerFactory { - private TupleTag mainTag; - Map, OutputTag>> mapping; + private TupleTag mainTag; + private Map, Integer> tagsToIds; + private Map, OutputTag>> tagsToOutputTags; + private Map, Coder>> tagsToCoders; + + // There is no side output. + @SuppressWarnings("unchecked") + public MultiOutputOutputManagerFactory( + TupleTag mainTag, Coder> mainCoder) { + this(mainTag, + new HashMap, OutputTag>>(), + ImmutableMap., Coder>>builder() + .put(mainTag, (Coder) mainCoder).build(), + ImmutableMap., Integer>builder() + .put(mainTag, 0).build()); + } public MultiOutputOutputManagerFactory( - TupleTag mainTag, - Map, OutputTag>> mapping) { + TupleTag mainTag, + Map, OutputTag>> tagsToOutputTags, + Map, Coder>> tagsToCoders, + Map, Integer> tagsToIds) { this.mainTag = mainTag; - this.mapping = mapping; + this.tagsToOutputTags = tagsToOutputTags; + this.tagsToCoders = tagsToCoders; + this.tagsToIds = tagsToIds; } @Override - public DoFnRunners.OutputManager create( - final Output>> output) { - return new DoFnRunners.OutputManager() { - @Override - public void output(TupleTag tag, WindowedValue value) { - if (tag.equals(mainTag)) { - @SuppressWarnings("unchecked") - WindowedValue outputValue = (WindowedValue) value; - output.collect(new StreamRecord<>(outputValue)); - } else { - @SuppressWarnings("unchecked") - OutputTag> outputTag = (OutputTag) mapping.get(tag); - output.>collect(outputTag, new StreamRecord<>(value)); - } - } - }; + public BufferedOutputManager create( + Output>> output, + StateInternals stateInternals) { + return new BufferedOutputManager<>( + output, mainTag, tagsToOutputTags, tagsToCoders, tagsToIds, stateInternals); } } @@ -742,11 +958,11 @@ public void output(TupleTag tag, WindowedValue value) { * {@link StepContext} for running {@link DoFn DoFns} on Flink. This does not allow * accessing state or timer internals. */ - protected class StepContext implements org.apache.beam.runners.core.StepContext { + protected class FlinkStepContext implements StepContext { @Override public StateInternals stateInternals() { - return stateInternals; + return keyedStateInternals; } @Override diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/SplittableDoFnOperator.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/SplittableDoFnOperator.java index be758a6f47643..b255bb4cc509d 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/SplittableDoFnOperator.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/SplittableDoFnOperator.java @@ -97,7 +97,7 @@ public void open() throws Exception { public StateInternals stateInternalsForKey(String key) { //this will implicitly be keyed by the key of the incoming // element or by the key of a firing timer - return (StateInternals) stateInternals; + return (StateInternals) keyedStateInternals; } }; TimerInternalsFactory timerInternalsFactory = new TimerInternalsFactory() { @@ -148,7 +148,7 @@ public void outputWindowedValue( public void fireTimer(InternalTimer timer) { doFnRunner.processElement(WindowedValue.valueInGlobalWindow( KeyedWorkItems.>timersWorkItem( - (String) stateInternals.getKey(), + (String) keyedStateInternals.getKey(), Collections.singletonList(timer.getNamespace())))); } diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/WindowDoFnOperator.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/WindowDoFnOperator.java index 78d585e3e7869..b1fb398031d18 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/WindowDoFnOperator.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/WindowDoFnOperator.java @@ -86,7 +86,7 @@ protected DoFn, KV> getDoFn() { public StateInternals stateInternalsForKey(K key) { //this will implicitly be keyed by the key of the incoming // element or by the key of a firing timer - return (StateInternals) stateInternals; + return (StateInternals) keyedStateInternals; } }; TimerInternalsFactory timerInternalsFactory = new TimerInternalsFactory() { @@ -112,7 +112,7 @@ public TimerInternals timerInternalsForKey(K key) { public void fireTimer(InternalTimer timer) { doFnRunner.processElement(WindowedValue.valueInGlobalWindow( KeyedWorkItems.timersWorkItem( - (K) stateInternals.getKey(), + (K) keyedStateInternals.getKey(), Collections.singletonList(timer.getNamespace())))); } diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/FlinkSplitStateInternals.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/FlinkSplitStateInternals.java index bb2a9fff45d9a..09e59fd511cd4 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/FlinkSplitStateInternals.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/state/FlinkSplitStateInternals.java @@ -167,7 +167,7 @@ private static class FlinkSplitBagState implements BagState { @Override public void add(T input) { try { - flinkStateBackend.getOperatorState(descriptor).add(input); + flinkStateBackend.getListState(descriptor).add(input); } catch (Exception e) { throw new RuntimeException("Error updating state.", e); } @@ -181,7 +181,7 @@ public BagState readLater() { @Override public Iterable read() { try { - Iterable result = flinkStateBackend.getOperatorState(descriptor).get(); + Iterable result = flinkStateBackend.getListState(descriptor).get(); return result != null ? result : Collections.emptyList(); } catch (Exception e) { throw new RuntimeException("Error updating state.", e); @@ -194,7 +194,7 @@ public ReadableState isEmpty() { @Override public Boolean read() { try { - Iterable result = flinkStateBackend.getOperatorState(descriptor).get(); + Iterable result = flinkStateBackend.getListState(descriptor).get(); // PartitionableListState.get() return empty collection When there is no element, // KeyedListState different. (return null) return result == null || Iterators.size(result.iterator()) == 0; @@ -214,7 +214,7 @@ public ReadableState readLater() { @Override public void clear() { try { - flinkStateBackend.getOperatorState(descriptor).clear(); + flinkStateBackend.getListState(descriptor).clear(); } catch (Exception e) { throw new RuntimeException("Error reading state.", e); } diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/PipelineOptionsTest.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/PipelineOptionsTest.java index eb0602658ae52..57086df1814b7 100644 --- a/runners/flink/src/test/java/org/apache/beam/runners/flink/PipelineOptionsTest.java +++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/PipelineOptionsTest.java @@ -20,6 +20,7 @@ import java.util.Collections; import java.util.HashMap; import org.apache.beam.runners.flink.translation.wrappers.streaming.DoFnOperator; +import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.StringUtf8Coder; import org.apache.beam.sdk.options.Default; import org.apache.beam.sdk.options.Description; @@ -60,13 +61,15 @@ public interface MyOptions extends FlinkPipelineOptions { @Test(expected = Exception.class) public void parDoBaseClassPipelineOptionsNullTest() { - new DoFnOperator<>( + TupleTag mainTag = new TupleTag<>("main-output"); + Coder> coder = WindowedValue.getValueOnlyCoder(StringUtf8Coder.of()); + DoFnOperator doFnOperator = new DoFnOperator<>( new TestDoFn(), "stepName", - WindowedValue.getValueOnlyCoder(StringUtf8Coder.of()), - new TupleTag("main-output"), + coder, + mainTag, Collections.>emptyList(), - new DoFnOperator.DefaultOutputManagerFactory(), + new DoFnOperator.MultiOutputOutputManagerFactory<>(mainTag, coder), WindowingStrategy.globalDefault(), new HashMap>(), Collections.>emptyList(), @@ -81,13 +84,16 @@ public void parDoBaseClassPipelineOptionsNullTest() { @Test public void parDoBaseClassPipelineOptionsSerializationTest() throws Exception { + TupleTag mainTag = new TupleTag<>("main-output"); + + Coder> coder = WindowedValue.getValueOnlyCoder(StringUtf8Coder.of()); DoFnOperator doFnOperator = new DoFnOperator<>( new TestDoFn(), "stepName", - WindowedValue.getValueOnlyCoder(StringUtf8Coder.of()), - new TupleTag("main-output"), + coder, + mainTag, Collections.>emptyList(), - new DoFnOperator.DefaultOutputManagerFactory(), + new DoFnOperator.MultiOutputOutputManagerFactory<>(mainTag, coder), WindowingStrategy.globalDefault(), new HashMap>(), Collections.>emptyList(), @@ -105,7 +111,6 @@ public void parDoBaseClassPipelineOptionsSerializationTest() throws Exception { OneInputStreamOperatorTestHarness, WindowedValue> testHarness = new OneInputStreamOperatorTestHarness<>(deserialized, typeInformation.createSerializer(new ExecutionConfig())); - testHarness.open(); // execute once to access options diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/streaming/DoFnOperatorTest.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/streaming/DoFnOperatorTest.java index 4d2a91254b6f6..ad17de89f401d 100644 --- a/runners/flink/src/test/java/org/apache/beam/runners/flink/streaming/DoFnOperatorTest.java +++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/streaming/DoFnOperatorTest.java @@ -52,6 +52,7 @@ import org.apache.beam.sdk.transforms.join.RawUnionValue; 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.PaneInfo; import org.apache.beam.sdk.util.WindowedValue; @@ -62,6 +63,7 @@ import org.apache.flink.api.common.typeinfo.BasicTypeInfo; import org.apache.flink.api.java.functions.KeySelector; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; +import org.apache.flink.streaming.runtime.tasks.OperatorStateHandles; import org.apache.flink.streaming.util.KeyedOneInputStreamOperatorTestHarness; import org.apache.flink.streaming.util.KeyedTwoInputStreamOperatorTestHarness; import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness; @@ -107,18 +109,17 @@ public class DoFnOperatorTest { @SuppressWarnings("unchecked") public void testSingleOutput() throws Exception { - WindowedValue.ValueOnlyWindowedValueCoder windowedValueCoder = - WindowedValue.getValueOnlyCoder(StringUtf8Coder.of()); + Coder> coder = WindowedValue.getValueOnlyCoder(StringUtf8Coder.of()); TupleTag outputTag = new TupleTag<>("main-output"); DoFnOperator doFnOperator = new DoFnOperator<>( new IdentityDoFn(), "stepName", - windowedValueCoder, + coder, outputTag, Collections.>emptyList(), - new DoFnOperator.DefaultOutputManagerFactory(), + new DoFnOperator.MultiOutputOutputManagerFactory<>(outputTag, coder), WindowingStrategy.globalDefault(), new HashMap>(), /* side-input mapping */ Collections.>emptyList(), /* side inputs */ @@ -143,26 +144,38 @@ public void testSingleOutput() throws Exception { @SuppressWarnings("unchecked") public void testMultiOutputOutput() throws Exception { - WindowedValue.ValueOnlyWindowedValueCoder windowedValueCoder = + WindowedValue.ValueOnlyWindowedValueCoder coder = WindowedValue.getValueOnlyCoder(StringUtf8Coder.of()); TupleTag mainOutput = new TupleTag<>("main-output"); TupleTag additionalOutput1 = new TupleTag<>("output-1"); TupleTag additionalOutput2 = new TupleTag<>("output-2"); - ImmutableMap, OutputTag> outputMapping = + ImmutableMap, OutputTag> tagsToOutputTags = ImmutableMap., OutputTag>builder() - .put(mainOutput, new OutputTag(mainOutput.getId()){}) .put(additionalOutput1, new OutputTag(additionalOutput1.getId()){}) .put(additionalOutput2, new OutputTag(additionalOutput2.getId()){}) .build(); + ImmutableMap, Coder>> tagsToCoders = + ImmutableMap., Coder>>builder() + .put(mainOutput, (Coder) coder) + .put(additionalOutput1, coder) + .put(additionalOutput2, coder) + .build(); + ImmutableMap, Integer> tagsToIds = + ImmutableMap., Integer>builder() + .put(mainOutput, 0) + .put(additionalOutput1, 1) + .put(additionalOutput2, 2) + .build(); DoFnOperator doFnOperator = new DoFnOperator<>( new MultiOutputDoFn(additionalOutput1, additionalOutput2), "stepName", - windowedValueCoder, + coder, mainOutput, ImmutableList.>of(additionalOutput1, additionalOutput2), - new DoFnOperator.MultiOutputOutputManagerFactory(mainOutput, outputMapping), + new DoFnOperator.MultiOutputOutputManagerFactory( + mainOutput, tagsToOutputTags, tagsToCoders, tagsToIds), WindowingStrategy.globalDefault(), new HashMap>(), /* side-input mapping */ Collections.>emptyList(), /* side inputs */ @@ -184,13 +197,13 @@ public void testMultiOutputOutput() throws Exception { WindowedValue.valueInGlobalWindow("got: hello"))); assertThat( - this.stripStreamRecord(testHarness.getSideOutput(outputMapping.get(additionalOutput1))), + this.stripStreamRecord(testHarness.getSideOutput(tagsToOutputTags.get(additionalOutput1))), contains( WindowedValue.valueInGlobalWindow("extra: one"), WindowedValue.valueInGlobalWindow("got: hello"))); assertThat( - this.stripStreamRecord(testHarness.getSideOutput(outputMapping.get(additionalOutput2))), + this.stripStreamRecord(testHarness.getSideOutput(tagsToOutputTags.get(additionalOutput2))), contains( WindowedValue.valueInGlobalWindow("extra: two"), WindowedValue.valueInGlobalWindow("got: hello"))); @@ -255,7 +268,7 @@ public void onEventTime(OnTimerContext context) { inputCoder, outputTag, Collections.>emptyList(), - new DoFnOperator.DefaultOutputManagerFactory(), + new DoFnOperator.MultiOutputOutputManagerFactory<>(outputTag, outputCoder), windowingStrategy, new HashMap>(), /* side-input mapping */ Collections.>emptyList(), /* side inputs */ @@ -329,20 +342,20 @@ public void processElement(ProcessContext context) { } }; - WindowedValue.FullWindowedValueCoder windowedValueCoder = - WindowedValue.getFullCoder( - VarIntCoder.of(), - windowingStrategy.getWindowFn().windowCoder()); + Coder> inputCoder = WindowedValue.getFullCoder( + VarIntCoder.of(), windowingStrategy.getWindowFn().windowCoder()); + Coder> outputCoder = WindowedValue.getFullCoder( + StringUtf8Coder.of(), windowingStrategy.getWindowFn().windowCoder()); TupleTag outputTag = new TupleTag<>("main-output"); DoFnOperator doFnOperator = new DoFnOperator<>( fn, "stepName", - windowedValueCoder, + inputCoder, outputTag, Collections.>emptyList(), - new DoFnOperator.DefaultOutputManagerFactory(), + new DoFnOperator.MultiOutputOutputManagerFactory<>(outputTag, outputCoder), windowingStrategy, new HashMap>(), /* side-input mapping */ Collections.>emptyList(), /* side inputs */ @@ -441,7 +454,7 @@ public void onTimer(OnTimerContext context, @StateId(stateId) ValueState } }; - WindowedValue.FullWindowedValueCoder> windowedValueCoder = + WindowedValue.FullWindowedValueCoder> coder = WindowedValue.getFullCoder( KvCoder.of(StringUtf8Coder.of(), VarIntCoder.of()), windowingStrategy.getWindowFn().windowCoder()); @@ -452,10 +465,10 @@ public void onTimer(OnTimerContext context, @StateId(stateId) ValueState new DoFnOperator<>( fn, "stepName", - windowedValueCoder, + coder, outputTag, Collections.>emptyList(), - new DoFnOperator.DefaultOutputManagerFactory>(), + new DoFnOperator.MultiOutputOutputManagerFactory<>(outputTag, coder), windowingStrategy, new HashMap>(), /* side-input mapping */ Collections.>emptyList(), /* side inputs */ @@ -531,8 +544,7 @@ public String getKey( public void testSideInputs(boolean keyed) throws Exception { - WindowedValue.ValueOnlyWindowedValueCoder windowedValueCoder = - WindowedValue.getValueOnlyCoder(StringUtf8Coder.of()); + Coder> coder = WindowedValue.getValueOnlyCoder(StringUtf8Coder.of()); TupleTag outputTag = new TupleTag<>("main-output"); @@ -550,10 +562,10 @@ public void testSideInputs(boolean keyed) throws Exception { DoFnOperator doFnOperator = new DoFnOperator<>( new IdentityDoFn(), "stepName", - windowedValueCoder, + coder, outputTag, Collections.>emptyList(), - new DoFnOperator.DefaultOutputManagerFactory(), + new DoFnOperator.MultiOutputOutputManagerFactory<>(outputTag, coder), WindowingStrategy.globalDefault(), sideInputMapping, /* side-input mapping */ ImmutableList.>of(view1, view2), /* side inputs */ @@ -631,6 +643,105 @@ public void testKeyedSideInputs() throws Exception { testSideInputs(true); } + @Test + @SuppressWarnings("unchecked") + public void testBundle() throws Exception { + + WindowedValue.ValueOnlyWindowedValueCoder windowedValueCoder = + WindowedValue.getValueOnlyCoder(StringUtf8Coder.of()); + + TupleTag outputTag = new TupleTag<>("main-output"); + FlinkPipelineOptions options = PipelineOptionsFactory.as(FlinkPipelineOptions.class); + options.setMaxBundleSize(2L); + options.setMaxBundleTimeMills(10L); + + IdentityDoFn doFn = new IdentityDoFn() { + @FinishBundle + public void finishBundle(FinishBundleContext context) { + context.output( + "finishBundle", BoundedWindow.TIMESTAMP_MIN_VALUE, GlobalWindow.INSTANCE); + } + }; + + DoFnOperator.MultiOutputOutputManagerFactory outputManagerFactory = + new DoFnOperator.MultiOutputOutputManagerFactory( + outputTag, + WindowedValue.getFullCoder(StringUtf8Coder.of(), GlobalWindow.Coder.INSTANCE)); + + DoFnOperator doFnOperator = new DoFnOperator<>( + doFn, + "stepName", + windowedValueCoder, + outputTag, + Collections.>emptyList(), + outputManagerFactory, + WindowingStrategy.globalDefault(), + new HashMap>(), /* side-input mapping */ + Collections.>emptyList(), /* side inputs */ + options, + null); + + OneInputStreamOperatorTestHarness, WindowedValue> testHarness = + new OneInputStreamOperatorTestHarness<>(doFnOperator); + + testHarness.open(); + + testHarness.processElement(new StreamRecord<>(WindowedValue.valueInGlobalWindow("a"))); + testHarness.processElement(new StreamRecord<>(WindowedValue.valueInGlobalWindow("b"))); + testHarness.processElement(new StreamRecord<>(WindowedValue.valueInGlobalWindow("c"))); + + // draw a snapshot + OperatorStateHandles snapshot = testHarness.snapshot(0, 0); + + // There is a finishBundle in snapshot() + // Elements will be buffered as part of finishing a bundle in snapshot() + assertThat( + this.stripStreamRecordFromWindowedValue(testHarness.getOutput()), + contains( + WindowedValue.valueInGlobalWindow("a"), + WindowedValue.valueInGlobalWindow("b"), + WindowedValue.valueInGlobalWindow("finishBundle"), + WindowedValue.valueInGlobalWindow("c"))); + + testHarness.close(); + + DoFnOperator newDoFnOperator = new DoFnOperator<>( + doFn, + "stepName", + windowedValueCoder, + outputTag, + Collections.>emptyList(), + outputManagerFactory, + WindowingStrategy.globalDefault(), + new HashMap>(), /* side-input mapping */ + Collections.>emptyList(), /* side inputs */ + options, + null); + + OneInputStreamOperatorTestHarness, WindowedValue> newHarness = + new OneInputStreamOperatorTestHarness<>(newDoFnOperator); + + // restore snapshot + newHarness.initializeState(snapshot); + + newHarness.open(); + + // startBundle will output the buffered elements. + newHarness.processElement(new StreamRecord<>(WindowedValue.valueInGlobalWindow("d"))); + + // check finishBundle by timeout + newHarness.setProcessingTime(10); + + assertThat( + this.stripStreamRecordFromWindowedValue(newHarness.getOutput()), + contains( + WindowedValue.valueInGlobalWindow("finishBundle"), + WindowedValue.valueInGlobalWindow("d"), + WindowedValue.valueInGlobalWindow("finishBundle"))); + + newHarness.close(); + } + private Iterable> stripStreamRecordFromWindowedValue( Iterable input) { From 06bd00cc228b00d42e136dc3496db21b10909f4f Mon Sep 17 00:00:00 2001 From: Colin Phipps Date: Mon, 24 Jul 2017 21:01:09 +0000 Subject: [PATCH 272/346] Add client-side throttling. The approach used is as described in https://landing.google.com/sre/book/chapters/handling-overload.html#client-side-throttling-a7sYUg . By backing off individual workers in response to high error rates, we relieve pressure on the Datastore service, increasing the chance that the workload can complete successfully. This matches the implementation in the Java SDK. --- .../io/gcp/datastore/v1/adaptive_throttler.py | 90 ++++++++++++++++++ .../datastore/v1/adaptive_throttler_test.py | 94 +++++++++++++++++++ .../io/gcp/datastore/v1/datastoreio.py | 16 +++- .../apache_beam/io/gcp/datastore/v1/helper.py | 20 +++- 4 files changed, 212 insertions(+), 8 deletions(-) create mode 100644 sdks/python/apache_beam/io/gcp/datastore/v1/adaptive_throttler.py create mode 100644 sdks/python/apache_beam/io/gcp/datastore/v1/adaptive_throttler_test.py diff --git a/sdks/python/apache_beam/io/gcp/datastore/v1/adaptive_throttler.py b/sdks/python/apache_beam/io/gcp/datastore/v1/adaptive_throttler.py new file mode 100644 index 0000000000000..4dfd6752012ca --- /dev/null +++ b/sdks/python/apache_beam/io/gcp/datastore/v1/adaptive_throttler.py @@ -0,0 +1,90 @@ +# +# 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. +# + +# Utility functions & classes that are _not_ specific to the datastore client. +# +# For internal use only; no backwards-compatibility guarantees. + +import random + +from apache_beam.io.gcp.datastore.v1 import util + + +class AdaptiveThrottler(object): + """Implements adaptive throttling. + + See + https://landing.google.com/sre/book/chapters/handling-overload.html#client-side-throttling-a7sYUg + for a full discussion of the use case and algorithm applied. + """ + + # The target minimum number of requests per samplePeriodMs, even if no + # requests succeed. Must be greater than 0, else we could throttle to zero. + # Because every decision is probabilistic, there is no guarantee that the + # request rate in any given interval will not be zero. (This is the +1 from + # the formula in + # https://landing.google.com/sre/book/chapters/handling-overload.html ) + MIN_REQUESTS = 1 + + def __init__(self, window_ms, bucket_ms, overload_ratio): + """Args: + window_ms: int, length of history to consider, in ms, to set throttling. + bucket_ms: int, granularity of time buckets that we store data in, in ms. + overload_ratio: float, the target ratio between requests sent and + successful requests. This is "K" in the formula in + https://landing.google.com/sre/book/chapters/handling-overload.html + """ + self._all_requests = util.MovingSum(window_ms, bucket_ms) + self._successful_requests = util.MovingSum(window_ms, bucket_ms) + self._overload_ratio = float(overload_ratio) + self._random = random.Random() + + def _throttling_probability(self, now): + if not self._all_requests.has_data(now): + return 0 + all_requests = self._all_requests.sum(now) + successful_requests = self._successful_requests.sum(now) + return max( + 0, (all_requests - self._overload_ratio * successful_requests) + / (all_requests + AdaptiveThrottler.MIN_REQUESTS)) + + def throttle_request(self, now): + """Determines whether one RPC attempt should be throttled. + + This should be called once each time the caller intends to send an RPC; if + it returns true, drop or delay that request (calling this function again + after the delay). + + Args: + now: int, time in ms since the epoch + Returns: + bool, True if the caller should throttle or delay the request. + """ + throttling_probability = self._throttling_probability(now) + self._all_requests.add(now, 1) + return self._random.uniform(0, 1) < throttling_probability + + def successful_request(self, now): + """Notifies the throttler of a successful request. + + Must be called once for each request (for which throttle_request was + previously called) that succeeded. + + Args: + now: int, time in ms since the epoch + """ + self._successful_requests.add(now, 1) diff --git a/sdks/python/apache_beam/io/gcp/datastore/v1/adaptive_throttler_test.py b/sdks/python/apache_beam/io/gcp/datastore/v1/adaptive_throttler_test.py new file mode 100644 index 0000000000000..93b91ad7cfab9 --- /dev/null +++ b/sdks/python/apache_beam/io/gcp/datastore/v1/adaptive_throttler_test.py @@ -0,0 +1,94 @@ +# +# 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 unittest +from mock import patch + +from apache_beam.io.gcp.datastore.v1.adaptive_throttler import AdaptiveThrottler + + +class AdaptiveThrottlerTest(unittest.TestCase): + + START_TIME = 1500000000000 + SAMPLE_PERIOD = 60000 + BUCKET = 1000 + OVERLOAD_RATIO = 2 + + def setUp(self): + self._throttler = AdaptiveThrottler( + AdaptiveThrottlerTest.SAMPLE_PERIOD, AdaptiveThrottlerTest.BUCKET, + AdaptiveThrottlerTest.OVERLOAD_RATIO) + + # As far as practical, keep these tests aligned with + # AdaptiveThrottlerTest.java. + + def test_no_initial_throttling(self): + self.assertEqual(0, self._throttler._throttling_probability( + AdaptiveThrottlerTest.START_TIME)) + + def test_no_throttling_if_no_errors(self): + for t in range(AdaptiveThrottlerTest.START_TIME, + AdaptiveThrottlerTest.START_TIME + 20): + self.assertFalse(self._throttler.throttle_request(t)) + self._throttler.successful_request(t) + self.assertEqual(0, self._throttler._throttling_probability( + AdaptiveThrottlerTest.START_TIME + 20)) + + def test_no_throttling_after_errors_expire(self): + for t in range(AdaptiveThrottlerTest.START_TIME, + AdaptiveThrottlerTest.START_TIME + + AdaptiveThrottlerTest.SAMPLE_PERIOD, 100): + self._throttler.throttle_request(t) + # And no sucessful_request + self.assertLess(0, self._throttler._throttling_probability( + AdaptiveThrottlerTest.START_TIME + AdaptiveThrottlerTest.SAMPLE_PERIOD + )) + for t in range(AdaptiveThrottlerTest.START_TIME + + AdaptiveThrottlerTest.SAMPLE_PERIOD, + AdaptiveThrottlerTest.START_TIME + + AdaptiveThrottlerTest.SAMPLE_PERIOD*2, 100): + self._throttler.throttle_request(t) + self._throttler.successful_request(t) + + self.assertEqual(0, self._throttler._throttling_probability( + AdaptiveThrottlerTest.START_TIME + + AdaptiveThrottlerTest.SAMPLE_PERIOD*2)) + + @patch('random.Random') + def test_throttling_after_errors(self, mock_random): + mock_random().uniform.side_effect = [x/10.0 for x in range(0, 10)]*2 + self._throttler = AdaptiveThrottler( + AdaptiveThrottlerTest.SAMPLE_PERIOD, AdaptiveThrottlerTest.BUCKET, + AdaptiveThrottlerTest.OVERLOAD_RATIO) + for t in range(AdaptiveThrottlerTest.START_TIME, + AdaptiveThrottlerTest.START_TIME + 20): + throttled = self._throttler.throttle_request(t) + # 1/3rd of requests succeeding. + if t % 3 == 1: + self._throttler.successful_request(t) + + if t > AdaptiveThrottlerTest.START_TIME + 10: + # Roughly 1/3rd succeeding, 1/3rd failing, 1/3rd throttled. + self.assertAlmostEqual( + 0.33, self._throttler._throttling_probability(t), delta=0.1) + # Given the mocked random numbers, we expect 10..13 to be throttled and + # 14+ to be unthrottled. + self.assertEqual(t < AdaptiveThrottlerTest.START_TIME + 14, throttled) + + +if __name__ == '__main__': + unittest.main() diff --git a/sdks/python/apache_beam/io/gcp/datastore/v1/datastoreio.py b/sdks/python/apache_beam/io/gcp/datastore/v1/datastoreio.py index 02588141e9c1d..3cfba93d2e8cc 100644 --- a/sdks/python/apache_beam/io/gcp/datastore/v1/datastoreio.py +++ b/sdks/python/apache_beam/io/gcp/datastore/v1/datastoreio.py @@ -32,6 +32,7 @@ from apache_beam.io.gcp.datastore.v1 import helper from apache_beam.io.gcp.datastore.v1 import query_splitter from apache_beam.io.gcp.datastore.v1 import util +from apache_beam.io.gcp.datastore.v1.adaptive_throttler import AdaptiveThrottler from apache_beam.transforms import Create from apache_beam.transforms import DoFn from apache_beam.transforms import FlatMap @@ -402,10 +403,15 @@ def __init__(self, project, fixed_batch_size=None): _Mutate.DatastoreWriteFn, "datastoreRpcSuccesses") self._rpc_errors = Metrics.counter( _Mutate.DatastoreWriteFn, "datastoreRpcErrors") + self._throttled_secs = Metrics.counter( + _Mutate.DatastoreWriteFn, "cumulativeThrottlingSeconds") + self._throttler = AdaptiveThrottler(window_ms=120000, bucket_ms=1000, + overload_ratio=1.25) - def _update_rpc_stats(self, successes=0, errors=0): + def _update_rpc_stats(self, successes=0, errors=0, throttled_secs=0): self._rpc_successes.inc(successes) self._rpc_errors.inc(errors) + self._throttled_secs.inc(throttled_secs) def start_bundle(self): self._mutations = [] @@ -415,7 +421,8 @@ def start_bundle(self): self._target_batch_size = self._fixed_batch_size else: self._batch_sizer = _Mutate._DynamicBatchSizer() - self._target_batch_size = self._batch_sizer.get_batch_size(time.time()) + self._target_batch_size = self._batch_sizer.get_batch_size( + time.time()*1000) def process(self, element): size = element.ByteSize() @@ -435,12 +442,13 @@ def _flush_batch(self): # Flush the current batch of mutations to Cloud Datastore. _, latency_ms = helper.write_mutations( self._datastore, self._project, self._mutations, - self._update_rpc_stats) + self._throttler, self._update_rpc_stats, + throttle_delay=_Mutate._WRITE_BATCH_TARGET_LATENCY_MS/1000) logging.debug("Successfully wrote %d mutations in %dms.", len(self._mutations), latency_ms) if not self._fixed_batch_size: - now = time.time() + now = time.time()*1000 self._batch_sizer.report_latency(now, latency_ms, len(self._mutations)) self._target_batch_size = self._batch_sizer.get_batch_size(now) diff --git a/sdks/python/apache_beam/io/gcp/datastore/v1/helper.py b/sdks/python/apache_beam/io/gcp/datastore/v1/helper.py index da14cc44236cc..5cde25540da4e 100644 --- a/sdks/python/apache_beam/io/gcp/datastore/v1/helper.py +++ b/sdks/python/apache_beam/io/gcp/datastore/v1/helper.py @@ -22,6 +22,7 @@ import errno from socket import error as SocketError +import logging import sys import time @@ -167,7 +168,8 @@ def is_key_valid(key): return key.path[-1].HasField('id') or key.path[-1].HasField('name') -def write_mutations(datastore, project, mutations, rpc_stats_callback=None): +def write_mutations(datastore, project, mutations, throttler, + rpc_stats_callback=None, throttle_delay=1): """A helper function to write a batch of mutations to Cloud Datastore. If a commit fails, it will be retried upto 5 times. All mutations in the @@ -180,8 +182,10 @@ def write_mutations(datastore, project, mutations, rpc_stats_callback=None): project: str, project id mutations: list of google.cloud.proto.datastore.v1.datastore_pb2.Mutation rpc_stats_callback: a function to call with arguments `successes` and - `failures`; this is called to record successful and failed RPCs to - Datastore. + `failures` and `throttled_secs`; this is called to record successful + and failed RPCs to Datastore and time spent waiting for throttling. + throttler: AdaptiveThrottler, to use to select requests to be throttled. + throttle_delay: float, time in seconds to sleep when throttled. Returns a tuple of: CommitResponse, the response from Datastore; @@ -196,12 +200,20 @@ def write_mutations(datastore, project, mutations, rpc_stats_callback=None): @retry.with_exponential_backoff(num_retries=5, retry_filter=retry_on_rpc_error) def commit(request): + # Client-side throttling. + while throttler.throttle_request(time.time()*1000): + logging.info("Delaying request for %ds due to previous failures", + throttle_delay) + time.sleep(throttle_delay) + rpc_stats_callback(throttled_secs=throttle_delay) + try: start_time = time.time() response = datastore.commit(request) end_time = time.time() - rpc_stats_callback(successes=1) + rpc_stats_callback(successes=1) + throttler.successful_request(start_time*1000) commit_time_ms = int((end_time-start_time)*1000) return response, commit_time_ms except (RPCError, SocketError): From cfb798830042b28eaf343103724779c90092535c Mon Sep 17 00:00:00 2001 From: Luke Cwik Date: Fri, 7 Jul 2017 14:02:58 -0700 Subject: [PATCH 273/346] [BEAM-1347] Plumb through a yet to be created state client through PTransformRunnerFactory --- .../beam/fn/harness/BeamFnDataReadRunner.java | 2 ++ .../fn/harness/BeamFnDataWriteRunner.java | 2 ++ .../beam/fn/harness/BoundedSourceRunner.java | 2 ++ .../beam/fn/harness/FnApiDoFnRunner.java | 8 ++++++ .../org/apache/beam/fn/harness/FnHarness.java | 7 ++++-- .../fn/harness/PTransformRunnerFactory.java | 5 +++- .../harness/control/ProcessBundleHandler.java | 11 ++++++-- .../fn/harness/state/BeamFnStateClient.java | 25 +++++++++++++++++++ .../beam/fn/harness/state/package-info.java | 22 ++++++++++++++++ .../fn/harness/BeamFnDataReadRunnerTest.java | 1 + .../fn/harness/BeamFnDataWriteRunnerTest.java | 1 + .../fn/harness/BoundedSourceRunnerTest.java | 1 + .../beam/fn/harness/FnApiDoFnRunnerTest.java | 1 + .../control/ProcessBundleHandlerTest.java | 9 +++++++ 14 files changed, 92 insertions(+), 5 deletions(-) create mode 100644 sdks/java/harness/src/main/java/org/apache/beam/fn/harness/state/BeamFnStateClient.java create mode 100644 sdks/java/harness/src/main/java/org/apache/beam/fn/harness/state/package-info.java diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/BeamFnDataReadRunner.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/BeamFnDataReadRunner.java index df0e5a29604da..f254ec4650c29 100644 --- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/BeamFnDataReadRunner.java +++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/BeamFnDataReadRunner.java @@ -33,6 +33,7 @@ import org.apache.beam.fn.harness.data.BeamFnDataClient; import org.apache.beam.fn.harness.fn.ThrowingConsumer; import org.apache.beam.fn.harness.fn.ThrowingRunnable; +import org.apache.beam.fn.harness.state.BeamFnStateClient; import org.apache.beam.fn.v1.BeamFnApi; import org.apache.beam.runners.core.construction.CoderTranslation; import org.apache.beam.runners.core.construction.RehydratedComponents; @@ -77,6 +78,7 @@ static class Factory public BeamFnDataReadRunner createRunnerForPTransform( PipelineOptions pipelineOptions, BeamFnDataClient beamFnDataClient, + BeamFnStateClient beamFnStateClient, String pTransformId, RunnerApi.PTransform pTransform, Supplier processBundleInstructionId, diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/BeamFnDataWriteRunner.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/BeamFnDataWriteRunner.java index 48b450aded292..179a2287912e9 100644 --- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/BeamFnDataWriteRunner.java +++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/BeamFnDataWriteRunner.java @@ -32,6 +32,7 @@ import org.apache.beam.fn.harness.fn.CloseableThrowingConsumer; import org.apache.beam.fn.harness.fn.ThrowingConsumer; import org.apache.beam.fn.harness.fn.ThrowingRunnable; +import org.apache.beam.fn.harness.state.BeamFnStateClient; import org.apache.beam.fn.v1.BeamFnApi; import org.apache.beam.runners.core.construction.CoderTranslation; import org.apache.beam.runners.core.construction.RehydratedComponents; @@ -72,6 +73,7 @@ static class Factory public BeamFnDataWriteRunner createRunnerForPTransform( PipelineOptions pipelineOptions, BeamFnDataClient beamFnDataClient, + BeamFnStateClient beamFnStateClient, String pTransformId, RunnerApi.PTransform pTransform, Supplier processBundleInstructionId, diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/BoundedSourceRunner.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/BoundedSourceRunner.java index 5f6509f1b51ec..c4daa0f636cf8 100644 --- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/BoundedSourceRunner.java +++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/BoundedSourceRunner.java @@ -31,6 +31,7 @@ import org.apache.beam.fn.harness.data.BeamFnDataClient; import org.apache.beam.fn.harness.fn.ThrowingConsumer; import org.apache.beam.fn.harness.fn.ThrowingRunnable; +import org.apache.beam.fn.harness.state.BeamFnStateClient; import org.apache.beam.sdk.common.runner.v1.RunnerApi; import org.apache.beam.sdk.io.BoundedSource; import org.apache.beam.sdk.io.Source.Reader; @@ -64,6 +65,7 @@ static class Factory, OutputT> public BoundedSourceRunner createRunnerForPTransform( PipelineOptions pipelineOptions, BeamFnDataClient beamFnDataClient, + BeamFnStateClient beamFnStateClient, String pTransformId, RunnerApi.PTransform pTransform, Supplier processBundleInstructionId, diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/FnApiDoFnRunner.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/FnApiDoFnRunner.java index 86168f97fe81b..d325bb29d318c 100644 --- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/FnApiDoFnRunner.java +++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/FnApiDoFnRunner.java @@ -35,6 +35,7 @@ import org.apache.beam.fn.harness.data.BeamFnDataClient; import org.apache.beam.fn.harness.fn.ThrowingConsumer; import org.apache.beam.fn.harness.fn.ThrowingRunnable; +import org.apache.beam.fn.harness.state.BeamFnStateClient; import org.apache.beam.runners.core.DoFnRunner; import org.apache.beam.runners.core.construction.ParDoTranslation; import org.apache.beam.runners.dataflow.util.DoFnInfo; @@ -48,6 +49,8 @@ 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; import org.apache.beam.sdk.transforms.windowing.PaneInfo; @@ -86,6 +89,7 @@ static class Factory public DoFnRunner createRunnerForPTransform( PipelineOptions pipelineOptions, BeamFnDataClient beamFnDataClient, + BeamFnStateClient beamFnStateClient, String pTransformId, RunnerApi.PTransform pTransform, Supplier processBundleInstructionId, @@ -165,6 +169,8 @@ public DoFnRunner createRunnerForPTransform( private final StartBundleContext startBundleContext; private final ProcessBundleContext processBundleContext; private final FinishBundleContext finishBundleContext; + private final WindowingStrategy windowingStrategy; + private final DoFnSignature doFnSignature; /** * The lifetime of this member is only valid during {@link #processElement(WindowedValue)}. @@ -186,6 +192,8 @@ public DoFnRunner createRunnerForPTransform( this.doFn = doFn; this.mainOutputConsumers = mainOutputConsumers; this.outputMap = outputMap; + this.windowingStrategy = windowingStrategy; + this.doFnSignature = DoFnSignatures.signatureForDoFn(doFn); this.doFnInvoker = DoFnInvokers.invokerFor(doFn); this.startBundleContext = new StartBundleContext(); this.processBundleContext = new ProcessBundleContext(); diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/FnHarness.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/FnHarness.java index 05ab44f7ff5a2..a79ecca858ff2 100644 --- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/FnHarness.java +++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/FnHarness.java @@ -109,8 +109,11 @@ public static void main(PipelineOptions options, BeamFnDataGrpcClient beamFnDataMultiplexer = new BeamFnDataGrpcClient( options, channelFactory::forDescriptor, streamObserverFactory::from); - ProcessBundleHandler processBundleHandler = - new ProcessBundleHandler(options, fnApiRegistry::getById, beamFnDataMultiplexer); + ProcessBundleHandler processBundleHandler = new ProcessBundleHandler( + options, + fnApiRegistry::getById, + beamFnDataMultiplexer, + null /* beamFnStateClient */); handlers.put(BeamFnApi.InstructionRequest.RequestCase.REGISTER, fnApiRegistry::register); handlers.put(BeamFnApi.InstructionRequest.RequestCase.PROCESS_BUNDLE, diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/PTransformRunnerFactory.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/PTransformRunnerFactory.java index 7cf0610049be6..4ef56d822d74b 100644 --- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/PTransformRunnerFactory.java +++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/PTransformRunnerFactory.java @@ -25,6 +25,7 @@ import org.apache.beam.fn.harness.data.BeamFnDataClient; import org.apache.beam.fn.harness.fn.ThrowingConsumer; import org.apache.beam.fn.harness.fn.ThrowingRunnable; +import org.apache.beam.fn.harness.state.BeamFnStateClient; import org.apache.beam.sdk.common.runner.v1.RunnerApi; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.util.WindowedValue; @@ -40,7 +41,8 @@ public interface PTransformRunnerFactory { * element processing, or during execution of start/finish. * * @param pipelineOptions Pipeline options - * @param beamFnDataClient + * @param beamFnDataClient A client for handling inbound and outbound data streams. + * @param beamFnStateClient A client for handling state requests. * @param pTransformId The id of the PTransform. * @param pTransform The PTransform definition. * @param processBundleInstructionId A supplier containing the active process bundle instruction @@ -58,6 +60,7 @@ public interface PTransformRunnerFactory { T createRunnerForPTransform( PipelineOptions pipelineOptions, BeamFnDataClient beamFnDataClient, + BeamFnStateClient beamFnStateClient, String pTransformId, RunnerApi.PTransform pTransform, Supplier processBundleInstructionId, diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/control/ProcessBundleHandler.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/control/ProcessBundleHandler.java index 1e735707ddb14..67c4d6778d8dd 100644 --- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/control/ProcessBundleHandler.java +++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/control/ProcessBundleHandler.java @@ -39,6 +39,7 @@ import org.apache.beam.fn.harness.data.BeamFnDataClient; import org.apache.beam.fn.harness.fn.ThrowingConsumer; import org.apache.beam.fn.harness.fn.ThrowingRunnable; +import org.apache.beam.fn.harness.state.BeamFnStateClient; import org.apache.beam.fn.v1.BeamFnApi; import org.apache.beam.sdk.common.runner.v1.RunnerApi; import org.apache.beam.sdk.options.PipelineOptions; @@ -83,6 +84,7 @@ public class ProcessBundleHandler { private final PipelineOptions options; private final Function fnApiRegistry; private final BeamFnDataClient beamFnDataClient; + private final BeamFnStateClient beamFnStateClient; private final Map urnToPTransformRunnerFactoryMap; private final PTransformRunnerFactory defaultPTransformRunnerFactory; @@ -90,8 +92,9 @@ public class ProcessBundleHandler { public ProcessBundleHandler( PipelineOptions options, Function fnApiRegistry, - BeamFnDataClient beamFnDataClient) { - this(options, fnApiRegistry, beamFnDataClient, REGISTERED_RUNNER_FACTORIES); + BeamFnDataClient beamFnDataClient, + BeamFnStateClient beamFnStateClient) { + this(options, fnApiRegistry, beamFnDataClient, beamFnStateClient, REGISTERED_RUNNER_FACTORIES); } @VisibleForTesting @@ -99,16 +102,19 @@ public ProcessBundleHandler( PipelineOptions options, Function fnApiRegistry, BeamFnDataClient beamFnDataClient, + BeamFnStateClient beamFnStateClient, Map urnToPTransformRunnerFactoryMap) { this.options = options; this.fnApiRegistry = fnApiRegistry; this.beamFnDataClient = beamFnDataClient; + this.beamFnStateClient = beamFnStateClient; this.urnToPTransformRunnerFactoryMap = urnToPTransformRunnerFactoryMap; this.defaultPTransformRunnerFactory = new PTransformRunnerFactory() { @Override public Object createRunnerForPTransform( PipelineOptions pipelineOptions, BeamFnDataClient beamFnDataClient, + BeamFnStateClient beanFnStateClient, String pTransformId, RunnerApi.PTransform pTransform, Supplier processBundleInstructionId, @@ -162,6 +168,7 @@ private void createRunnerAndConsumersForPTransformRecursively( .createRunnerForPTransform( options, beamFnDataClient, + beamFnStateClient, pTransformId, pTransform, processBundleInstructionId, diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/state/BeamFnStateClient.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/state/BeamFnStateClient.java new file mode 100644 index 0000000000000..81505308f6da7 --- /dev/null +++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/state/BeamFnStateClient.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.fn.harness.state; + +/** + * TODO: Define interface required for handling state calls. + */ +public interface BeamFnStateClient { + +} diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/state/package-info.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/state/package-info.java new file mode 100644 index 0000000000000..feadb7da7eb89 --- /dev/null +++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/state/package-info.java @@ -0,0 +1,22 @@ +/* + * 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. + */ + +/** + * State client and state caching. + */ +package org.apache.beam.fn.harness.state; diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/BeamFnDataReadRunnerTest.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/BeamFnDataReadRunnerTest.java index 92e60885a87ff..e5b4968b3f613 100644 --- a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/BeamFnDataReadRunnerTest.java +++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/BeamFnDataReadRunnerTest.java @@ -141,6 +141,7 @@ public void testCreatingAndProcessingBeamFnDataReadRunner() throws Exception { new BeamFnDataReadRunner.Factory().createRunnerForPTransform( PipelineOptionsFactory.create(), mockBeamFnDataClient, + null /* beamFnStateClient */, "pTransformId", pTransform, Suppliers.ofInstance(bundleId)::get, diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/BeamFnDataWriteRunnerTest.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/BeamFnDataWriteRunnerTest.java index ffa3a2d9876df..c4b717a5d113f 100644 --- a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/BeamFnDataWriteRunnerTest.java +++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/BeamFnDataWriteRunnerTest.java @@ -127,6 +127,7 @@ public void testCreatingAndProcessingBeamFnDataWriteRunner() throws Exception { new BeamFnDataWriteRunner.Factory().createRunnerForPTransform( PipelineOptionsFactory.create(), mockBeamFnDataClient, + null /* beamFnStateClient */, "ptransformId", pTransform, Suppliers.ofInstance(bundleId)::get, diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/BoundedSourceRunnerTest.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/BoundedSourceRunnerTest.java index b9f22e854e764..135495a5354d2 100644 --- a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/BoundedSourceRunnerTest.java +++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/BoundedSourceRunnerTest.java @@ -140,6 +140,7 @@ public void testCreatingAndProcessingSourceFromFactory() throws Exception { new BoundedSourceRunner.Factory<>().createRunnerForPTransform( PipelineOptionsFactory.create(), null /* beamFnDataClient */, + null /* beamFnStateClient */, "pTransformId", pTransform, Suppliers.ofInstance("57L")::get, diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/FnApiDoFnRunnerTest.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/FnApiDoFnRunnerTest.java index efa8fcf195b18..ebec608f7fb30 100644 --- a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/FnApiDoFnRunnerTest.java +++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/FnApiDoFnRunnerTest.java @@ -126,6 +126,7 @@ public void testCreatingAndProcessingDoFn() throws Exception { new FnApiDoFnRunner.Factory<>().createRunnerForPTransform( PipelineOptionsFactory.create(), null /* beamFnDataClient */, + null /* beamFnStateClient */, pTransformId, pTransform, Suppliers.ofInstance("57L")::get, diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/control/ProcessBundleHandlerTest.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/control/ProcessBundleHandlerTest.java index 0a94b5b1b206a..d0e1faf6f2461 100644 --- a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/control/ProcessBundleHandlerTest.java +++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/control/ProcessBundleHandlerTest.java @@ -35,6 +35,7 @@ import org.apache.beam.fn.harness.data.BeamFnDataClient; import org.apache.beam.fn.harness.fn.ThrowingConsumer; import org.apache.beam.fn.harness.fn.ThrowingRunnable; +import org.apache.beam.fn.harness.state.BeamFnStateClient; import org.apache.beam.fn.v1.BeamFnApi; import org.apache.beam.sdk.common.runner.v1.RunnerApi; import org.apache.beam.sdk.options.PipelineOptions; @@ -91,6 +92,7 @@ public void testOrderOfStartAndFinishCalls() throws Exception { public Object createRunnerForPTransform( PipelineOptions pipelineOptions, BeamFnDataClient beamFnDataClient, + BeamFnStateClient beamFnStateClient, String pTransformId, RunnerApi.PTransform pTransform, Supplier processBundleInstructionId, @@ -115,6 +117,7 @@ public Object createRunnerForPTransform( PipelineOptionsFactory.create(), fnApiRegistry::get, beamFnDataClient, + null /* beamFnStateClient */, ImmutableMap.of( DATA_INPUT_URN, startFinishRecorder, DATA_OUTPUT_URN, startFinishRecorder)); @@ -147,11 +150,13 @@ public void testCreatingPTransformExceptionsArePropagated() throws Exception { PipelineOptionsFactory.create(), fnApiRegistry::get, beamFnDataClient, + null /* beamFnStateClient */, ImmutableMap.of(DATA_INPUT_URN, new PTransformRunnerFactory() { @Override public Object createRunnerForPTransform( PipelineOptions pipelineOptions, BeamFnDataClient beamFnDataClient, + BeamFnStateClient beamFnStateClient, String pTransformId, RunnerApi.PTransform pTransform, Supplier processBundleInstructionId, @@ -185,11 +190,13 @@ public void testPTransformStartExceptionsArePropagated() throws Exception { PipelineOptionsFactory.create(), fnApiRegistry::get, beamFnDataClient, + null /* beamFnStateClient */, ImmutableMap.of(DATA_INPUT_URN, new PTransformRunnerFactory() { @Override public Object createRunnerForPTransform( PipelineOptions pipelineOptions, BeamFnDataClient beamFnDataClient, + BeamFnStateClient beamFnStateClient, String pTransformId, RunnerApi.PTransform pTransform, Supplier processBundleInstructionId, @@ -224,11 +231,13 @@ public void testPTransformFinishExceptionsArePropagated() throws Exception { PipelineOptionsFactory.create(), fnApiRegistry::get, beamFnDataClient, + null /* beamFnStateClient */, ImmutableMap.of(DATA_INPUT_URN, new PTransformRunnerFactory() { @Override public Object createRunnerForPTransform( PipelineOptions pipelineOptions, BeamFnDataClient beamFnDataClient, + BeamFnStateClient beamFnStateClient, String pTransformId, RunnerApi.PTransform pTransform, Supplier processBundleInstructionId, From 8dc6e1666f3f113fe5ee854f4c7060e0fbd614e1 Mon Sep 17 00:00:00 2001 From: "chamikara@google.com" Date: Thu, 17 Aug 2017 18:21:44 -0700 Subject: [PATCH 274/346] Fixes a pydocs validation failure due to a recent commit. --- .../io/gcp/datastore/v1/adaptive_throttler.py | 16 ++++++++++------ 1 file changed, 10 insertions(+), 6 deletions(-) diff --git a/sdks/python/apache_beam/io/gcp/datastore/v1/adaptive_throttler.py b/sdks/python/apache_beam/io/gcp/datastore/v1/adaptive_throttler.py index 4dfd6752012ca..7d94f24ca859d 100644 --- a/sdks/python/apache_beam/io/gcp/datastore/v1/adaptive_throttler.py +++ b/sdks/python/apache_beam/io/gcp/datastore/v1/adaptive_throttler.py @@ -41,12 +41,16 @@ class AdaptiveThrottler(object): MIN_REQUESTS = 1 def __init__(self, window_ms, bucket_ms, overload_ratio): - """Args: - window_ms: int, length of history to consider, in ms, to set throttling. - bucket_ms: int, granularity of time buckets that we store data in, in ms. - overload_ratio: float, the target ratio between requests sent and - successful requests. This is "K" in the formula in - https://landing.google.com/sre/book/chapters/handling-overload.html + """Initializes AdaptiveThrottler. + + Args: + window_ms: int, length of history to consider, in ms, to set + throttling. + bucket_ms: int, granularity of time buckets that we store data in, in + ms. + overload_ratio: float, the target ratio between requests sent and + successful requests. This is "K" in the formula in + https://landing.google.com/sre/book/chapters/handling-overload.html. """ self._all_requests = util.MovingSum(window_ms, bucket_ms) self._successful_requests = util.MovingSum(window_ms, bucket_ms) From 518c158f82249091a54dca17ae348734f5abe633 Mon Sep 17 00:00:00 2001 From: Reuven Lax Date: Tue, 15 Aug 2017 21:58:14 -0700 Subject: [PATCH 275/346] Ensure that each triggered load generates a different job id (for the case of streaming triggered file loads), and add test coverage to catch this. --- .../beam/sdk/io/gcp/bigquery/BigQueryHelpers.java | 12 ++++++++---- .../apache/beam/sdk/io/gcp/bigquery/WriteRename.java | 3 ++- .../apache/beam/sdk/io/gcp/bigquery/WriteTables.java | 4 ++-- .../beam/sdk/io/gcp/bigquery/BigQueryIOTest.java | 2 +- .../beam/sdk/io/gcp/bigquery/FakeJobService.java | 9 +++++++++ 5 files changed, 22 insertions(+), 8 deletions(-) diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryHelpers.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryHelpers.java index 09508e0a46283..78dcdde097ad4 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryHelpers.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryHelpers.java @@ -225,15 +225,19 @@ static void verifyTablePresence(DatasetService datasetService, TableReference ta } // Create a unique job id for a table load. - static String createJobId(String prefix, TableDestination tableDestination, int partition) { + static String createJobId(String prefix, TableDestination tableDestination, int partition, + long index) { // Job ID must be different for each partition of each table. String destinationHash = Hashing.murmur3_128().hashUnencodedChars(tableDestination.toString()).toString(); + String jobId = String.format("%s_%s", prefix, destinationHash); if (partition >= 0) { - return String.format("%s_%s_%05d", prefix, destinationHash, partition); - } else { - return String.format("%s_%s", prefix, destinationHash); + jobId += String.format("_%05d", partition); + } + if (index >= 0) { + jobId += String.format("_%05d", index); } + return jobId; } @VisibleForTesting diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/WriteRename.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/WriteRename.java index eb1da5f7ffec8..ff694765d30a5 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/WriteRename.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/WriteRename.java @@ -101,7 +101,8 @@ private void writeRename( // Make sure each destination table gets a unique job id. String jobIdPrefix = - BigQueryHelpers.createJobId(c.sideInput(jobIdToken), finalTableDestination, -1); + BigQueryHelpers.createJobId(c.sideInput(jobIdToken), finalTableDestination, -1, + c.pane().getIndex()); copy( bqServices.getJobService(c.getPipelineOptions().as(BigQueryOptions.class)), diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/WriteTables.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/WriteTables.java index 24911a7ccc125..c8fab75f75739 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/WriteTables.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/WriteTables.java @@ -119,8 +119,8 @@ public void processElement(ProcessContext c) throws Exception { Integer partition = c.element().getKey().getShardNumber(); List partitionFiles = Lists.newArrayList(c.element().getValue()); - String jobIdPrefix = - BigQueryHelpers.createJobId(c.sideInput(jobIdToken), tableDestination, partition); + String jobIdPrefix = BigQueryHelpers.createJobId( + c.sideInput(jobIdToken), tableDestination, partition, c.pane().getIndex()); if (!singlePartition) { tableReference.setTableId(jobIdPrefix); 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 3d53b7e39ed63..43a494eb3c96e 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 @@ -1955,7 +1955,7 @@ public void testWriteTables() throws Exception { String tableName = String.format("project-id:dataset-id.table%05d", i); TableDestination tableDestination = new TableDestination(tableName, tableName); for (int j = 0; j < numPartitions; ++j) { - String tempTableId = BigQueryHelpers.createJobId(jobIdToken, tableDestination, j); + String tempTableId = BigQueryHelpers.createJobId(jobIdToken, tableDestination, j, -1); List filesPerPartition = Lists.newArrayList(); for (int k = 0; k < numFilesPerPartition; ++k) { String filename = Paths.get(baseDir.toString(), diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/FakeJobService.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/FakeJobService.java index 2045bb7bda26f..7d5101d782f9d 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/FakeJobService.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/FakeJobService.java @@ -109,6 +109,7 @@ private static class JobInfo { public void startLoadJob(JobReference jobRef, JobConfigurationLoad loadConfig) throws InterruptedException, IOException { synchronized (allJobs) { + verifyUniqueJobId(jobRef.getJobId()); Job job = new Job(); job.setJobReference(jobRef); job.setConfiguration(new JobConfiguration().setLoad(loadConfig)); @@ -141,6 +142,7 @@ public void startExtractJob(JobReference jobRef, JobConfigurationExtract extract checkArgument(extractConfig.getDestinationFormat().equals("AVRO"), "Only extract to AVRO is supported"); synchronized (allJobs) { + verifyUniqueJobId(jobRef.getJobId()); ++numExtractJobCalls; Job job = new Job(); @@ -175,6 +177,7 @@ public void startQueryJob(JobReference jobRef, JobConfigurationQuery query) public void startCopyJob(JobReference jobRef, JobConfigurationTableCopy copyConfig) throws IOException, InterruptedException { synchronized (allJobs) { + verifyUniqueJobId(jobRef.getJobId()); Job job = new Job(); job.setJobReference(jobRef); job.setConfiguration(new JobConfiguration().setCopy(copyConfig)); @@ -257,6 +260,12 @@ public Job getJob(JobReference jobRef) throws InterruptedException { } } + private void verifyUniqueJobId(String jobId) throws IOException { + if (allJobs.containsColumn(jobId)) { + throw new IOException("Duplicate job id " + jobId); + } + } + private JobStatus runJob(Job job) throws InterruptedException, IOException { if (job.getConfiguration().getLoad() != null) { return runLoadJob(job.getJobReference(), job.getConfiguration().getLoad()); From 9ac94e7d45b0c57ba16f48f129c595bbbf041c1f Mon Sep 17 00:00:00 2001 From: Reuven Lax Date: Thu, 17 Aug 2017 12:57:47 -0700 Subject: [PATCH 276/346] Fix failing test. --- .../org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOTest.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 43a494eb3c96e..0ece3ee0ba1df 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 @@ -1955,7 +1955,7 @@ public void testWriteTables() throws Exception { String tableName = String.format("project-id:dataset-id.table%05d", i); TableDestination tableDestination = new TableDestination(tableName, tableName); for (int j = 0; j < numPartitions; ++j) { - String tempTableId = BigQueryHelpers.createJobId(jobIdToken, tableDestination, j, -1); + String tempTableId = BigQueryHelpers.createJobId(jobIdToken, tableDestination, j, 0); List filesPerPartition = Lists.newArrayList(); for (int k = 0; k < numFilesPerPartition; ++k) { String filename = Paths.get(baseDir.toString(), From 1b81f1dc2bfad434fb764c61106679b4d6c94377 Mon Sep 17 00:00:00 2001 From: "chamikara@google.com" Date: Thu, 10 Aug 2017 17:35:37 -0700 Subject: [PATCH 277/346] Updates ByteKeyRangeTracker so that getFractionConsumed() does not fail for completed trackers. After this update: * getFractionConsumed() returns 1.0 after markDone() is set. * getFractionConsumed() returns 1.0 after tryReturnRecordAt() is invoked for a position that is larger than or equal to the end key. This is similar to how getFractionConsumed() method of OffsetRangeTracker is implemented. --- .../sdk/io/range/ByteKeyRangeTracker.java | 5 ++++ .../sdk/io/range/ByteKeyRangeTrackerTest.java | 23 +++++++++++++++++++ 2 files changed, 28 insertions(+) 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 b889ec755fc4c..509e434f9d450 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 @@ -127,7 +127,12 @@ public synchronized boolean trySplitAtPosition(ByteKey splitPosition) { public synchronized double getFractionConsumed() { if (position == null) { return 0; + } else if (done) { + return 1.0; + } else if (position.compareTo(range.getEndKey()) >= 0) { + return 1.0; } + return range.estimateFractionForKey(position); } diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/range/ByteKeyRangeTrackerTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/range/ByteKeyRangeTrackerTest.java index 8deaf442e554c..0523d7548353a 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/range/ByteKeyRangeTrackerTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/range/ByteKeyRangeTrackerTest.java @@ -38,6 +38,7 @@ public class ByteKeyRangeTrackerTest { private static final ByteKey NEW_MIDDLE_KEY = ByteKey.of(0x24); private static final ByteKey BEFORE_END_KEY = ByteKey.of(0x33); private static final ByteKey END_KEY = ByteKey.of(0x34); + private static final ByteKey KEY_LARGER_THAN_END = ByteKey.of(0x35); private static final double INITIAL_RANGE_SIZE = 0x34 - 0x12; private static final ByteKeyRange INITIAL_RANGE = ByteKeyRange.of(INITIAL_START_KEY, END_KEY); private static final double NEW_RANGE_SIZE = 0x34 - 0x14; @@ -98,6 +99,28 @@ public void testGetFractionConsumed() { assertEquals(1 - 1 / INITIAL_RANGE_SIZE, tracker.getFractionConsumed(), delta); } + @Test + public void testGetFractionConsumedAfterDone() { + ByteKeyRangeTracker tracker = ByteKeyRangeTracker.of(INITIAL_RANGE); + double delta = 0.00001; + + assertTrue(tracker.tryReturnRecordAt(true, INITIAL_START_KEY)); + tracker.markDone(); + + assertEquals(1.0, tracker.getFractionConsumed(), delta); + } + + @Test + public void testGetFractionConsumedAfterOutOfRangeClaim() { + ByteKeyRangeTracker tracker = ByteKeyRangeTracker.of(INITIAL_RANGE); + double delta = 0.00001; + + assertTrue(tracker.tryReturnRecordAt(true, INITIAL_START_KEY)); + assertTrue(tracker.tryReturnRecordAt(false, KEY_LARGER_THAN_END)); + + assertEquals(1.0, tracker.getFractionConsumed(), delta); + } + /** Tests for {@link ByteKeyRangeTracker#getFractionConsumed()} with updated start key. */ @Test public void testGetFractionConsumedUpdateStartKey() { From b949aa1bbfd7fbb1a8159e6d650dae6196015e5c Mon Sep 17 00:00:00 2001 From: Luke Cwik Date: Wed, 16 Aug 2017 16:44:59 -0700 Subject: [PATCH 278/346] [BEAM-1347] Convert an InputStream into an Iterable using the Beam Fn data specification This is towards sharing common code that supports the Beam Fn State API and the Beam Fn Data API. --- .../beam/fn/harness/stream/DataStreams.java | 73 +++++++- .../fn/harness/stream/DataStreamsTest.java | 165 +++++++++++++----- 2 files changed, 192 insertions(+), 46 deletions(-) diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/stream/DataStreams.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/stream/DataStreams.java index d23d784df5dc6..69671601fbc8c 100644 --- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/stream/DataStreams.java +++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/stream/DataStreams.java @@ -17,19 +17,24 @@ */ package org.apache.beam.fn.harness.stream; +import static com.google.common.base.Preconditions.checkState; + import com.google.common.io.ByteStreams; +import com.google.common.io.CountingInputStream; import com.google.protobuf.ByteString; import java.io.IOException; import java.io.InputStream; import java.io.OutputStream; +import java.io.PushbackInputStream; import java.util.Iterator; import java.util.NoSuchElementException; import java.util.concurrent.BlockingQueue; import org.apache.beam.fn.harness.fn.CloseableThrowingConsumer; +import org.apache.beam.sdk.coders.Coder; /** * {@link #inbound(Iterator)} treats multiple {@link ByteString}s as a single input stream and - * {@link #outbound(CloseableThrowingConsumer)} treats a single {@link OutputStream} as mulitple + * {@link #outbound(CloseableThrowingConsumer)} treats a single {@link OutputStream} as multiple * {@link ByteString}s. */ public class DataStreams { @@ -99,6 +104,72 @@ public int read(byte[] b, int off, int len) throws IOException { } } + /** + * An adapter which converts an {@link InputStream} to an {@link Iterator} of {@code T} values + * using the specified {@link Coder}. + * + *

              Note that this adapter follows the Beam Fn API specification for forcing values that decode + * consuming zero bytes to consuming exactly one byte. + * + *

              Note that access to the underlying {@link InputStream} is lazy and will only be invoked on + * first access to {@link #next()} or {@link #hasNext()}. + */ + public static class DataStreamDecoder implements Iterator { + private enum State { READ_REQUIRED, HAS_NEXT, EOF }; + + private final CountingInputStream countingInputStream; + private final PushbackInputStream pushbackInputStream; + private final Coder coder; + private State currentState; + private T next; + public DataStreamDecoder(Coder coder, InputStream inputStream) { + this.currentState = State.READ_REQUIRED; + this.coder = coder; + this.pushbackInputStream = new PushbackInputStream(inputStream, 1); + this.countingInputStream = new CountingInputStream(pushbackInputStream); + } + + @Override + public boolean hasNext() { + switch (currentState) { + case EOF: + return false; + case READ_REQUIRED: + try { + int nextByte = pushbackInputStream.read(); + if (nextByte == -1) { + currentState = State.EOF; + return false; + } + + pushbackInputStream.unread(nextByte); + long count = countingInputStream.getCount(); + next = coder.decode(countingInputStream); + // Skip one byte if decoding the value consumed 0 bytes. + if (countingInputStream.getCount() - count == 0) { + checkState(countingInputStream.read() != -1, "Unexpected EOF reached"); + } + currentState = State.HAS_NEXT; + } catch (IOException e) { + throw new IllegalStateException(e); + } + // fall through expected + case HAS_NEXT: + return true; + } + throw new IllegalStateException(String.format("Unknown state %s", currentState)); + } + + @Override + public T next() { + if (!hasNext()) { + throw new NoSuchElementException(); + } + currentState = State.READ_REQUIRED; + return next; + } + } + /** * Allows for one or more writing threads to append values to this iterator while one reading * thread reads values. {@link #hasNext()} and {@link #next()} will block until a value is diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/stream/DataStreamsTest.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/stream/DataStreamsTest.java index d1415700d967b..f7a87e15e8775 100644 --- a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/stream/DataStreamsTest.java +++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/stream/DataStreamsTest.java @@ -17,76 +17,151 @@ */ package org.apache.beam.fn.harness.stream; +import static org.junit.Assert.assertArrayEquals; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assume.assumeTrue; import com.google.common.collect.Iterators; +import com.google.common.io.ByteStreams; +import com.google.common.io.CountingOutputStream; import com.google.protobuf.ByteString; +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; import java.io.IOException; import java.util.Arrays; +import java.util.Iterator; import java.util.List; +import java.util.NoSuchElementException; import java.util.concurrent.ArrayBlockingQueue; import java.util.concurrent.CompletableFuture; import java.util.concurrent.SynchronousQueue; import org.apache.beam.fn.harness.stream.DataStreams.BlockingQueueIterator; +import org.apache.beam.fn.harness.stream.DataStreams.DataStreamDecoder; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.coders.StringUtf8Coder; +import org.apache.beam.sdk.transforms.windowing.GlobalWindow; +import org.junit.Rule; import org.junit.Test; +import org.junit.experimental.runners.Enclosed; +import org.junit.rules.ExpectedException; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; /** Tests for {@link DataStreams}. */ -@RunWith(JUnit4.class) +@RunWith(Enclosed.class) public class DataStreamsTest { - private static final ByteString BYTES_A = ByteString.copyFromUtf8("TestData"); - private static final ByteString BYTES_B = ByteString.copyFromUtf8("SomeOtherTestData"); - - @Test - public void testEmptyRead() throws Exception { - assertEquals(ByteString.EMPTY, read()); - assertEquals(ByteString.EMPTY, read(ByteString.EMPTY)); - assertEquals(ByteString.EMPTY, read(ByteString.EMPTY, ByteString.EMPTY)); - } - @Test - public void testRead() throws Exception { - assertEquals(BYTES_A.concat(BYTES_B), read(BYTES_A, BYTES_B)); - assertEquals(BYTES_A.concat(BYTES_B), read(BYTES_A, ByteString.EMPTY, BYTES_B)); - assertEquals(BYTES_A.concat(BYTES_B), read(BYTES_A, BYTES_B, ByteString.EMPTY)); + /** Tests for {@link DataStreams.Inbound}. */ + @RunWith(JUnit4.class) + public static class InboundTest { + private static final ByteString BYTES_A = ByteString.copyFromUtf8("TestData"); + private static final ByteString BYTES_B = ByteString.copyFromUtf8("SomeOtherTestData"); + + @Test + public void testEmptyRead() throws Exception { + assertEquals(ByteString.EMPTY, read()); + assertEquals(ByteString.EMPTY, read(ByteString.EMPTY)); + assertEquals(ByteString.EMPTY, read(ByteString.EMPTY, ByteString.EMPTY)); + } + + @Test + public void testRead() throws Exception { + assertEquals(BYTES_A.concat(BYTES_B), read(BYTES_A, BYTES_B)); + assertEquals(BYTES_A.concat(BYTES_B), read(BYTES_A, ByteString.EMPTY, BYTES_B)); + assertEquals(BYTES_A.concat(BYTES_B), read(BYTES_A, BYTES_B, ByteString.EMPTY)); + } + + private static ByteString read(ByteString... bytes) throws IOException { + return ByteString.readFrom(DataStreams.inbound(Arrays.asList(bytes).iterator())); + } } - @Test(timeout = 10_000) - public void testBlockingQueueIteratorWithoutBlocking() throws Exception { - BlockingQueueIterator iterator = - new BlockingQueueIterator<>(new ArrayBlockingQueue<>(3)); + /** Tests for {@link DataStreams.BlockingQueueIterator}. */ + @RunWith(JUnit4.class) + public static class BlockingQueueIteratorTest { + @Test(timeout = 10_000) + public void testBlockingQueueIteratorWithoutBlocking() throws Exception { + BlockingQueueIterator iterator = + new BlockingQueueIterator<>(new ArrayBlockingQueue<>(3)); - iterator.accept("A"); - iterator.accept("B"); - iterator.close(); + iterator.accept("A"); + iterator.accept("B"); + iterator.close(); - assertEquals(Arrays.asList("A", "B"), - Arrays.asList(Iterators.toArray(iterator, String.class))); + assertEquals(Arrays.asList("A", "B"), + Arrays.asList(Iterators.toArray(iterator, String.class))); + } + + @Test(timeout = 10_000) + public void testBlockingQueueIteratorWithBlocking() throws Exception { + // The synchronous queue only allows for one element to transfer at a time and blocks + // the sending/receiving parties until both parties are there. + final BlockingQueueIterator iterator = + new BlockingQueueIterator<>(new SynchronousQueue<>()); + final CompletableFuture> valuesFuture = new CompletableFuture<>(); + Thread appender = new Thread() { + @Override + public void run() { + valuesFuture.complete(Arrays.asList(Iterators.toArray(iterator, String.class))); + } + }; + appender.start(); + iterator.accept("A"); + iterator.accept("B"); + iterator.close(); + assertEquals(Arrays.asList("A", "B"), valuesFuture.get()); + appender.join(); + } } - @Test(timeout = 10_000) - public void testBlockingQueueIteratorWithBlocking() throws Exception { - // The synchronous queue only allows for one element to transfer at a time and blocks - // the sending/receiving parties until both parties are there. - final BlockingQueueIterator iterator = - new BlockingQueueIterator<>(new SynchronousQueue<>()); - final CompletableFuture> valuesFuture = new CompletableFuture<>(); - Thread appender = new Thread() { - @Override - public void run() { - valuesFuture.complete(Arrays.asList(Iterators.toArray(iterator, String.class))); + /** Tests for {@link DataStreams.DataStreamDecoder}. */ + @RunWith(JUnit4.class) + public static class DataStreamDecoderTest { + @Rule public ExpectedException thrown = ExpectedException.none(); + + @Test + public void testEmptyInputStream() throws Exception { + testDecoderWith(StringUtf8Coder.of()); + } + + @Test + public void testNonEmptyInputStream() throws Exception { + testDecoderWith(StringUtf8Coder.of(), "A", "BC", "DEF", "GHIJ"); + } + + @Test + public void testNonEmptyInputStreamWithZeroLengthCoder() throws Exception { + CountingOutputStream countingOutputStream = + new CountingOutputStream(ByteStreams.nullOutputStream()); + GlobalWindow.Coder.INSTANCE.encode(GlobalWindow.INSTANCE, countingOutputStream); + assumeTrue(countingOutputStream.getCount() == 0); + + testDecoderWith(GlobalWindow.Coder.INSTANCE, GlobalWindow.INSTANCE, GlobalWindow.INSTANCE); + } + + private void testDecoderWith(Coder coder, T... expected) throws IOException { + ByteArrayOutputStream baos = new ByteArrayOutputStream(); + for (T value : expected) { + int size = baos.size(); + coder.encode(value, baos); + // Pad an arbitrary byte when values encode to zero bytes + if (baos.size() - size == 0) { + baos.write(0); + } } - }; - appender.start(); - iterator.accept("A"); - iterator.accept("B"); - iterator.close(); - assertEquals(Arrays.asList("A", "B"), valuesFuture.get()); - appender.join(); - } - private static ByteString read(ByteString... bytes) throws IOException { - return ByteString.readFrom(DataStreams.inbound(Arrays.asList(bytes).iterator())); + Iterator decoder = + new DataStreamDecoder<>(coder, new ByteArrayInputStream(baos.toByteArray())); + + Object[] actual = Iterators.toArray(decoder, Object.class); + assertArrayEquals(expected, actual); + + assertFalse(decoder.hasNext()); + assertFalse(decoder.hasNext()); + + thrown.expect(NoSuchElementException.class); + decoder.next(); + } } } From 17ff7fb1c7002f6bca57d1153773adae6028783d Mon Sep 17 00:00:00 2001 From: huafengw Date: Thu, 17 Aug 2017 11:58:33 +0800 Subject: [PATCH 279/346] [BEAM-79] Add Gearpump runner profile to Java8 Archetype POM --- .../src/main/resources/archetype-resources/pom.xml | 13 +++++++++++++ 1 file changed, 13 insertions(+) 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 af4fbd3832e38..ffdb066046667 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 @@ -222,6 +222,19 @@ + + gearpump-runner + + + + org.apache.beam + beam-runners-gearpump + ${beam.version} + runtime + + + + spark-runner + + 4.0.0 + + + org.apache.beam + parent + 0.2.0-incubating-SNAPSHOT + ../../pom.xml + + + java-integration-all + + Apache Beam :: Integration Tests :: Java All + + jar + + + true + + + + + + + kr.motd.maven + os-maven-plugin + 1.4.0.Final + + + + + + org.apache.maven.plugins + maven-compiler-plugin + + + + org.apache.maven.plugins + maven-surefire-plugin + + + + + + + + + + org.apache.maven.plugins + maven-dependency-plugin + + + + org.apache.maven.plugins + maven-checkstyle-plugin + + + + + + org.apache.maven.plugins + maven-source-plugin + + + + + org.apache.maven.plugins + maven-shade-plugin + + + package + + shade + + + ${project.artifactId}-bundled-${project.version} + + + *:* + + + + + *:* + + META-INF/*.SF + META-INF/*.DSA + META-INF/*.RSA + + + + + + + + + + org.apache.maven.plugins + maven-jar-plugin + + + + + org.apache.avro + avro-maven-plugin + ${avro.version} + + + schemas + generate-sources + + schema + + + ${project.basedir}/src/main/ + ${project.build.directory}/generated-sources/java + + + + + + + + org.jacoco + jacoco-maven-plugin + + + + + + + + org.apache.beam + java-sdk-all + + + + + org.apache.beam.runners + google-cloud-dataflow-java + ${project.version} + + + + + org.apache.beam.runners + direct + ${project.version} + + + + + org.apache.beam.runners + flink_2.10 + ${project.version} + + + + org.apache.flink + flink-shaded-hadoop2 + 1.0.3 + provided + + + + + com.google.apis + google-api-services-dataflow + ${dataflow.version} + + + + joda-time + joda-time + + + + com.fasterxml.jackson.core + jackson-core + ${jackson.version} + + + + com.google.apis + google-api-services-bigquery + ${bigquery.version} + + + + com.google.cloud.bigdataoss + gcsio + ${google-cloud-bigdataoss.version} + + + + com.fasterxml.jackson.core + jackson-annotations + ${jackson.version} + + + + com.fasterxml.jackson.core + jackson-databind + ${jackson.version} + + + + org.apache.avro + avro + ${avro.version} + + + + com.google.guava + guava + ${guava.version} + + + + org.hamcrest + hamcrest-all + ${hamcrest.version} + + + + org.slf4j + slf4j-api + ${slf4j.version} + compile + + + + org.slf4j + slf4j-jdk14 + ${slf4j.version} + + runtime + + + + com.google.code.findbugs + jsr305 + ${jsr305.version} + runtime + + + + log4j + log4j + 1.2.17 + runtime + + + + junit + junit + ${junit.version} + compile + + + + io.netty + netty-tcnative-boringssl-static + 1.1.33.Fork13 + ${os.detected.classifier} + runtime + + + diff --git a/integration/java/src/main/java/org/apache/beam/integration/nexmark/AbstractSimulator.java b/integration/java/src/main/java/org/apache/beam/integration/nexmark/AbstractSimulator.java new file mode 100644 index 0000000000000..6473c35c39964 --- /dev/null +++ b/integration/java/src/main/java/org/apache/beam/integration/nexmark/AbstractSimulator.java @@ -0,0 +1,212 @@ +/* + * 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.integration.nexmark; + +import org.apache.beam.sdk.transforms.windowing.BoundedWindow; +import org.apache.beam.sdk.values.TimestampedValue; + +import org.joda.time.Duration; +import org.joda.time.Instant; + +import java.util.ArrayList; +import java.util.Iterator; +import java.util.List; + +import javax.annotation.Nullable; + +/** + * Abstract base class for simulator of a query. + * + * @param Type of input elements. + * @param Type of output elements. + */ +abstract class AbstractSimulator { + /** Window size for action bucket sampling. */ + public static final Duration WINDOW_SIZE = Duration.standardMinutes(1); + + /** Input event stream we should draw from. */ + private final Iterator> input; + + /** Set to true when no more results. */ + private boolean isDone; + + /** + * Results which have not yet been returned by the {@link #results} iterator. + */ + private final List> pendingResults; + + /** + * Current window timestamp (ms since epoch). + */ + private long currentWindow; + + /** + * Number of (possibly intermediate) results for the current window. + */ + private long currentCount; + + /** + * Result counts per window which have not yet been returned by the {@link #resultsPerWindow} + * iterator. + */ + private final List pendingCounts; + + public AbstractSimulator(Iterator> input) { + this.input = input; + isDone = false; + pendingResults = new ArrayList<>(); + currentWindow = BoundedWindow.TIMESTAMP_MIN_VALUE.getMillis(); + currentCount = 0; + pendingCounts = new ArrayList<>(); + } + + /** Called by implementors of {@link #run}: Fetch the next input element. */ + @Nullable + protected TimestampedValue nextInput() { + if (!input.hasNext()) { + return null; + } + TimestampedValue timestampedInput = input.next(); + NexmarkUtils.info("input: %s", timestampedInput); + return timestampedInput; + } + + /** + * Called by implementors of {@link #run}: Capture an intermediate result, for the purpose of + * recording the expected activity of the query over time. + */ + protected void addIntermediateResult(TimestampedValue result) { + NexmarkUtils.info("intermediate result: %s", result); + updateCounts(result.getTimestamp()); + } + + /** + * Called by implementors of {@link #run}: Capture a final result, for the purpose of checking + * semantic correctness. + */ + protected void addResult(TimestampedValue result) { + NexmarkUtils.info("result: %s", result); + pendingResults.add(result); + updateCounts(result.getTimestamp()); + } + + /** + * Update window and counts. + */ + private void updateCounts(Instant timestamp) { + long window = timestamp.getMillis() - timestamp.getMillis() % WINDOW_SIZE.getMillis(); + if (window > currentWindow) { + if (currentWindow > BoundedWindow.TIMESTAMP_MIN_VALUE.getMillis()) { + pendingCounts.add(currentCount); + } + currentCount = 0; + currentWindow = window; + } + currentCount++; + } + + /** Called by implementors of {@link #run}: Record that no more results will be emitted. */ + protected void allDone() { + isDone = true; + } + + /** + * Overridden by derived classes to do the next increment of work. Each call should + * call one or more of {@link #nextInput}, {@link #addIntermediateResult}, {@link #addResult} + * or {@link #allDone}. It is ok for a single call to emit more than one result via + * {@link #addResult}. It is ok for a single call to run the entire simulation, though + * this will prevent the {@link #results} and {@link #resultsPerWindow} iterators to + * stall. + */ + protected abstract void run(); + + /** + * Return iterator over all expected timestamped results. The underlying simulator state is + * changed. Only one of {@link #results} or {@link #resultsPerWindow} can be called. + */ + public Iterator> results() { + return new Iterator>() { + @Override + public boolean hasNext() { + while (true) { + if (!pendingResults.isEmpty()) { + return true; + } + if (isDone) { + return false; + } + run(); + } + } + + @Override + public TimestampedValue next() { + TimestampedValue result = pendingResults.get(0); + pendingResults.remove(0); + return result; + } + + @Override + public void remove() { + throw new UnsupportedOperationException(); + } + }; + } + + /** + * Return an iterator over the number of results per {@link #WINDOW_SIZE} period. The underlying + * simulator state is changed. Only one of {@link #results} or {@link #resultsPerWindow} can be + * called. + */ + public Iterator resultsPerWindow() { + return new Iterator() { + @Override + public boolean hasNext() { + while (true) { + if (!pendingCounts.isEmpty()) { + return true; + } + if (isDone) { + if (currentCount > 0) { + pendingCounts.add(currentCount); + currentCount = 0; + currentWindow = BoundedWindow.TIMESTAMP_MAX_VALUE.getMillis(); + return true; + } else { + return false; + } + } + run(); + } + } + + @Override + public Long next() { + Long result = pendingCounts.get(0); + pendingCounts.remove(0); + return result; + } + + @Override + public void remove() { + throw new UnsupportedOperationException(); + } + }; + } +} diff --git a/integration/java/src/main/java/org/apache/beam/integration/nexmark/Auction.java b/integration/java/src/main/java/org/apache/beam/integration/nexmark/Auction.java new file mode 100644 index 0000000000000..94f2647e28d7d --- /dev/null +++ b/integration/java/src/main/java/org/apache/beam/integration/nexmark/Auction.java @@ -0,0 +1,190 @@ +/* + * 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.integration.nexmark; + +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.StringUtf8Coder; +import org.apache.beam.sdk.coders.VarLongCoder; + +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.core.JsonProcessingException; + +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.io.Serializable; + +/** + * An auction submitted by a person. + */ +public class Auction implements KnownSize, Serializable { + private static final Coder LONG_CODER = VarLongCoder.of(); + private static final Coder STRING_CODER = StringUtf8Coder.of(); + + public static final Coder CODER = new AtomicCoder() { + @Override + public void encode(Auction value, OutputStream outStream, + Coder.Context context) + throws CoderException, IOException { + LONG_CODER.encode(value.id, outStream, Context.NESTED); + STRING_CODER.encode(value.itemName, outStream, Context.NESTED); + STRING_CODER.encode(value.description, outStream, Context.NESTED); + LONG_CODER.encode(value.initialBid, outStream, Context.NESTED); + LONG_CODER.encode(value.reserve, outStream, Context.NESTED); + LONG_CODER.encode(value.dateTime, outStream, Context.NESTED); + LONG_CODER.encode(value.expires, outStream, Context.NESTED); + LONG_CODER.encode(value.seller, outStream, Context.NESTED); + LONG_CODER.encode(value.category, outStream, Context.NESTED); + STRING_CODER.encode(value.extra, outStream, Context.NESTED); + } + + @Override + public Auction decode( + InputStream inStream, Coder.Context context) + throws CoderException, IOException { + long id = LONG_CODER.decode(inStream, Context.NESTED); + String itemName = STRING_CODER.decode(inStream, Context.NESTED); + String description = STRING_CODER.decode(inStream, Context.NESTED); + long initialBid = LONG_CODER.decode(inStream, Context.NESTED); + long reserve = LONG_CODER.decode(inStream, Context.NESTED); + long dateTime = LONG_CODER.decode(inStream, Context.NESTED); + long expires = LONG_CODER.decode(inStream, Context.NESTED); + long seller = LONG_CODER.decode(inStream, Context.NESTED); + long category = LONG_CODER.decode(inStream, Context.NESTED); + String extra = STRING_CODER.decode(inStream, Context.NESTED); + return new Auction( + id, itemName, description, initialBid, reserve, dateTime, expires, seller, category, + extra); + } + }; + + + /** Id of auction. */ + @JsonProperty + public final long id; // primary key + + /** Extra auction properties. */ + @JsonProperty + public final String itemName; + + @JsonProperty + public final String description; + + /** Initial bid price, in cents. */ + @JsonProperty + public final long initialBid; + + /** Reserve price, in cents. */ + @JsonProperty + public final long reserve; + + @JsonProperty + public final long dateTime; + + /** When does auction expire? (ms since epoch). Bids at or after this time are ignored. */ + @JsonProperty + public final long expires; + + /** Id of person who instigated auction. */ + @JsonProperty + public final long seller; // foreign key: Person.id + + /** Id of category auction is listed under. */ + @JsonProperty + public final long category; // foreign key: Category.id + + /** Additional arbitrary payload for performance testing. */ + @JsonProperty + public final String extra; + + + // For Avro only. + @SuppressWarnings("unused") + private Auction() { + id = 0; + itemName = null; + description = null; + initialBid = 0; + reserve = 0; + dateTime = 0; + expires = 0; + seller = 0; + category = 0; + extra = null; + } + + public Auction(long id, String itemName, String description, long initialBid, long reserve, + long dateTime, long expires, long seller, long category, String extra) { + this.id = id; + this.itemName = itemName; + this.description = description; + this.initialBid = initialBid; + this.reserve = reserve; + this.dateTime = dateTime; + this.expires = expires; + this.seller = seller; + this.category = category; + this.extra = extra; + } + + /** + * Return a copy of auction which capture the given annotation. + * (Used for debugging). + */ + public Auction withAnnotation(String annotation) { + return new Auction(id, itemName, description, initialBid, reserve, dateTime, expires, seller, + category, annotation + ": " + extra); + } + + /** + * Does auction have {@code annotation}? (Used for debugging.) + */ + public boolean hasAnnotation(String annotation) { + return extra.startsWith(annotation + ": "); + } + + /** + * Remove {@code annotation} from auction. (Used for debugging.) + */ + public Auction withoutAnnotation(String annotation) { + if (hasAnnotation(annotation)) { + return new Auction(id, itemName, description, initialBid, reserve, dateTime, expires, seller, + category, extra.substring(annotation.length() + 2)); + } else { + return this; + } + } + + @Override + public long sizeInBytes() { + return 8 + itemName.length() + 1 + description.length() + 1 + 8 + 8 + 8 + 8 + 8 + 8 + + extra.length() + 1; + } + + @Override + public String toString() { + try { + return NexmarkUtils.MAPPER.writeValueAsString(this); + } catch (JsonProcessingException e) { + throw new RuntimeException(e); + } + } +} diff --git a/integration/java/src/main/java/org/apache/beam/integration/nexmark/AuctionBid.java b/integration/java/src/main/java/org/apache/beam/integration/nexmark/AuctionBid.java new file mode 100644 index 0000000000000..8c3697a3780e3 --- /dev/null +++ b/integration/java/src/main/java/org/apache/beam/integration/nexmark/AuctionBid.java @@ -0,0 +1,87 @@ +/* + * 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.integration.nexmark; + +import org.apache.beam.sdk.coders.AtomicCoder; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.coders.CoderException; + +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.core.JsonProcessingException; + +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.io.Serializable; + +/** + * Result of {@link WinningBids} transform. + */ +public class AuctionBid implements KnownSize, Serializable { + public static final Coder CODER = new AtomicCoder() { + @Override + public void encode(AuctionBid value, OutputStream outStream, + Coder.Context context) + throws CoderException, IOException { + Auction.CODER.encode(value.auction, outStream, Context.NESTED); + Bid.CODER.encode(value.bid, outStream, Context.NESTED); + } + + @Override + public AuctionBid decode( + InputStream inStream, Coder.Context context) + throws CoderException, IOException { + Auction auction = Auction.CODER.decode(inStream, Context.NESTED); + Bid bid = Bid.CODER.decode(inStream, Context.NESTED); + return new AuctionBid(auction, bid); + } + }; + + @JsonProperty + public final Auction auction; + + @JsonProperty + public final Bid bid; + + // For Avro only. + @SuppressWarnings("unused") + private AuctionBid() { + auction = null; + bid = null; + } + + public AuctionBid(Auction auction, Bid bid) { + this.auction = auction; + this.bid = bid; + } + + @Override + public long sizeInBytes() { + return auction.sizeInBytes() + bid.sizeInBytes(); + } + + @Override + public String toString() { + try { + return NexmarkUtils.MAPPER.writeValueAsString(this); + } catch (JsonProcessingException e) { + throw new RuntimeException(e); + } + } +} diff --git a/integration/java/src/main/java/org/apache/beam/integration/nexmark/AuctionCount.java b/integration/java/src/main/java/org/apache/beam/integration/nexmark/AuctionCount.java new file mode 100644 index 0000000000000..a0fbebc36e2de --- /dev/null +++ b/integration/java/src/main/java/org/apache/beam/integration/nexmark/AuctionCount.java @@ -0,0 +1,90 @@ +/* + * 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.integration.nexmark; + +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.VarLongCoder; + +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.core.JsonProcessingException; + +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.io.Serializable; + +/** + * Result of {@link Query5}. + */ +public class AuctionCount implements KnownSize, Serializable { + private static final Coder LONG_CODER = VarLongCoder.of(); + + public static final Coder CODER = new AtomicCoder() { + @Override + public void encode(AuctionCount value, OutputStream outStream, + Coder.Context context) + throws CoderException, IOException { + LONG_CODER.encode(value.auction, outStream, Context.NESTED); + LONG_CODER.encode(value.count, outStream, Context.NESTED); + } + + @Override + public AuctionCount decode( + InputStream inStream, Coder.Context context) + throws CoderException, IOException { + long auction = LONG_CODER.decode(inStream, Context.NESTED); + long count = LONG_CODER.decode(inStream, Context.NESTED); + return new AuctionCount(auction, count); + } + }; + + @JsonProperty + public final long auction; + + @JsonProperty + public final long count; + + // For Avro only. + @SuppressWarnings("unused") + private AuctionCount() { + auction = 0; + count = 0; + } + + public AuctionCount(long auction, long count) { + this.auction = auction; + this.count = count; + } + + @Override + public long sizeInBytes() { + return 8 + 8; + } + + @Override + public String toString() { + try { + return NexmarkUtils.MAPPER.writeValueAsString(this); + } catch (JsonProcessingException e) { + throw new RuntimeException(e); + } + } +} diff --git a/integration/java/src/main/java/org/apache/beam/integration/nexmark/AuctionPrice.java b/integration/java/src/main/java/org/apache/beam/integration/nexmark/AuctionPrice.java new file mode 100644 index 0000000000000..4f25a9b64de8b --- /dev/null +++ b/integration/java/src/main/java/org/apache/beam/integration/nexmark/AuctionPrice.java @@ -0,0 +1,91 @@ +/* + * 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.integration.nexmark; + +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.VarLongCoder; + +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.core.JsonProcessingException; + +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.io.Serializable; + +/** + * Result of {@link Query2}. + */ +public class AuctionPrice implements KnownSize, Serializable { + private static final Coder LONG_CODER = VarLongCoder.of(); + + public static final Coder CODER = new AtomicCoder() { + @Override + public void encode(AuctionPrice value, OutputStream outStream, + Coder.Context context) + throws CoderException, IOException { + LONG_CODER.encode(value.auction, outStream, Context.NESTED); + LONG_CODER.encode(value.price, outStream, Context.NESTED); + } + + @Override + public AuctionPrice decode( + InputStream inStream, Coder.Context context) + throws CoderException, IOException { + long auction = LONG_CODER.decode(inStream, Context.NESTED); + long price = LONG_CODER.decode(inStream, Context.NESTED); + return new AuctionPrice(auction, price); + } + }; + + @JsonProperty + public final long auction; + + /** Price in cents. */ + @JsonProperty + public final long price; + + // For Avro only. + @SuppressWarnings("unused") + private AuctionPrice() { + auction = 0; + price = 0; + } + + public AuctionPrice(long auction, long price) { + this.auction = auction; + this.price = price; + } + + @Override + public long sizeInBytes() { + return 8 + 8; + } + + @Override + public String toString() { + try { + return NexmarkUtils.MAPPER.writeValueAsString(this); + } catch (JsonProcessingException e) { + throw new RuntimeException(e); + } + } +} diff --git a/integration/java/src/main/java/org/apache/beam/integration/nexmark/BEAM_ON_FLINK_ON_GCP.md b/integration/java/src/main/java/org/apache/beam/integration/nexmark/BEAM_ON_FLINK_ON_GCP.md new file mode 100644 index 0000000000000..d1b51e8a02be9 --- /dev/null +++ b/integration/java/src/main/java/org/apache/beam/integration/nexmark/BEAM_ON_FLINK_ON_GCP.md @@ -0,0 +1,282 @@ + + +# Running NexMark on Beam on Flink on Google Compute Platform + +Here's how to create a cluster of VMs on Google Compute Platform, deploy +Flink to them, and invoke a NexMark pipeline using the Beam-on-Flink +runner. + +These instructions are somewhat baroque and I hope they can be +simplified over time. + +## Prerequisites + +You'll need: + +* the Google Cloud SDK +* a clone of the Beam repository +* a Flink binary distribution +* a project on Google Compute Platform. + +## Establish the shell environment + +``` +# Beam root +BEAM= +# Flink root +FLINK_VER=flink-1.0.3 +FLINK= +# Google Cloud project +PROJECT= +# Google Cloud zone +ZONE= +# Cloud commands +GCLOUD= +GSUTIL= +``` + +## Establish VM names for Flink master and workers + +``` +MASTER=flink-m +NUM_WORKERS=5 +WORKERS="" +for (( i = 0; i < $NUM_WORKERS; i++ )); do + WORKERS="$WORKERS flink-w-$i" +done +ALL="$MASTER $WORKERS" +``` + +## Build Beam + +``` +( cd $BEAM && mvn clean install ) +``` + +## Bring up the cluster + +Establish project defaults and authenticate: +``` +$GCLOUD init +$GCLOUD auth login +``` + +Build Google Cloud Dataproc cluster: +``` +$GCLOUD beta dataproc clusters create \ + --project=$PROJECT \ + --zone=$ZONE \ + --bucket=nexmark \ + --scopes=cloud-platform \ + --num-workers=$NUM_WORKERS \ + --image-version=preview \ + flink +``` + +Force google_compute_engine ssh keys to be generated locally: +``` +$GCLOUD compute ssh \ + --project=$PROJECT \ + --zone=$ZONE \ + $MASTER \ + --command 'exit' +``` + +Open ports on the VMs: +``` +$GCLOUD compute firewall-rules create allow-monitoring --allow tcp:8080-8081 +$GCLOUD compute firewall-rules create allow-debug --allow tcp:5555 +``` + +Establish keys on master and workers +**CAUTION:** This will leave your private key on your master VM. +Better would be to create a key just for inter-worker ssh. +``` +for m in $ALL; do + echo "*** $m ***" + $GCLOUD beta compute scp \ + --project=$PROJECT \ + --zone=$ZONE \ + ~/.ssh/google_compute_engine.pub $m:~/.ssh/ +done +$GCLOUD beta compute scp \ + --project=$PROJECT \ + --zone=$ZONE \ + ~/.ssh/google_compute_engine $MASTER:~/.ssh/ +``` + +Collect IP addresses for workers: +``` +MASTER_EXT_IP=$($GCLOUD compute instances describe \ + --project=$PROJECT \ + --zone=$ZONE \ + $MASTER | grep natIP: | sed 's/[ ]*natIP:[ ]*//') +MASTER_IP=$($GCLOUD compute instances describe \ + --project=$PROJECT \ + --zone=$ZONE \ + $MASTER | grep networkIP: | sed 's/[ ]*networkIP:[ ]*//') +WORKER_IPS="" +for m in $WORKERS; do + echo "*** $m ***" + WORKER_IP=$($GCLOUD compute instances describe \ + --project=$PROJECT \ + --zone=$ZONE \ + $m | grep networkIP: | sed 's/[ ]*networkIP:[ ]*//') + WORKER_IPS="$WORKER_IPS $WORKER_IP" +done +``` + +Configure Flink: +``` +cat $FLINK/conf/flink-conf.yaml \ + | sed "s|.*\(jobmanager.rpc.address\):.*|\1: $MASTER_IP|g" \ + | sed "s|.*\(jobmanager.heap.mb\):.*|\1: 4096|g" \ + | sed "s|.*\(taskmanager.heap.mb\):.*|\1: 8192|g" \ + | sed "s|.*\(parallelism.default\):.*|\1: $(($NUM_WORKERS * 4))|g" \ + | sed "s|.*\(fs.hdfs.hadoopconf\):.*|\1: /etc/hadoop/conf|g" \ + | sed "s|.*\(taskmanager.numberOfTaskSlots\):.*|\1: 4|g" \ + | sed "s|.*\(jobmanager.web.submit.enable\):.*|\1: false|g" \ + | sed "s|.*\(env.ssh.opts\):.*||g" \ + > ~/flink-conf.yaml +cat $FLINK/conf/log4j.properties \ + | sed "s|.*\(log4j.rootLogger\)=.*|\1=ERROR, file|g" \ + > ~/log4j.properties +echo "env.ssh.opts: -i /home/$USER/.ssh/google_compute_engine -o StrictHostKeyChecking=no" >> ~/flink-conf.yaml +echo "$MASTER_IP:8081" > ~/masters +echo -n > ~/slaves +for ip in $WORKER_IPS; do + echo $ip >> ~/slaves +done +cp -f \ + ~/flink-conf.yaml \ + ~/masters ~/slaves \ + ~/log4j.properties \ + $FLINK/conf/ +``` + +Package configured Flink for distribution to workers: +``` +( cd ~/ && tar -cvzf ~/flink.tgz $FLINK/* ) +``` + +Distribute: +``` +$GSUTIL cp ~/flink.tgz gs://nexmark +for m in $ALL; do + echo "*** $m ***" + $GCLOUD compute ssh \ + --project=$PROJECT \ + --zone=$ZONE \ + $m \ + --command 'gsutil cp gs://nexmark/flink.tgz ~/ && tar -xvzf ~/flink.tgz' +done +``` + +Start the Flink cluster: +``` +$GCLOUD compute ssh \ + --project=$PROJECT \ + --zone=$ZONE \ + $MASTER \ + --command "~/$FLINK_VER/bin/start-cluster.sh" +``` + +Bring up the Flink monitoring UI: +``` +/usr/bin/google-chrome $MASTER_EXT_IP:8081 & +``` + +## Run NexMark + +Distribute the Beam + NexMark jar to all workers: +``` +$GSUTIL cp $BEAM/integration/java/target/java-integration-all-bundled-0.2.0-incubating-SNAPSHOT.jar gs://nexmark +for m in $ALL; do + echo "*** $m ***" + $GCLOUD compute ssh \ + --project=$PROJECT \ + --zone=$ZONE \ + $m \ + --command "gsutil cp gs://nexmark/java-integration-all-bundled-0.2.0-incubating-SNAPSHOT.jar ~/$FLINK_VER/lib/" +done +``` + +Create a Pubsub topic and subscription for testing: +``` +$GCLOUD alpha pubsub \ + --project=$PROJECT \ + topics create flink_test + +$GCLOUD alpha pubsub \ + --project=$PROJECT \ + subscriptions create flink_test \ + --topic flink_test \ + --ack-deadline=60 \ + --topic-project=$PROJECT +``` + +Launch! +**NOTE:** As of flink-1.0.3 this will throw a `NullPointerException` +in `org.apache.beam.sdk.io.PubsubUnboundedSink$WriterFn.startBundle`. +See Jira issue [BEAM-196](https://issues.apache.org/jira/browse/BEAM-196). + +``` +$GCLOUD compute ssh \ + --project=$PROJECT \ + --zone=$ZONE \ + $MASTER \ + --command "~/$FLINK_VER/bin/flink run \ + -c org.apache.beam.integration.nexmark.NexmarkFlinkDriver \ + ~/$FLINK_VER/lib/java-integration-all-bundled-0.2.0-incubating-SNAPSHOT.jar \ + --project=$PROJECT \ + --streaming=true \ + --query=0 \ + --sourceType=PUBSUB \ + --pubSubMode=COMBINED \ + --pubsubTopic=flink_test \ + --resourceNameMode=VERBATIM \ + --manageResources=false \ + --monitorJobs=false \ + --numEventGenerators=5 \ + --firstEventRate=1000 \ + --nextEventRate=1000 \ + --isRateLimited=true \ + --numEvents=0 \ + --useWallclockEventTime=true \ + --usePubsubPublishTime=true" +``` + +## Teardown the cluster + +Stop the Flink cluster: +``` +$GCLOUD compute ssh \ + --project=$PROJECT \ + --zone=$ZONE \ + $MASTER \ + --command "~/$FLINK_VER/bin/stop-cluster.sh" +``` + +Teardown the Dataproc cluster: +``` +$GCLOUD beta dataproc clusters delete \ + --project=$PROJECT \ + flink +``` diff --git a/integration/java/src/main/java/org/apache/beam/integration/nexmark/Bid.java b/integration/java/src/main/java/org/apache/beam/integration/nexmark/Bid.java new file mode 100644 index 0000000000000..ce2184b221565 --- /dev/null +++ b/integration/java/src/main/java/org/apache/beam/integration/nexmark/Bid.java @@ -0,0 +1,178 @@ +/* + * 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.integration.nexmark; + +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.StringUtf8Coder; +import org.apache.beam.sdk.coders.VarLongCoder; + +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.core.JsonProcessingException; + +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.io.Serializable; +import java.util.Comparator; + +/** + * A bid for an item on auction. + */ +public class Bid implements KnownSize, Serializable { + private static final Coder LONG_CODER = VarLongCoder.of(); + private static final Coder STRING_CODER = StringUtf8Coder.of(); + + public static final Coder CODER = new AtomicCoder() { + @Override + public void encode(Bid value, OutputStream outStream, + Coder.Context context) + throws CoderException, IOException { + LONG_CODER.encode(value.auction, outStream, Context.NESTED); + LONG_CODER.encode(value.bidder, outStream, Context.NESTED); + LONG_CODER.encode(value.price, outStream, Context.NESTED); + LONG_CODER.encode(value.dateTime, outStream, Context.NESTED); + STRING_CODER.encode(value.extra, outStream, Context.NESTED); + } + + @Override + public Bid decode( + InputStream inStream, Coder.Context context) + throws CoderException, IOException { + long auction = LONG_CODER.decode(inStream, Context.NESTED); + long bidder = LONG_CODER.decode(inStream, Context.NESTED); + long price = LONG_CODER.decode(inStream, Context.NESTED); + long dateTime = LONG_CODER.decode(inStream, Context.NESTED); + String extra = STRING_CODER.decode(inStream, Context.NESTED); + return new Bid(auction, bidder, price, dateTime, extra); + } + }; + + /** + * Comparator to order bids by ascending price then descending time + * (for finding winning bids). + */ + public static final Comparator PRICE_THEN_DESCENDING_TIME = new Comparator() { + @Override + public int compare(Bid left, Bid right) { + int i = Double.compare(left.price, right.price); + if (i != 0) { + return i; + } + return Long.compare(right.dateTime, left.dateTime); + } + }; + + /** + * Comparator to order bids by ascending time then ascending price. + * (for finding most recent bids). + */ + public static final Comparator ASCENDING_TIME_THEN_PRICE = new Comparator() { + @Override + public int compare(Bid left, Bid right) { + int i = Long.compare(left.dateTime, right.dateTime); + if (i != 0) { + return i; + } + return Double.compare(left.price, right.price); + } + }; + + /** Id of auction this bid is for. */ + @JsonProperty + public final long auction; // foreign key: Auction.id + + /** Id of person bidding in auction. */ + @JsonProperty + public final long bidder; // foreign key: Person.id + + /** Price of bid, in cents. */ + @JsonProperty + public final long price; + + /** + * Instant at which bid was made (ms since epoch). + * NOTE: This may be earlier than the system's event time. + */ + @JsonProperty + public final long dateTime; + + /** Additional arbitrary payload for performance testing. */ + @JsonProperty + public final String extra; + + // For Avro only. + @SuppressWarnings("unused") + private Bid() { + auction = 0; + bidder = 0; + price = 0; + dateTime = 0; + extra = null; + } + + public Bid(long auction, long bidder, long price, long dateTime, String extra) { + this.auction = auction; + this.bidder = bidder; + this.price = price; + this.dateTime = dateTime; + this.extra = extra; + } + + /** + * Return a copy of bid which capture the given annotation. + * (Used for debugging). + */ + public Bid withAnnotation(String annotation) { + return new Bid(auction, bidder, price, dateTime, annotation + ": " + extra); + } + + /** + * Does bid have {@code annotation}? (Used for debugging.) + */ + public boolean hasAnnotation(String annotation) { + return extra.startsWith(annotation + ": "); + } + + /** + * Remove {@code annotation} from bid. (Used for debugging.) + */ + public Bid withoutAnnotation(String annotation) { + if (hasAnnotation(annotation)) { + return new Bid(auction, bidder, price, dateTime, extra.substring(annotation.length() + 2)); + } else { + return this; + } + } + + @Override + public long sizeInBytes() { + return 8 + 8 + 8 + 8 + extra.length() + 1; + } + + @Override + public String toString() { + try { + return NexmarkUtils.MAPPER.writeValueAsString(this); + } catch (JsonProcessingException e) { + throw new RuntimeException(e); + } + } +} diff --git a/integration/java/src/main/java/org/apache/beam/integration/nexmark/BidsPerSession.java b/integration/java/src/main/java/org/apache/beam/integration/nexmark/BidsPerSession.java new file mode 100644 index 0000000000000..cfdd170431bb7 --- /dev/null +++ b/integration/java/src/main/java/org/apache/beam/integration/nexmark/BidsPerSession.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.integration.nexmark; + +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.VarLongCoder; + +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.core.JsonProcessingException; + +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.io.Serializable; + +/** + * Result of query 11. + */ +public class BidsPerSession implements KnownSize, Serializable { + private static final Coder LONG_CODER = VarLongCoder.of(); + + public static final Coder CODER = new AtomicCoder() { + @Override + public void encode(BidsPerSession value, OutputStream outStream, + Coder.Context context) + throws CoderException, IOException { + LONG_CODER.encode(value.personId, outStream, Context.NESTED); + LONG_CODER.encode(value.bidsPerSession, outStream, Context.NESTED); + } + + @Override + public BidsPerSession decode( + InputStream inStream, Coder.Context context) + throws CoderException, IOException { + long personId = LONG_CODER.decode(inStream, Context.NESTED); + long bidsPerSession = LONG_CODER.decode(inStream, Context.NESTED); + return new BidsPerSession(personId, bidsPerSession); + } + }; + + @JsonProperty + public final long personId; + + @JsonProperty + public final long bidsPerSession; + + public BidsPerSession() { + personId = 0; + bidsPerSession = 0; + } + + public BidsPerSession(long personId, long bidsPerSession) { + this.personId = personId; + this.bidsPerSession = bidsPerSession; + } + + @Override + public long sizeInBytes() { + // Two longs. + return 8 + 8; + } + + @Override + public String toString() { + try { + return NexmarkUtils.MAPPER.writeValueAsString(this); + } catch (JsonProcessingException e) { + throw new RuntimeException(e); + } + } +} diff --git a/integration/java/src/main/java/org/apache/beam/integration/nexmark/BoundedEventSource.java b/integration/java/src/main/java/org/apache/beam/integration/nexmark/BoundedEventSource.java new file mode 100644 index 0000000000000..f6cc16aef16f7 --- /dev/null +++ b/integration/java/src/main/java/org/apache/beam/integration/nexmark/BoundedEventSource.java @@ -0,0 +1,197 @@ +/* + * 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.integration.nexmark; + +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.io.BoundedSource; +import org.apache.beam.sdk.options.PipelineOptions; +import org.apache.beam.sdk.values.TimestampedValue; + +import org.joda.time.Instant; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.NoSuchElementException; + +import javax.annotation.Nullable; + +/** + * A custom, bounded source of event records. + */ +class BoundedEventSource extends BoundedSource { + /** Configuration we generate events against. */ + private final GeneratorConfig config; + + /** How many bounded sources to create. */ + private final int numEventGenerators; + + public BoundedEventSource(GeneratorConfig config, int numEventGenerators) { + this.config = config; + this.numEventGenerators = numEventGenerators; + } + + /** A reader to pull events from the generator. */ + private static class EventReader extends BoundedReader { + /** + * Event source we purporting to be reading from. + * (We can't use Java's capture-outer-class pointer since we must update + * this field on calls to splitAtFraction.) + */ + private BoundedEventSource source; + + /** Generator we are reading from. */ + private final Generator generator; + + private boolean reportedStop; + + @Nullable + private TimestampedValue currentEvent; + + public EventReader(BoundedEventSource source, GeneratorConfig config) { + this.source = source; + generator = new Generator(config); + reportedStop = false; + } + + @Override + public synchronized boolean start() { + NexmarkUtils.info("starting bounded generator %s", generator); + return advance(); + } + + @Override + public synchronized boolean advance() { + if (!generator.hasNext()) { + // No more events. + if (!reportedStop) { + reportedStop = true; + NexmarkUtils.info("stopped bounded generator %s", generator); + } + return false; + } + currentEvent = generator.next(); + return true; + } + + @Override + public synchronized Event getCurrent() throws NoSuchElementException { + if (currentEvent == null) { + throw new NoSuchElementException(); + } + return currentEvent.getValue(); + } + + @Override + public synchronized Instant getCurrentTimestamp() throws NoSuchElementException { + if (currentEvent == null) { + throw new NoSuchElementException(); + } + return currentEvent.getTimestamp(); + } + + @Override + public void close() throws IOException { + // Nothing to close. + } + + @Override + public synchronized Double getFractionConsumed() { + return generator.getFractionConsumed(); + } + + @Override + public synchronized BoundedSource getCurrentSource() { + return source; + } + + @Override + @Nullable + public synchronized BoundedEventSource splitAtFraction(double fraction) { + long startId = generator.getCurrentConfig().getStartEventId(); + long stopId = generator.getCurrentConfig().getStopEventId(); + long size = stopId - startId; + long splitEventId = startId + Math.min((int) (size * fraction), size); + if (splitEventId <= generator.getNextEventId() || splitEventId == stopId) { + // Already passed this position or split results in left or right being empty. + NexmarkUtils.info("split failed for bounded generator %s at %f", generator, fraction); + return null; + } + + NexmarkUtils.info("about to split bounded generator %s at %d", generator, splitEventId); + + // Scale back the event space of the current generator, and return a generator config + // representing the event space we just 'stole' from the current generator. + GeneratorConfig remainingConfig = generator.splitAtEventId(splitEventId); + + NexmarkUtils.info("split bounded generator into %s and %s", generator, remainingConfig); + + // At this point + // generator.events() ++ new Generator(remainingConfig).events() + // == originalGenerator.events() + + // We need a new source to represent the now smaller key space for this reader, so + // that we can maintain the invariant that + // this.getCurrentSource().createReader(...) + // will yield the same output as this. + source = new BoundedEventSource(generator.getCurrentConfig(), source.numEventGenerators); + + // Return a source from which we may read the 'stolen' event space. + return new BoundedEventSource(remainingConfig, source.numEventGenerators); + } + } + + @Override + public List splitIntoBundles( + long desiredBundleSizeBytes, PipelineOptions options) { + NexmarkUtils.info("slitting bounded source %s into %d sub-sources", config, numEventGenerators); + List results = new ArrayList<>(); + // Ignore desiredBundleSizeBytes and use numEventGenerators instead. + for (GeneratorConfig subConfig : config.split(numEventGenerators)) { + results.add(new BoundedEventSource(subConfig, 1)); + } + return results; + } + + @Override + public long getEstimatedSizeBytes(PipelineOptions options) { + return config.getEstimatedSizeBytes(); + } + + @Override + public boolean producesSortedKeys(PipelineOptions options) { + return false; + } + + @Override + public EventReader createReader(PipelineOptions options) { + NexmarkUtils.info("creating initial bounded reader for %s", config); + return new EventReader(this, config); + } + + @Override + public void validate() { + // Nothing to validate. + } + + @Override + public Coder getDefaultOutputCoder() { + return Event.CODER; + } +} diff --git a/integration/java/src/main/java/org/apache/beam/integration/nexmark/CategoryPrice.java b/integration/java/src/main/java/org/apache/beam/integration/nexmark/CategoryPrice.java new file mode 100644 index 0000000000000..ab5d92d264d4c --- /dev/null +++ b/integration/java/src/main/java/org/apache/beam/integration/nexmark/CategoryPrice.java @@ -0,0 +1,100 @@ +/* + * 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.integration.nexmark; + +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.VarIntCoder; +import org.apache.beam.sdk.coders.VarLongCoder; + +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.core.JsonProcessingException; + +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.io.Serializable; + +/** + * Result of {@link Query4}. + */ +public class CategoryPrice implements KnownSize, Serializable { + private static final Coder LONG_CODER = VarLongCoder.of(); + private static final Coder INT_CODER = VarIntCoder.of(); + + public static final Coder CODER = new AtomicCoder() { + @Override + public void encode(CategoryPrice value, OutputStream outStream, + Coder.Context context) + throws CoderException, IOException { + LONG_CODER.encode(value.category, outStream, Context.NESTED); + LONG_CODER.encode(value.price, outStream, Context.NESTED); + INT_CODER.encode(value.isLast ? 1 : 0, outStream, Context.NESTED); + } + + @Override + public CategoryPrice decode( + InputStream inStream, Coder.Context context) + throws CoderException, IOException { + long category = LONG_CODER.decode(inStream, Context.NESTED); + long price = LONG_CODER.decode(inStream, Context.NESTED); + boolean isLast = INT_CODER.decode(inStream, context) != 0; + return new CategoryPrice(category, price, isLast); + } + }; + + @JsonProperty + public final long category; + + /** Price in cents. */ + @JsonProperty + public final long price; + + @JsonProperty + public final boolean isLast; + + // For Avro only. + @SuppressWarnings("unused") + private CategoryPrice() { + category = 0; + price = 0; + isLast = false; + } + + public CategoryPrice(long category, long price, boolean isLast) { + this.category = category; + this.price = price; + this.isLast = isLast; + } + + @Override + public long sizeInBytes() { + return 8 + 8 + 1; + } + + @Override + public String toString() { + try { + return NexmarkUtils.MAPPER.writeValueAsString(this); + } catch (JsonProcessingException e) { + throw new RuntimeException(e); + } + } +} diff --git a/integration/java/src/main/java/org/apache/beam/integration/nexmark/Done.java b/integration/java/src/main/java/org/apache/beam/integration/nexmark/Done.java new file mode 100644 index 0000000000000..659da441ed115 --- /dev/null +++ b/integration/java/src/main/java/org/apache/beam/integration/nexmark/Done.java @@ -0,0 +1,83 @@ +/* + * 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.integration.nexmark; + +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.StringUtf8Coder; + +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.core.JsonProcessingException; + +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.io.Serializable; + +/** + * Result of query 10. + */ +public class Done implements KnownSize, Serializable { + private static final Coder STRING_CODER = StringUtf8Coder.of(); + + public static final Coder CODER = new AtomicCoder() { + @Override + public void encode(Done value, OutputStream outStream, + Coder.Context context) + throws CoderException, IOException { + STRING_CODER.encode(value.message, outStream, Context.NESTED); + } + + @Override + public Done decode( + InputStream inStream, Coder.Context context) + throws CoderException, IOException { + String message = STRING_CODER.decode(inStream, Context.NESTED); + return new Done(message); + } + }; + + @JsonProperty + public final String message; + + // For Avro only. + @SuppressWarnings("unused") + public Done() { + message = null; + } + + public Done(String message) { + this.message = message; + } + + @Override + public long sizeInBytes() { + return message.length(); + } + + @Override + public String toString() { + try { + return NexmarkUtils.MAPPER.writeValueAsString(this); + } catch (JsonProcessingException e) { + throw new RuntimeException(e); + } + } +} diff --git a/integration/java/src/main/java/org/apache/beam/integration/nexmark/Event.java b/integration/java/src/main/java/org/apache/beam/integration/nexmark/Event.java new file mode 100644 index 0000000000000..a382b8ebed436 --- /dev/null +++ b/integration/java/src/main/java/org/apache/beam/integration/nexmark/Event.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.integration.nexmark; + +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.VarIntCoder; + +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.io.Serializable; + +import javax.annotation.Nullable; + +/** + * An event in the auction system, either a (new) {@link Person}, a (new) {@link Auction}, + * or a {@link Bid}. + */ +public class Event implements KnownSize, Serializable { + private static final Coder INT_CODER = VarIntCoder.of(); + + public static final Coder CODER = new AtomicCoder() { + @Override + public void encode(Event value, OutputStream outStream, Coder.Context context) + throws CoderException, IOException { + if (value.newPerson != null) { + INT_CODER.encode(0, outStream, Context.NESTED); + Person.CODER.encode(value.newPerson, outStream, Context.NESTED); + } else if (value.newAuction != null) { + INT_CODER.encode(1, outStream, Context.NESTED); + Auction.CODER.encode(value.newAuction, outStream, Context.NESTED); + } else if (value.bid != null) { + INT_CODER.encode(2, outStream, Context.NESTED); + Bid.CODER.encode(value.bid, outStream, Context.NESTED); + } else { + throw new RuntimeException("invalid event"); + } + } + + @Override + public Event decode( + InputStream inStream, Coder.Context context) + throws CoderException, IOException { + int tag = INT_CODER.decode(inStream, context); + if (tag == 0) { + Person person = Person.CODER.decode(inStream, Context.NESTED); + return new Event(person); + } else if (tag == 1) { + Auction auction = Auction.CODER.decode(inStream, Context.NESTED); + return new Event(auction); + } else if (tag == 2) { + Bid bid = Bid.CODER.decode(inStream, Context.NESTED); + return new Event(bid); + } else { + throw new RuntimeException("invalid event encoding"); + } + } + }; + + @Nullable + @org.apache.avro.reflect.Nullable + public final Person newPerson; + + @Nullable + @org.apache.avro.reflect.Nullable + public final Auction newAuction; + + @Nullable + @org.apache.avro.reflect.Nullable + public final Bid bid; + + // For Avro only. + @SuppressWarnings("unused") + private Event() { + newPerson = null; + newAuction = null; + bid = null; + } + + public Event(Person newPerson) { + this.newPerson = newPerson; + newAuction = null; + bid = null; + } + + public Event(Auction newAuction) { + newPerson = null; + this.newAuction = newAuction; + bid = null; + } + + public Event(Bid bid) { + newPerson = null; + newAuction = null; + this.bid = bid; + } + + /** + * Return a copy of event which captures {@code annotation}. + * (Used for debugging). + */ + public Event withAnnotation(String annotation) { + if (newPerson != null) { + return new Event(newPerson.withAnnotation(annotation)); + } else if (newAuction != null) { + return new Event(newAuction.withAnnotation(annotation)); + } else { + return new Event(bid.withAnnotation(annotation)); + } + } + + /** + * Does event have {@code annotation}? (Used for debugging.) + */ + public boolean hasAnnotation(String annotation) { + if (newPerson != null) { + return newPerson.hasAnnotation(annotation); + } else if (newAuction != null) { + return newAuction.hasAnnotation(annotation); + } else { + return bid.hasAnnotation(annotation); + } + } + + /** + * Remove {@code annotation} from event. (Used for debugging.) + */ + public Event withoutAnnotation(String annotation) { + if (newPerson != null) { + return new Event(newPerson.withoutAnnotation(annotation)); + } else if (newAuction != null) { + return new Event(newAuction.withoutAnnotation(annotation)); + } else { + return new Event(bid.withoutAnnotation(annotation)); + } + } + + @Override + public long sizeInBytes() { + if (newPerson != null) { + return 1 + newPerson.sizeInBytes(); + } else if (newAuction != null) { + return 1 + newAuction.sizeInBytes(); + } else if (bid != null) { + return 1 + bid.sizeInBytes(); + } else { + throw new RuntimeException("invalid event"); + } + } + + @Override + public String toString() { + if (newPerson != null) { + return newPerson.toString(); + } else if (newAuction != null) { + return newAuction.toString(); + } else if (bid != null) { + return bid.toString(); + } else { + throw new RuntimeException("invalid event"); + } + } +} diff --git a/integration/java/src/main/java/org/apache/beam/integration/nexmark/Generator.java b/integration/java/src/main/java/org/apache/beam/integration/nexmark/Generator.java new file mode 100644 index 0000000000000..98f4f0042b99e --- /dev/null +++ b/integration/java/src/main/java/org/apache/beam/integration/nexmark/Generator.java @@ -0,0 +1,590 @@ +/* + * 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.integration.nexmark; + +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.VarLongCoder; +import org.apache.beam.sdk.io.UnboundedSource; +import org.apache.beam.sdk.values.TimestampedValue; +import com.google.common.base.Preconditions; + +import org.joda.time.Instant; + +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.io.Serializable; +import java.util.Arrays; +import java.util.Iterator; +import java.util.List; +import java.util.Random; + +/** + * A generator for synthetic events. We try to make the data vaguely reasonable. We also ensure + * most primary key/foreign key relations are correct. Eg: a {@link Bid} event will usually have + * valid auction and bidder ids which can be joined to already-generated Auction and Person events. + * + *

              To help with testing, we generate timestamps relative to a given {@code baseTime}. Each new + * event is given a timestamp advanced from the previous timestamp by {@code interEventDelayUs} + * (in microseconds). The event stream is thus fully deterministic and does not depend on + * wallclock time. + * + *

              This class implements {@link org.apache.beam.sdk.io.UnboundedSource.CheckpointMark} + * so that we can resume generating events from a saved snapshot. + */ +public class Generator implements Iterator>, Serializable { + /** + * Keep the number of categories small so the example queries will find results even with + * a small batch of events. + */ + private static final int NUM_CATEGORIES = 5; + + /** Smallest random string size. */ + private static final int MIN_STRING_LENGTH = 3; + + /** + * Keep the number of states small so that the example queries will find results even with + * a small batch of events. + */ + private static final List US_STATES = Arrays.asList(("AZ,CA,ID,OR,WA,WY").split(",")); + + private static final List US_CITIES = + Arrays.asList( + ("Phoenix,Los Angeles,San Francisco,Boise,Portland,Bend,Redmond,Seattle,Kent,Cheyenne") + .split(",")); + + private static final List FIRST_NAMES = + Arrays.asList(("Peter,Paul,Luke,John,Saul,Vicky,Kate,Julie,Sarah,Deiter,Walter").split(",")); + + private static final List LAST_NAMES = + Arrays.asList(("Shultz,Abrams,Spencer,White,Bartels,Walton,Smith,Jones,Noris").split(",")); + + /** + * Number of yet-to-be-created people and auction ids allowed. + */ + private static final int PERSON_ID_LEAD = 10; + private static final int AUCTION_ID_LEAD = 10; + + /** + * Fraction of people/auctions which may be 'hot' sellers/bidders/auctions are 1 + * over these values. + */ + private static final int HOT_AUCTION_RATIO = 100; + private static final int HOT_SELLER_RATIO = 100; + private static final int HOT_BIDDER_RATIO = 100; + + /** + * Just enough state to be able to restore a generator back to where it was checkpointed. + */ + public static class Checkpoint implements UnboundedSource.CheckpointMark { + private static final Coder LONG_CODER = VarLongCoder.of(); + + /** Coder for this class. */ + public static final Coder CODER_INSTANCE = + new AtomicCoder() { + @Override + public void encode( + Checkpoint value, + OutputStream outStream, + Coder.Context context) + throws CoderException, IOException { + LONG_CODER.encode(value.numEvents, outStream, Context.NESTED); + LONG_CODER.encode(value.wallclockBaseTime, outStream, Context.NESTED); + } + + @Override + public Checkpoint decode( + InputStream inStream, Coder.Context context) + throws CoderException, IOException { + long numEvents = LONG_CODER.decode(inStream, Context.NESTED); + long wallclockBaseTime = LONG_CODER.decode(inStream, Context.NESTED); + return new Checkpoint(numEvents, wallclockBaseTime); + } + }; + + private long numEvents; + private long wallclockBaseTime; + + private Checkpoint(long numEvents, long wallclockBaseTime) { + this.numEvents = numEvents; + this.wallclockBaseTime = wallclockBaseTime; + } + + public Generator toGenerator(GeneratorConfig config) { + return new Generator(config, numEvents, wallclockBaseTime); + } + + @Override + public void finalizeCheckpoint() throws IOException { + // Nothing to finalize. + } + + @Override + public String toString() { + return String.format("Generator.Checkpoint{numEvents:%d;wallclockBaseTime:%d}", + numEvents, wallclockBaseTime); + } + } + + /** + * The next event and its various timestamps. Ordered by increasing wallclock timestamp, then + * (arbitrary but stable) event hash order. + */ + public static class NextEvent implements Comparable { + /** When, in wallclock time, should this event be emitted? */ + public final long wallclockTimestamp; + + /** When, in event time, should this event be considered to have occured? */ + public final long eventTimestamp; + + /** The event itself. */ + public final Event event; + + /** The minimum of this and all future event timestamps. */ + public final long watermark; + + public NextEvent(long wallclockTimestamp, long eventTimestamp, Event event, long watermark) { + this.wallclockTimestamp = wallclockTimestamp; + this.eventTimestamp = eventTimestamp; + this.event = event; + this.watermark = watermark; + } + + /** + * Return a deep clone of next event with delay added to wallclock timestamp and + * event annotate as 'LATE'. + */ + public NextEvent withDelay(long delayMs) { + return new NextEvent( + wallclockTimestamp + delayMs, eventTimestamp, event.withAnnotation("LATE"), watermark); + } + + @Override + public int compareTo(NextEvent other) { + int i = Long.compare(wallclockTimestamp, other.wallclockTimestamp); + if (i != 0) { + return i; + } + return Integer.compare(event.hashCode(), other.event.hashCode()); + } + } + + /** + * Configuration to generate events against. Note that it may be replaced by a call to + * {@link #splitAtEventId}. + */ + private GeneratorConfig config; + + /** Number of events generated by this generator. */ + private long numEvents; + + /** + * Wallclock time at which we emitted the first event (ms since epoch). Initially -1. + */ + private long wallclockBaseTime; + + private Generator(GeneratorConfig config, long numEvents, long wallclockBaseTime) { + Preconditions.checkNotNull(config); + this.config = config; + this.numEvents = numEvents; + this.wallclockBaseTime = wallclockBaseTime; + } + + /** + * Create a fresh generator according to {@code config}. + */ + public Generator(GeneratorConfig config) { + this(config, 0, -1); + } + + /** + * Return a checkpoint for the current generator. + */ + public Checkpoint toCheckpoint() { + return new Checkpoint(numEvents, wallclockBaseTime); + } + + /** + * Return a deep clone of this generator. + */ + @Override + public Generator clone() { + return new Generator(config.clone(), numEvents, wallclockBaseTime); + } + + /** + * Return the current config for this generator. Note that configs may be replaced by {@link + * #splitAtEventId}. + */ + public GeneratorConfig getCurrentConfig() { + return config; + } + + /** + * Mutate this generator so that it will only generate events up to but not including + * {@code eventId}. Return a config to represent the events this generator will no longer yield. + * The generators will run in on a serial timeline. + */ + public GeneratorConfig splitAtEventId(long eventId) { + long newMaxEvents = eventId - (config.firstEventId + config.firstEventNumber); + GeneratorConfig remainConfig = config.cloneWith(config.firstEventId, + config.maxEvents - newMaxEvents, config.firstEventNumber + newMaxEvents); + config = config.cloneWith(config.firstEventId, newMaxEvents, config.firstEventNumber); + return remainConfig; + } + + /** + * Return the next 'event id'. Though events don't have ids we can simulate them to + * help with bookkeeping. + */ + public long getNextEventId() { + return config.firstEventId + config.nextAdjustedEventNumber(numEvents); + } + + /** + * Return the last valid person id (ignoring FIRST_PERSON_ID). Will be the current person id if + * due to generate a person. + */ + private long lastBase0PersonId() { + long eventId = getNextEventId(); + long epoch = eventId / GeneratorConfig.PROPORTION_DENOMINATOR; + long offset = eventId % GeneratorConfig.PROPORTION_DENOMINATOR; + if (offset >= GeneratorConfig.PERSON_PROPORTION) { + // About to generate an auction or bid. + // Go back to the last person generated in this epoch. + offset = GeneratorConfig.PERSON_PROPORTION - 1; + } + // About to generate a person. + return epoch * GeneratorConfig.PERSON_PROPORTION + offset; + } + + /** + * Return the last valid auction id (ignoring FIRST_AUCTION_ID). Will be the current auction id if + * due to generate an auction. + */ + private long lastBase0AuctionId() { + long eventId = getNextEventId(); + long epoch = eventId / GeneratorConfig.PROPORTION_DENOMINATOR; + long offset = eventId % GeneratorConfig.PROPORTION_DENOMINATOR; + if (offset < GeneratorConfig.PERSON_PROPORTION) { + // About to generate a person. + // Go back to the last auction in the last epoch. + epoch--; + offset = GeneratorConfig.AUCTION_PROPORTION - 1; + } else if (offset >= GeneratorConfig.PERSON_PROPORTION + GeneratorConfig.AUCTION_PROPORTION) { + // About to generate a bid. + // Go back to the last auction generated in this epoch. + offset = GeneratorConfig.AUCTION_PROPORTION - 1; + } else { + // About to generate an auction. + offset -= GeneratorConfig.PERSON_PROPORTION; + } + return epoch * GeneratorConfig.AUCTION_PROPORTION + offset; + } + + /** return a random US state. */ + private static String nextUSState(Random random) { + return US_STATES.get(random.nextInt(US_STATES.size())); + } + + /** Return a random US city. */ + private static String nextUSCity(Random random) { + return US_CITIES.get(random.nextInt(US_CITIES.size())); + } + + /** Return a random person name. */ + private static String nextPersonName(Random random) { + return FIRST_NAMES.get(random.nextInt(FIRST_NAMES.size())) + " " + + LAST_NAMES.get(random.nextInt(LAST_NAMES.size())); + } + + /** Return a random string of up to {@code maxLength}. */ + private static String nextString(Random random, int maxLength) { + int len = MIN_STRING_LENGTH + random.nextInt(maxLength - MIN_STRING_LENGTH); + StringBuilder sb = new StringBuilder(); + while (len-- > 0) { + if (random.nextInt(13) == 0) { + sb.append(' '); + } else { + sb.append((char) ('a' + random.nextInt(26))); + } + } + return sb.toString().trim(); + } + + /** Return a random string of exactly {@code length}. */ + private static String nextExactString(Random random, int length) { + StringBuilder sb = new StringBuilder(); + while (length-- > 0) { + sb.append((char) ('a' + random.nextInt(26))); + } + return sb.toString(); + } + + /** Return a random email address. */ + private static String nextEmail(Random random) { + return nextString(random, 7) + "@" + nextString(random, 5) + ".com"; + } + + /** Return a random credit card number. */ + private static String nextCreditCard(Random random) { + StringBuilder sb = new StringBuilder(); + for (int i = 0; i < 4; i++) { + if (i > 0) { + sb.append(' '); + } + sb.append(String.format("%04d", random.nextInt(10000))); + } + return sb.toString(); + } + + /** Return a random price. */ + private static long nextPrice(Random random) { + return Math.round(Math.pow(10.0, random.nextDouble() * 6.0) * 100.0); + } + + /** Return a random time delay, in milliseconds, for length of auctions. */ + private long nextAuctionLengthMs(Random random, long timestamp) { + // What's our current event number? + long currentEventNumber = config.nextAdjustedEventNumber(numEvents); + // How many events till we've generated numInFlightAuctions? + long numEventsForAuctions = + (config.configuration.numInFlightAuctions * GeneratorConfig.PROPORTION_DENOMINATOR) + / GeneratorConfig.AUCTION_PROPORTION; + // When will the auction numInFlightAuctions beyond now be generated? + long futureAuction = + config.timestampAndInterEventDelayUsForEvent(currentEventNumber + numEventsForAuctions) + .getKey(); + // System.out.printf("*** auction will be for %dms (%d events ahead) ***\n", + // futureAuction - timestamp, numEventsForAuctions); + // Choose a length with average horizonMs. + long horizonMs = futureAuction - timestamp; + return 1L + nextLong(random, Math.max(horizonMs * 2, 1L)); + } + + /** + * Return a random {@code string} such that {@code currentSize + string.length()} is on average + * {@code averageSize}. + */ + private static String nextExtra(Random random, int currentSize, int desiredAverageSize) { + if (currentSize > desiredAverageSize) { + return ""; + } + desiredAverageSize -= currentSize; + int delta = (int) Math.round(desiredAverageSize * 0.2); + int minSize = desiredAverageSize - delta; + int desiredSize = minSize + (delta == 0 ? 0 : random.nextInt(2 * delta)); + return nextExactString(random, desiredSize); + } + + /** Return a random long from {@code [0, n)}. */ + private static long nextLong(Random random, long n) { + if (n < Integer.MAX_VALUE) { + return random.nextInt((int) n); + } else { + // TODO: Very skewed distribution! Bad! + return Math.abs(random.nextLong()) % n; + } + } + + /** + * Generate and return a random person with next available id. + */ + private Person nextPerson(Random random, long timestamp) { + long id = lastBase0PersonId() + GeneratorConfig.FIRST_PERSON_ID; + String name = nextPersonName(random); + String email = nextEmail(random); + String creditCard = nextCreditCard(random); + String city = nextUSCity(random); + String state = nextUSState(random); + int currentSize = + 8 + name.length() + email.length() + creditCard.length() + city.length() + state.length(); + String extra = nextExtra(random, currentSize, config.configuration.avgPersonByteSize); + return new Person(id, name, email, creditCard, city, state, timestamp, extra); + } + + /** + * Return a random person id (base 0). + */ + private long nextBase0PersonId(Random random) { + // Choose a random person from any of the 'active' people, plus a few 'leads'. + // By limiting to 'active' we ensure the density of bids or auctions per person + // does not decrease over time for long running jobs. + // By choosing a person id ahead of the last valid person id we will make + // newPerson and newAuction events appear to have been swapped in time. + long numPeople = lastBase0PersonId() + 1; + long activePeople = Math.min(numPeople, config.configuration.numActivePeople); + long n = nextLong(random, activePeople + PERSON_ID_LEAD); + return numPeople - activePeople + n; + } + + /** + * Return a random auction id (base 0). + */ + private long nextBase0AuctionId(Random random) { + // Choose a random auction for any of those which are likely to still be in flight, + // plus a few 'leads'. + // Note that ideally we'd track non-expired auctions exactly, but that state + // is difficult to split. + long minAuction = Math.max(lastBase0AuctionId() - config.configuration.numInFlightAuctions, 0); + long maxAuction = lastBase0AuctionId(); + return minAuction + nextLong(random, maxAuction - minAuction + 1 + AUCTION_ID_LEAD); + } + + /** + * Generate and return a random auction with next available id. + */ + private Auction nextAuction(Random random, long timestamp) { + long id = lastBase0AuctionId() + GeneratorConfig.FIRST_AUCTION_ID; + + long seller; + // Here P(auction will be for a hot seller) = 1 - 1/hotSellersRatio. + if (random.nextInt(config.configuration.hotSellersRatio) > 0) { + // Choose the first person in the batch of last HOT_SELLER_RATIO people. + seller = (lastBase0PersonId() / HOT_SELLER_RATIO) * HOT_SELLER_RATIO; + } else { + seller = nextBase0PersonId(random); + } + seller += GeneratorConfig.FIRST_PERSON_ID; + + long category = GeneratorConfig.FIRST_CATEGORY_ID + random.nextInt(NUM_CATEGORIES); + long initialBid = nextPrice(random); + long dateTime = timestamp; + long expires = timestamp + nextAuctionLengthMs(random, timestamp); + String name = nextString(random, 20); + String desc = nextString(random, 100); + long reserve = initialBid + nextPrice(random); + int currentSize = 8 + name.length() + desc.length() + 8 + 8 + 8 + 8 + 8; + String extra = nextExtra(random, currentSize, config.configuration.avgAuctionByteSize); + return new Auction(id, name, desc, initialBid, reserve, dateTime, expires, seller, category, + extra); + } + + /** + * Generate and return a random bid with next available id. + */ + private Bid nextBid(Random random, long timestamp) { + long auction; + // Here P(bid will be for a hot auction) = 1 - 1/hotAuctionRatio. + if (random.nextInt(config.configuration.hotAuctionRatio) > 0) { + // Choose the first auction in the batch of last HOT_AUCTION_RATIO auctions. + auction = (lastBase0AuctionId() / HOT_AUCTION_RATIO) * HOT_AUCTION_RATIO; + } else { + auction = nextBase0AuctionId(random); + } + auction += GeneratorConfig.FIRST_AUCTION_ID; + + long bidder; + // Here P(bid will be by a hot bidder) = 1 - 1/hotBiddersRatio + if (random.nextInt(config.configuration.hotBiddersRatio) > 0) { + // Choose the second person (so hot bidders and hot sellers don't collide) in the batch of + // last HOT_BIDDER_RATIO people. + bidder = (lastBase0PersonId() / HOT_BIDDER_RATIO) * HOT_BIDDER_RATIO + 1; + } else { + bidder = nextBase0PersonId(random); + } + bidder += GeneratorConfig.FIRST_PERSON_ID; + + long price = nextPrice(random); + int currentSize = 8 + 8 + 8 + 8; + String extra = nextExtra(random, currentSize, config.configuration.avgBidByteSize); + return new Bid(auction, bidder, price, timestamp, extra); + } + + @Override + public boolean hasNext() { + return numEvents < config.maxEvents; + } + + /** + * Return the next event. The outer timestamp is in wallclock time and corresponds to + * when the event should fire. The inner timestamp is in event-time and represents the + * time the event is purported to have taken place in the simulation. + */ + public NextEvent nextEvent() { + if (wallclockBaseTime < 0) { + wallclockBaseTime = System.currentTimeMillis(); + } + // When, in event time, we should generate the event. Monotonic. + long eventTimestamp = + config.timestampAndInterEventDelayUsForEvent(config.nextEventNumber(numEvents)).getKey(); + // When, in event time, the event should say it was generated. Depending on outOfOrderGroupSize + // may have local jitter. + long adjustedEventTimestamp = + config.timestampAndInterEventDelayUsForEvent(config.nextAdjustedEventNumber(numEvents)) + .getKey(); + // The minimum of this and all future adjusted event timestamps. Accounts for jitter in + // the event timestamp. + long watermark = + config.timestampAndInterEventDelayUsForEvent(config.nextEventNumberForWatermark(numEvents)) + .getKey(); + // When, in wallclock time, we should emit the event. + long wallclockTimestamp = wallclockBaseTime + (eventTimestamp - getCurrentConfig().baseTime); + + // Seed the random number generator with the next 'event id'. + Random random = new Random(getNextEventId()); + long rem = getNextEventId() % GeneratorConfig.PROPORTION_DENOMINATOR; + + Event event; + if (rem < GeneratorConfig.PERSON_PROPORTION) { + event = new Event(nextPerson(random, adjustedEventTimestamp)); + } else if (rem < GeneratorConfig.PERSON_PROPORTION + GeneratorConfig.AUCTION_PROPORTION) { + event = new Event(nextAuction(random, adjustedEventTimestamp)); + } else { + event = new Event(nextBid(random, adjustedEventTimestamp)); + } + + numEvents++; + return new NextEvent(wallclockTimestamp, adjustedEventTimestamp, event, watermark); + } + + @Override + public TimestampedValue next() { + NextEvent next = nextEvent(); + return TimestampedValue.of(next.event, new Instant(next.eventTimestamp)); + } + + @Override + public void remove() { + throw new UnsupportedOperationException(); + } + + /** + * Return how many microseconds till we emit the next event. + */ + public long currentInterEventDelayUs() { + return config.timestampAndInterEventDelayUsForEvent(config.nextEventNumber(numEvents)) + .getValue(); + } + + /** + * Return an estimate of fraction of output consumed. + */ + public double getFractionConsumed() { + return (double) numEvents / config.maxEvents; + } + + @Override + public String toString() { + return String.format("Generator{config:%s; numEvents:%d; wallclockBaseTime:%d}", config, + numEvents, wallclockBaseTime); + } +} diff --git a/integration/java/src/main/java/org/apache/beam/integration/nexmark/GeneratorConfig.java b/integration/java/src/main/java/org/apache/beam/integration/nexmark/GeneratorConfig.java new file mode 100644 index 0000000000000..59aaf492be699 --- /dev/null +++ b/integration/java/src/main/java/org/apache/beam/integration/nexmark/GeneratorConfig.java @@ -0,0 +1,295 @@ +/* + * 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.integration.nexmark; + +import org.apache.beam.sdk.values.KV; + +import java.io.Serializable; +import java.util.ArrayList; +import java.util.List; + +/** + * Parameters controlling how {@link Generator} synthesizes {@link Event} elements. + */ +class GeneratorConfig implements Serializable { + /** + * We start the ids at specific values to help ensure the queries find a match even on + * small synthesized dataset sizes. + */ + public static final long FIRST_AUCTION_ID = 1000L; + public static final long FIRST_PERSON_ID = 1000L; + public static final long FIRST_CATEGORY_ID = 10L; + + /** + * Proportions of people/auctions/bids to synthesize. + */ + public static final int PERSON_PROPORTION = 1; + public static final int AUCTION_PROPORTION = 3; + public static final int BID_PROPORTION = 46; + public static final int PROPORTION_DENOMINATOR = + PERSON_PROPORTION + AUCTION_PROPORTION + BID_PROPORTION; + + /** + * Environment options. + */ + public final NexmarkConfiguration configuration; + + /** + * Delay between events, in microseconds. If the array has more than one entry then + * the rate is changed every {@link #stepLengthSec}, and wraps around. + */ + public final long[] interEventDelayUs; + + /** + * Delay before changing the current inter-event delay. + */ + public final long stepLengthSec; + + /** + * Time for first event (ms since epoch). + */ + public final long baseTime; + + /** + * Event id of first event to be generated. Event ids are unique over all generators, and + * are used as a seed to generate each event's data. + */ + public final long firstEventId; + + /** + * Maximum number of events to generate. + */ + public final long maxEvents; + + /** + * First event number. Generators running in parallel time may share the same event number, + * and the event number is used to determine the event timestamp. + */ + public final long firstEventNumber; + + /** + * True period of epoch in milliseconds. Derived from above. + * (Ie time to run through cycle for all interEventDelayUs entries). + */ + public final long epochPeriodMs; + + /** + * Number of events per epoch. Derived from above. + * (Ie number of events to run through cycle for all interEventDelayUs entries). + */ + public final long eventsPerEpoch; + + public GeneratorConfig( + NexmarkConfiguration configuration, long baseTime, long firstEventId, + long maxEventsOrZero, long firstEventNumber) { + this.configuration = configuration; + this.interEventDelayUs = configuration.rateShape.interEventDelayUs( + configuration.firstEventRate, configuration.nextEventRate, + configuration.rateUnit, configuration.numEventGenerators); + this.stepLengthSec = configuration.rateShape.stepLengthSec(configuration.ratePeriodSec); + this.baseTime = baseTime; + this.firstEventId = firstEventId; + if (maxEventsOrZero == 0) { + // Scale maximum down to avoid overflow in getEstimatedSizeBytes. + this.maxEvents = + Long.MAX_VALUE / (PROPORTION_DENOMINATOR + * Math.max( + Math.max(configuration.avgPersonByteSize, configuration.avgAuctionByteSize), + configuration.avgBidByteSize)); + } else { + this.maxEvents = maxEventsOrZero; + } + this.firstEventNumber = firstEventNumber; + + long eventsPerEpoch = 0; + long epochPeriodMs = 0; + if (interEventDelayUs.length > 1) { + for (int i = 0; i < interEventDelayUs.length; i++) { + long numEventsForThisCycle = (stepLengthSec * 1_000_000L) / interEventDelayUs[i]; + eventsPerEpoch += numEventsForThisCycle; + epochPeriodMs += (numEventsForThisCycle * interEventDelayUs[i]) / 1000L; + } + } + this.eventsPerEpoch = eventsPerEpoch; + this.epochPeriodMs = epochPeriodMs; + } + + /** + * Return a clone of this config. + */ + @Override + public GeneratorConfig clone() { + return new GeneratorConfig(configuration, baseTime, firstEventId, maxEvents, firstEventNumber); + } + + /** + * Return clone of this config except with given parameters. + */ + public GeneratorConfig cloneWith(long firstEventId, long maxEvents, long firstEventNumber) { + return new GeneratorConfig(configuration, baseTime, firstEventId, maxEvents, firstEventNumber); + } + + /** + * Split this config into {@code n} sub-configs with roughly equal number of + * possible events, but distinct value spaces. The generators will run on parallel timelines. + * This config should no longer be used. + */ + public List split(int n) { + List results = new ArrayList<>(); + if (n == 1) { + // No split required. + results.add(this); + } else { + long subMaxEvents = maxEvents / n; + long subFirstEventId = firstEventId; + for (int i = 0; i < n; i++) { + if (i == n - 1) { + // Don't loose any events to round-down. + subMaxEvents = maxEvents - subMaxEvents * (n - 1); + } + results.add(cloneWith(subFirstEventId, subMaxEvents, firstEventNumber)); + subFirstEventId += subMaxEvents; + } + } + return results; + } + + /** + * Return an estimate of the bytes needed by {@code numEvents}. + */ + public long estimatedBytesForEvents(long numEvents) { + long numPersons = + (numEvents * GeneratorConfig.PERSON_PROPORTION) / GeneratorConfig.PROPORTION_DENOMINATOR; + long numAuctions = (numEvents * AUCTION_PROPORTION) / PROPORTION_DENOMINATOR; + long numBids = (numEvents * BID_PROPORTION) / PROPORTION_DENOMINATOR; + return numPersons * configuration.avgPersonByteSize + + numAuctions * configuration.avgAuctionByteSize + + numBids * configuration.avgBidByteSize; + } + + /** + * Return an estimate of the byte-size of all events a generator for this config would yield. + */ + public long getEstimatedSizeBytes() { + return estimatedBytesForEvents(maxEvents); + } + + /** + * Return the first 'event id' which could be generated from this config. Though events don't + * have ids we can simulate them to help bookkeeping. + */ + public long getStartEventId() { + return firstEventId + firstEventNumber; + } + + /** + * Return one past the last 'event id' which could be generated from this config. + */ + public long getStopEventId() { + return firstEventId + firstEventNumber + maxEvents; + } + + /** + * Return the next event number for a generator which has so far emitted {@code numEvents}. + */ + public long nextEventNumber(long numEvents) { + return firstEventNumber + numEvents; + } + + /** + * Return the next event number for a generator which has so far emitted {@code numEvents}, + * but adjusted to account for {@code outOfOrderGroupSize}. + */ + public long nextAdjustedEventNumber(long numEvents) { + long n = configuration.outOfOrderGroupSize; + long eventNumber = nextEventNumber(numEvents); + long base = (eventNumber / n) * n; + long offset = (eventNumber * 953) % n; + return base + offset; + } + + /** + * Return the event number who's event time will be a suitable watermark for + * a generator which has so far emitted {@code numEvents}. + */ + public long nextEventNumberForWatermark(long numEvents) { + long n = configuration.outOfOrderGroupSize; + long eventNumber = nextEventNumber(numEvents); + return (eventNumber / n) * n; + } + + /** + * What timestamp should the event with {@code eventNumber} have for this generator? And + * what inter-event delay (in microseconds) is current? + */ + public KV timestampAndInterEventDelayUsForEvent(long eventNumber) { + if (interEventDelayUs.length == 1) { + long timestamp = baseTime + (eventNumber * interEventDelayUs[0]) / 1000L; + return KV.of(timestamp, interEventDelayUs[0]); + } + + long epoch = eventNumber / eventsPerEpoch; + long n = eventNumber % eventsPerEpoch; + long offsetInEpochMs = 0; + for (int i = 0; i < interEventDelayUs.length; i++) { + long numEventsForThisCycle = (stepLengthSec * 1_000_000L) / interEventDelayUs[i]; + if (n < numEventsForThisCycle) { + long offsetInCycleUs = n * interEventDelayUs[i]; + long timestamp = + baseTime + epoch * epochPeriodMs + offsetInEpochMs + (offsetInCycleUs / 1000L); + return KV.of(timestamp, interEventDelayUs[i]); + } + n -= numEventsForThisCycle; + offsetInEpochMs += (numEventsForThisCycle * interEventDelayUs[i]) / 1000L; + } + throw new RuntimeException("internal eventsPerEpoch incorrect"); // can't reach + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder(); + sb.append("GeneratorConfig"); + sb.append("{configuration:"); + sb.append(configuration.toString()); + sb.append(";interEventDelayUs=["); + for (int i = 0; i < interEventDelayUs.length; i++) { + if (i > 0) { + sb.append(","); + } + sb.append(interEventDelayUs[i]); + } + sb.append("]"); + sb.append(";stepLengthSec:"); + sb.append(stepLengthSec); + sb.append(";baseTime:"); + sb.append(baseTime); + sb.append(";firstEventId:"); + sb.append(firstEventId); + sb.append(";maxEvents:"); + sb.append(maxEvents); + sb.append(";firstEventNumber:"); + sb.append(firstEventNumber); + sb.append(";epochPeriodMs:"); + sb.append(epochPeriodMs); + sb.append(";eventsPerEpoch:"); + sb.append(eventsPerEpoch); + sb.append("}"); + return sb.toString(); + } +} diff --git a/integration/java/src/main/java/org/apache/beam/integration/nexmark/IdNameReserve.java b/integration/java/src/main/java/org/apache/beam/integration/nexmark/IdNameReserve.java new file mode 100644 index 0000000000000..c72b76aa63f73 --- /dev/null +++ b/integration/java/src/main/java/org/apache/beam/integration/nexmark/IdNameReserve.java @@ -0,0 +1,100 @@ +/* + * 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.integration.nexmark; + +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.StringUtf8Coder; +import org.apache.beam.sdk.coders.VarLongCoder; + +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.core.JsonProcessingException; + +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.io.Serializable; + +/** + * Result type of {@link Query8}. + */ +public class IdNameReserve implements KnownSize, Serializable { + private static final Coder LONG_CODER = VarLongCoder.of(); + private static final Coder STRING_CODER = StringUtf8Coder.of(); + + public static final Coder CODER = new AtomicCoder() { + @Override + public void encode(IdNameReserve value, OutputStream outStream, + Coder.Context context) + throws CoderException, IOException { + LONG_CODER.encode(value.id, outStream, Context.NESTED); + STRING_CODER.encode(value.name, outStream, Context.NESTED); + LONG_CODER.encode(value.reserve, outStream, Context.NESTED); + } + + @Override + public IdNameReserve decode( + InputStream inStream, Coder.Context context) + throws CoderException, IOException { + long id = LONG_CODER.decode(inStream, Context.NESTED); + String name = STRING_CODER.decode(inStream, Context.NESTED); + long reserve = LONG_CODER.decode(inStream, Context.NESTED); + return new IdNameReserve(id, name, reserve); + } + }; + + @JsonProperty + public final long id; + + @JsonProperty + public final String name; + + /** Reserve price in cents. */ + @JsonProperty + public final long reserve; + + // For Avro only. + @SuppressWarnings("unused") + private IdNameReserve() { + id = 0; + name = null; + reserve = 0; + } + + public IdNameReserve(long id, String name, long reserve) { + this.id = id; + this.name = name; + this.reserve = reserve; + } + + @Override + public long sizeInBytes() { + return 8 + name.length() + 1 + 8; + } + + @Override + public String toString() { + try { + return NexmarkUtils.MAPPER.writeValueAsString(this); + } catch (JsonProcessingException e) { + throw new RuntimeException(e); + } + } +} diff --git a/integration/java/src/main/java/org/apache/beam/integration/nexmark/KnownSize.java b/integration/java/src/main/java/org/apache/beam/integration/nexmark/KnownSize.java new file mode 100644 index 0000000000000..394b6db43eeda --- /dev/null +++ b/integration/java/src/main/java/org/apache/beam/integration/nexmark/KnownSize.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.integration.nexmark; + +/** + * Interface for elements which can quickly estimate their encoded byte size. + */ +public interface KnownSize { + long sizeInBytes(); +} + diff --git a/integration/java/src/main/java/org/apache/beam/integration/nexmark/Monitor.java b/integration/java/src/main/java/org/apache/beam/integration/nexmark/Monitor.java new file mode 100644 index 0000000000000..6874578937086 --- /dev/null +++ b/integration/java/src/main/java/org/apache/beam/integration/nexmark/Monitor.java @@ -0,0 +1,102 @@ +/* + * 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.integration.nexmark; + +import org.apache.beam.sdk.transforms.Aggregator; +import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.transforms.Max.MaxLongFn; +import org.apache.beam.sdk.transforms.Min.MinLongFn; +import org.apache.beam.sdk.transforms.PTransform; +import org.apache.beam.sdk.transforms.ParDo; +import org.apache.beam.sdk.transforms.Sum.SumLongFn; +import org.apache.beam.sdk.values.PCollection; + +import java.io.Serializable; + +/** + * A monitor of elements with support for later retrieving their aggregators. + * + * @param Type of element we are monitoring. + */ +public class Monitor implements Serializable { + private class MonitorDoFn extends DoFn { + public final Aggregator elementCounter = + createAggregator(counterNamePrefix + "_elements", new SumLongFn()); + public final Aggregator bytesCounter = + createAggregator(counterNamePrefix + "_bytes", new SumLongFn()); + public final Aggregator startTime = + createAggregator(counterNamePrefix + "_startTime", new MinLongFn()); + public final Aggregator endTime = + createAggregator(counterNamePrefix + "_endTime", new MaxLongFn()); + public final Aggregator startTimestamp = + createAggregator("startTimestamp", new MinLongFn()); + public final Aggregator endTimestamp = + createAggregator("endTimestamp", new MaxLongFn()); + + @Override + public void processElement(ProcessContext c) { + elementCounter.addValue(1L); + bytesCounter.addValue(c.element().sizeInBytes()); + long now = System.currentTimeMillis(); + startTime.addValue(now); + endTime.addValue(now); + startTimestamp.addValue(c.timestamp().getMillis()); + endTimestamp.addValue(c.timestamp().getMillis()); + c.output(c.element()); + } + } + + final MonitorDoFn doFn; + final PTransform, PCollection> transform; + private String counterNamePrefix; + + public Monitor(String name, String counterNamePrefix) { + this.counterNamePrefix = counterNamePrefix; + doFn = new MonitorDoFn(); + transform = ParDo.named(name + ".Monitor").of(doFn); + } + + public PTransform, PCollection> getTransform() { + return transform; + } + + public Aggregator getElementCounter() { + return doFn.elementCounter; + } + + public Aggregator getBytesCounter() { + return doFn.bytesCounter; + } + + public Aggregator getStartTime() { + return doFn.startTime; + } + + public Aggregator getEndTime() { + return doFn.endTime; + } + + public Aggregator getStartTimestamp() { + return doFn.startTimestamp; + } + + public Aggregator getEndTimestamp() { + return doFn.endTimestamp; + } +} diff --git a/integration/java/src/main/java/org/apache/beam/integration/nexmark/NameCityStateId.java b/integration/java/src/main/java/org/apache/beam/integration/nexmark/NameCityStateId.java new file mode 100644 index 0000000000000..2753d2ecf9356 --- /dev/null +++ b/integration/java/src/main/java/org/apache/beam/integration/nexmark/NameCityStateId.java @@ -0,0 +1,106 @@ +/* + * 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.integration.nexmark; + +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.StringUtf8Coder; +import org.apache.beam.sdk.coders.VarLongCoder; + +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.core.JsonProcessingException; + +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.io.Serializable; + +/** + * Result of {@link Query3}. + */ +public class NameCityStateId implements KnownSize, Serializable { + private static final Coder LONG_CODER = VarLongCoder.of(); + private static final Coder STRING_CODER = StringUtf8Coder.of(); + + public static final Coder CODER = new AtomicCoder() { + @Override + public void encode(NameCityStateId value, OutputStream outStream, + Coder.Context context) + throws CoderException, IOException { + STRING_CODER.encode(value.name, outStream, Context.NESTED); + STRING_CODER.encode(value.city, outStream, Context.NESTED); + STRING_CODER.encode(value.state, outStream, Context.NESTED); + LONG_CODER.encode(value.id, outStream, Context.NESTED); + } + + @Override + public NameCityStateId decode( + InputStream inStream, Coder.Context context) + throws CoderException, IOException { + String name = STRING_CODER.decode(inStream, Context.NESTED); + String city = STRING_CODER.decode(inStream, Context.NESTED); + String state = STRING_CODER.decode(inStream, Context.NESTED); + long id = LONG_CODER.decode(inStream, Context.NESTED); + return new NameCityStateId(name, city, state, id); + } + }; + + @JsonProperty + public final String name; + + @JsonProperty + public final String city; + + @JsonProperty + public final String state; + + @JsonProperty + public final long id; + + // For Avro only. + @SuppressWarnings("unused") + private NameCityStateId() { + name = null; + city = null; + state = null; + id = 0; + } + + public NameCityStateId(String name, String city, String state, long id) { + this.name = name; + this.city = city; + this.state = state; + this.id = id; + } + + @Override + public long sizeInBytes() { + return name.length() + 1 + city.length() + 1 + state.length() + 1 + 8; + } + + @Override + public String toString() { + try { + return NexmarkUtils.MAPPER.writeValueAsString(this); + } catch (JsonProcessingException e) { + throw new RuntimeException(e); + } + } +} diff --git a/integration/java/src/main/java/org/apache/beam/integration/nexmark/NexmarkConfiguration.java b/integration/java/src/main/java/org/apache/beam/integration/nexmark/NexmarkConfiguration.java new file mode 100644 index 0000000000000..2292ba527ac7f --- /dev/null +++ b/integration/java/src/main/java/org/apache/beam/integration/nexmark/NexmarkConfiguration.java @@ -0,0 +1,662 @@ +/* + * 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.integration.nexmark; + +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.core.JsonProcessingException; + +import java.io.IOException; +import java.io.Serializable; +import java.util.Objects; + + +/** + * Configuration controlling how a query is run. May be supplied by command line or + * programmatically. We only capture properties which may influence the resulting + * pipeline performance, as captured by {@link NexmarkPerf}. + */ +class NexmarkConfiguration implements Serializable { + public static final NexmarkConfiguration DEFAULT = new NexmarkConfiguration(); + + /** If {@literal true}, include additional debugging and monitoring stats. */ + @JsonProperty + public boolean debug = true; + + /** Which query to run, in [0,9]. */ + @JsonProperty + public int query = 0; + + /** Where events come from. */ + @JsonProperty + public NexmarkUtils.SourceType sourceType = NexmarkUtils.SourceType.DIRECT; + + /** Where results go to. */ + @JsonProperty + public NexmarkUtils.SinkType sinkType = NexmarkUtils.SinkType.DEVNULL; + + /** + * Control whether pub/sub publishing is done in a stand-alone pipeline or is integrated + * into the overall query pipeline. + */ + @JsonProperty + public NexmarkUtils.PubSubMode pubSubMode = NexmarkUtils.PubSubMode.COMBINED; + + /** + * Number of events to generate. If zero, generate as many as possible without overflowing + * internal counters etc. + */ + @JsonProperty + public long numEvents = 100000; + + /** + * Number of event generators to use. Each generates events in its own timeline. + */ + @JsonProperty + public int numEventGenerators = 100; + + /** + * Shape of event rate curve. + */ + @JsonProperty + public NexmarkUtils.RateShape rateShape = NexmarkUtils.RateShape.SINE; + + /** + * Initial overall event rate (in {@link #rateUnit}). + */ + @JsonProperty + public int firstEventRate = 10000; + + /** + * Next overall event rate (in {@link #rateUnit}). + */ + @JsonProperty + public int nextEventRate = 10000; + + /** + * Unit for rates. + */ + @JsonProperty + public NexmarkUtils.RateUnit rateUnit = NexmarkUtils.RateUnit.PER_SECOND; + + /** + * Overall period of rate shape, in seconds. + */ + @JsonProperty + public int ratePeriodSec = 600; + + /** + * Time in seconds to preload the subscription with data, at the initial input rate of the + * pipeline. + */ + @JsonProperty + public int preloadSeconds = 0; + + /** + * If true, and in streaming mode, generate events only when they are due according to their + * timestamp. + */ + @JsonProperty + public boolean isRateLimited = false; + + /** + * If true, use wallclock time as event time. Otherwise, use a deterministic + * time in the past so that multiple runs will see exactly the same event streams + * and should thus have exactly the same results. + */ + @JsonProperty + public boolean useWallclockEventTime = false; + + /** Average idealized size of a 'new person' event, in bytes. */ + @JsonProperty + public int avgPersonByteSize = 200; + + /** Average idealized size of a 'new auction' event, in bytes. */ + @JsonProperty + public int avgAuctionByteSize = 500; + + /** Average idealized size of a 'bid' event, in bytes. */ + @JsonProperty + public int avgBidByteSize = 100; + + /** Ratio of bids to 'hot' auctions compared to all other auctions. */ + @JsonProperty + public int hotAuctionRatio = 1; + + /** Ratio of auctions for 'hot' sellers compared to all other people. */ + @JsonProperty + public int hotSellersRatio = 1; + + /** Ratio of bids for 'hot' bidders compared to all other people. */ + @JsonProperty + public int hotBiddersRatio = 1; + + /** Window size, in seconds, for queries 3, 5, 7 and 8. */ + @JsonProperty + public long windowSizeSec = 10; + + /** Sliding window period, in seconds, for query 5. */ + @JsonProperty + public long windowPeriodSec = 5; + + /** Number of seconds to hold back events according to their reported timestamp. */ + @JsonProperty + public long watermarkHoldbackSec = 0; + + /** Average number of auction which should be inflight at any time, per generator. */ + @JsonProperty + public int numInFlightAuctions = 100; + + /** Maximum number of people to consider as active for placing auctions or bids. */ + @JsonProperty + public int numActivePeople = 1000; + + /** Coder strategy to follow. */ + @JsonProperty + public NexmarkUtils.CoderStrategy coderStrategy = NexmarkUtils.CoderStrategy.HAND; + + /** + * Delay, in milliseconds, for each event. This will peg one core for this number + * of milliseconds to simulate CPU-bound computation. + */ + @JsonProperty + public long cpuDelayMs = 0; + + /** + * Extra data, in bytes, to save to persistent state for each event. This will force + * i/o all the way to durable storage to simulate an I/O-bound computation. + */ + @JsonProperty + public long diskBusyBytes = 0; + + /** + * Skip factor for query 2. We select bids for every {@code auctionSkip}'th auction. + */ + @JsonProperty + public int auctionSkip = 123; + + /** + * Fanout for queries 4 (groups by category id), 5 and 7 (find a global maximum). + */ + @JsonProperty + public int fanout = 5; + + /** + * Length of occasional delay to impose on events (in seconds). + */ + @JsonProperty + public long occasionalDelaySec = 0; + + /** + * Probability that an event will be delayed by delayS. + */ + @JsonProperty + public double probDelayedEvent = 0.0; + + /** + * Maximum size of each log file (in events). For Query10 only. + */ + @JsonProperty + public int maxLogEvents = 100_000; + + /** + * If true, use pub/sub publish time instead of event time. + */ + @JsonProperty + public boolean usePubsubPublishTime = false; + + /** + * Number of events in out-of-order groups. 1 implies no out-of-order events. 1000 implies + * every 1000 events per generator are emitted in pseudo-random order. + */ + @JsonProperty + public long outOfOrderGroupSize = 1; + + /** + * Replace any properties of this configuration which have been supplied by the command line. + */ + public void overrideFromOptions(Options options) { + if (options.getDebug() != null) { + debug = options.getDebug(); + } + if (options.getQuery() != null) { + query = options.getQuery(); + } + if (options.getSourceType() != null) { + sourceType = options.getSourceType(); + } + if (options.getSinkType() != null) { + sinkType = options.getSinkType(); + } + if (options.getPubSubMode() != null) { + pubSubMode = options.getPubSubMode(); + } + if (options.getNumEvents() != null) { + numEvents = options.getNumEvents(); + } + if (options.getNumEventGenerators() != null) { + numEventGenerators = options.getNumEventGenerators(); + } + if (options.getRateShape() != null) { + rateShape = options.getRateShape(); + } + if (options.getFirstEventRate() != null) { + firstEventRate = options.getFirstEventRate(); + } + if (options.getNextEventRate() != null) { + nextEventRate = options.getNextEventRate(); + } + if (options.getRateUnit() != null) { + rateUnit = options.getRateUnit(); + } + if (options.getRatePeriodSec() != null) { + ratePeriodSec = options.getRatePeriodSec(); + } + if (options.getPreloadSeconds() != null) { + preloadSeconds = options.getPreloadSeconds(); + } + if (options.getIsRateLimited() != null) { + isRateLimited = options.getIsRateLimited(); + } + if (options.getUseWallclockEventTime() != null) { + useWallclockEventTime = options.getUseWallclockEventTime(); + } + if (options.getAvgPersonByteSize() != null) { + avgPersonByteSize = options.getAvgPersonByteSize(); + } + if (options.getAvgAuctionByteSize() != null) { + avgAuctionByteSize = options.getAvgAuctionByteSize(); + } + if (options.getAvgBidByteSize() != null) { + avgBidByteSize = options.getAvgBidByteSize(); + } + if (options.getHotAuctionRatio() != null) { + hotAuctionRatio = options.getHotAuctionRatio(); + } + if (options.getHotSellersRatio() != null) { + hotSellersRatio = options.getHotSellersRatio(); + } + if (options.getHotBiddersRatio() != null) { + hotBiddersRatio = options.getHotBiddersRatio(); + } + if (options.getWindowSizeSec() != null) { + windowSizeSec = options.getWindowSizeSec(); + } + if (options.getWindowPeriodSec() != null) { + windowPeriodSec = options.getWindowPeriodSec(); + } + if (options.getWatermarkHoldbackSec() != null) { + watermarkHoldbackSec = options.getWatermarkHoldbackSec(); + } + if (options.getNumInFlightAuctions() != null) { + numInFlightAuctions = options.getNumInFlightAuctions(); + } + if (options.getNumActivePeople() != null) { + numActivePeople = options.getNumActivePeople(); + } + if (options.getCoderStrategy() != null) { + coderStrategy = options.getCoderStrategy(); + } + if (options.getCpuDelayMs() != null) { + cpuDelayMs = options.getCpuDelayMs(); + } + if (options.getDiskBusyBytes() != null) { + diskBusyBytes = options.getDiskBusyBytes(); + } + if (options.getAuctionSkip() != null) { + auctionSkip = options.getAuctionSkip(); + } + if (options.getFanout() != null) { + fanout = options.getFanout(); + } + if (options.getOccasionalDelaySec() != null) { + occasionalDelaySec = options.getOccasionalDelaySec(); + } + if (options.getProbDelayedEvent() != null) { + probDelayedEvent = options.getProbDelayedEvent(); + } + if (options.getMaxLogEvents() != null) { + maxLogEvents = options.getMaxLogEvents(); + } + if (options.getUsePubsubPublishTime() != null) { + usePubsubPublishTime = options.getUsePubsubPublishTime(); + } + if (options.getOutOfOrderGroupSize() != null) { + outOfOrderGroupSize = options.getOutOfOrderGroupSize(); + } + } + + /** + * Return clone of configuration with given label. + */ + @Override + public NexmarkConfiguration clone() { + NexmarkConfiguration result = new NexmarkConfiguration(); + result.debug = debug; + result.query = query; + result.sourceType = sourceType; + result.sinkType = sinkType; + result.pubSubMode = pubSubMode; + result.numEvents = numEvents; + result.numEventGenerators = numEventGenerators; + result.rateShape = rateShape; + result.firstEventRate = firstEventRate; + result.nextEventRate = nextEventRate; + result.rateUnit = rateUnit; + result.ratePeriodSec = ratePeriodSec; + result.preloadSeconds = preloadSeconds; + result.isRateLimited = isRateLimited; + result.useWallclockEventTime = useWallclockEventTime; + result.avgPersonByteSize = avgPersonByteSize; + result.avgAuctionByteSize = avgAuctionByteSize; + result.avgBidByteSize = avgBidByteSize; + result.hotAuctionRatio = hotAuctionRatio; + result.hotSellersRatio = hotSellersRatio; + result.hotBiddersRatio = hotBiddersRatio; + result.windowSizeSec = windowSizeSec; + result.windowPeriodSec = windowPeriodSec; + result.watermarkHoldbackSec = watermarkHoldbackSec; + result.numInFlightAuctions = numInFlightAuctions; + result.numActivePeople = numActivePeople; + result.coderStrategy = coderStrategy; + result.cpuDelayMs = cpuDelayMs; + result.diskBusyBytes = diskBusyBytes; + result.auctionSkip = auctionSkip; + result.fanout = fanout; + result.occasionalDelaySec = occasionalDelaySec; + result.probDelayedEvent = probDelayedEvent; + result.maxLogEvents = maxLogEvents; + result.usePubsubPublishTime = usePubsubPublishTime; + result.outOfOrderGroupSize = outOfOrderGroupSize; + return result; + } + + /** + * Return short description of configuration (suitable for use in logging). We only render + * the core fields plus those which do not have default values. + */ + public String toShortString() { + StringBuilder sb = new StringBuilder(); + sb.append(String.format("query:%d", query)); + if (debug != DEFAULT.debug) { + sb.append(String.format("; debug:%s", debug)); + } + if (sourceType != DEFAULT.sourceType) { + sb.append(String.format("; sourceType:%s", sourceType)); + } + if (sinkType != DEFAULT.sinkType) { + sb.append(String.format("; sinkType:%s", sinkType)); + } + if (pubSubMode != DEFAULT.pubSubMode) { + sb.append(String.format("; pubSubMode:%s", pubSubMode)); + } + if (numEvents != DEFAULT.numEvents) { + sb.append(String.format("; numEvents:%d", numEvents)); + } + if (numEventGenerators != DEFAULT.numEventGenerators) { + sb.append(String.format("; numEventGenerators:%d", numEventGenerators)); + } + if (rateShape != DEFAULT.rateShape) { + sb.append(String.format("; rateShape:%s", rateShape)); + } + if (firstEventRate != DEFAULT.firstEventRate || nextEventRate != DEFAULT.nextEventRate) { + sb.append(String.format("; firstEventRate:%d", firstEventRate)); + sb.append(String.format("; nextEventRate:%d", nextEventRate)); + } + if (rateUnit != DEFAULT.rateUnit) { + sb.append(String.format("; rateUnit:%s", rateUnit)); + } + if (ratePeriodSec != DEFAULT.ratePeriodSec) { + sb.append(String.format("; ratePeriodSec:%d", ratePeriodSec)); + } + if (preloadSeconds != DEFAULT.preloadSeconds) { + sb.append(String.format("; preloadSeconds:%d", preloadSeconds)); + } + if (isRateLimited != DEFAULT.isRateLimited) { + sb.append(String.format("; isRateLimited:%s", isRateLimited)); + } + if (useWallclockEventTime != DEFAULT.useWallclockEventTime) { + sb.append(String.format("; useWallclockEventTime:%s", useWallclockEventTime)); + } + if (avgPersonByteSize != DEFAULT.avgPersonByteSize) { + sb.append(String.format("; avgPersonByteSize:%d", avgPersonByteSize)); + } + if (avgAuctionByteSize != DEFAULT.avgAuctionByteSize) { + sb.append(String.format("; avgAuctionByteSize:%d", avgAuctionByteSize)); + } + if (avgBidByteSize != DEFAULT.avgBidByteSize) { + sb.append(String.format("; avgBidByteSize:%d", avgBidByteSize)); + } + if (hotAuctionRatio != DEFAULT.hotAuctionRatio) { + sb.append(String.format("; hotAuctionRatio:%d", hotAuctionRatio)); + } + if (hotSellersRatio != DEFAULT.hotSellersRatio) { + sb.append(String.format("; hotSellersRatio:%d", hotSellersRatio)); + } + if (hotBiddersRatio != DEFAULT.hotBiddersRatio) { + sb.append(String.format("; hotBiddersRatio:%d", hotBiddersRatio)); + } + if (windowSizeSec != DEFAULT.windowSizeSec) { + sb.append(String.format("; windowSizeSec:%d", windowSizeSec)); + } + if (windowPeriodSec != DEFAULT.windowPeriodSec) { + sb.append(String.format("; windowPeriodSec:%d", windowPeriodSec)); + } + if (watermarkHoldbackSec != DEFAULT.watermarkHoldbackSec) { + sb.append(String.format("; watermarkHoldbackSec:%d", watermarkHoldbackSec)); + } + if (numInFlightAuctions != DEFAULT.numInFlightAuctions) { + sb.append(String.format("; numInFlightAuctions:%d", numInFlightAuctions)); + } + if (numActivePeople != DEFAULT.numActivePeople) { + sb.append(String.format("; numActivePeople:%d", numActivePeople)); + } + if (coderStrategy != DEFAULT.coderStrategy) { + sb.append(String.format("; coderStrategy:%s", coderStrategy)); + } + if (cpuDelayMs != DEFAULT.cpuDelayMs) { + sb.append(String.format("; cpuSlowdownMs:%d", cpuDelayMs)); + } + if (diskBusyBytes != DEFAULT.diskBusyBytes) { + sb.append(String.format("; diskBuysBytes:%d", diskBusyBytes)); + } + if (auctionSkip != DEFAULT.auctionSkip) { + sb.append(String.format("; auctionSkip:%d", auctionSkip)); + } + if (fanout != DEFAULT.fanout) { + sb.append(String.format("; fanout:%d", fanout)); + } + if (occasionalDelaySec != DEFAULT.occasionalDelaySec) { + sb.append(String.format("; occasionalDelaySec:%d", occasionalDelaySec)); + } + if (probDelayedEvent != DEFAULT.probDelayedEvent) { + sb.append(String.format("; probDelayedEvent:%f", probDelayedEvent)); + } + if (maxLogEvents != DEFAULT.maxLogEvents) { + sb.append(String.format("; maxLogEvents:%d", maxLogEvents)); + } + if (usePubsubPublishTime != DEFAULT.usePubsubPublishTime) { + sb.append(String.format("; usePubsubPublishTime:%s", usePubsubPublishTime)); + } + if (outOfOrderGroupSize != DEFAULT.outOfOrderGroupSize) { + sb.append(String.format("; outOfOrderGroupSize:%d", outOfOrderGroupSize)); + } + return sb.toString(); + } + + /** + * Return full description as a string. + */ + @Override + public String toString() { + try { + return NexmarkUtils.MAPPER.writeValueAsString(this); + } catch (JsonProcessingException e) { + throw new RuntimeException(e); + } + } + + /** + * Parse an object from {@code string}. + * + * @throws IOException + */ + public static NexmarkConfiguration fromString(String string) { + try { + return NexmarkUtils.MAPPER.readValue(string, NexmarkConfiguration.class); + } catch (IOException e) { + throw new RuntimeException("Unable to parse nexmark configuration: ", e); + } + } + + @Override + public int hashCode() { + return Objects.hash(debug, query, sourceType, sinkType, pubSubMode, + numEvents, numEventGenerators, rateShape, firstEventRate, nextEventRate, rateUnit, + ratePeriodSec, preloadSeconds, isRateLimited, useWallclockEventTime, avgPersonByteSize, + avgAuctionByteSize, avgBidByteSize, hotAuctionRatio, hotSellersRatio, hotBiddersRatio, + windowSizeSec, windowPeriodSec, watermarkHoldbackSec, numInFlightAuctions, numActivePeople, + coderStrategy, cpuDelayMs, diskBusyBytes, auctionSkip, fanout, + occasionalDelaySec, probDelayedEvent, maxLogEvents, usePubsubPublishTime, + outOfOrderGroupSize); + } + + @Override + public boolean equals(Object obj) { + if (this == obj) { + return true; + } + if (obj == null) { + return false; + } + if (getClass() != obj.getClass()) { + return false; + } + NexmarkConfiguration other = (NexmarkConfiguration) obj; + if (debug != other.debug) { + return false; + } + if (auctionSkip != other.auctionSkip) { + return false; + } + if (avgAuctionByteSize != other.avgAuctionByteSize) { + return false; + } + if (avgBidByteSize != other.avgBidByteSize) { + return false; + } + if (avgPersonByteSize != other.avgPersonByteSize) { + return false; + } + if (coderStrategy != other.coderStrategy) { + return false; + } + if (cpuDelayMs != other.cpuDelayMs) { + return false; + } + if (diskBusyBytes != other.diskBusyBytes) { + return false; + } + if (fanout != other.fanout) { + return false; + } + if (firstEventRate != other.firstEventRate) { + return false; + } + if (hotAuctionRatio != other.hotAuctionRatio) { + return false; + } + if (hotBiddersRatio != other.hotBiddersRatio) { + return false; + } + if (hotSellersRatio != other.hotSellersRatio) { + return false; + } + if (isRateLimited != other.isRateLimited) { + return false; + } + if (maxLogEvents != other.maxLogEvents) { + return false; + } + if (nextEventRate != other.nextEventRate) { + return false; + } + if (rateUnit != other.rateUnit) { + return false; + } + if (numEventGenerators != other.numEventGenerators) { + return false; + } + if (numEvents != other.numEvents) { + return false; + } + if (numInFlightAuctions != other.numInFlightAuctions) { + return false; + } + if (numActivePeople != other.numActivePeople) { + return false; + } + if (occasionalDelaySec != other.occasionalDelaySec) { + return false; + } + if (preloadSeconds != other.preloadSeconds) { + return false; + } + if (Double.doubleToLongBits(probDelayedEvent) + != Double.doubleToLongBits(other.probDelayedEvent)) { + return false; + } + if (pubSubMode != other.pubSubMode) { + return false; + } + if (ratePeriodSec != other.ratePeriodSec) { + return false; + } + if (rateShape != other.rateShape) { + return false; + } + if (query != other.query) { + return false; + } + if (sinkType != other.sinkType) { + return false; + } + if (sourceType != other.sourceType) { + return false; + } + if (useWallclockEventTime != other.useWallclockEventTime) { + return false; + } + if (watermarkHoldbackSec != other.watermarkHoldbackSec) { + return false; + } + if (windowPeriodSec != other.windowPeriodSec) { + return false; + } + if (windowSizeSec != other.windowSizeSec) { + return false; + } + if (usePubsubPublishTime != other.usePubsubPublishTime) { + return false; + } + if (outOfOrderGroupSize != other.outOfOrderGroupSize) { + return false; + } + return true; + } +} diff --git a/integration/java/src/main/java/org/apache/beam/integration/nexmark/NexmarkDriver.java b/integration/java/src/main/java/org/apache/beam/integration/nexmark/NexmarkDriver.java new file mode 100644 index 0000000000000..dbc1ce27a5049 --- /dev/null +++ b/integration/java/src/main/java/org/apache/beam/integration/nexmark/NexmarkDriver.java @@ -0,0 +1,297 @@ +/* + * 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.integration.nexmark; + +import org.joda.time.Duration; +import org.joda.time.Instant; + +import java.io.IOException; +import java.nio.charset.StandardCharsets; +import java.nio.file.Files; +import java.nio.file.Paths; +import java.nio.file.StandardOpenOption; +import java.util.ArrayList; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; + +import javax.annotation.Nullable; + +/** + * An implementation of the 'NEXMark queries' for Google Dataflow. + * These are 11 queries over a three table schema representing on online auction system: + *

                + *
              • {@link Person} represents a person submitting an item for auction and/or making a bid + * on an auction. + *
              • {@link Auction} represents an item under auction. + *
              • {@link Bid} represents a bid for an item under auction. + *
              + * The queries exercise many aspects of streaming dataflow. + *

              + *

              We synthesize the creation of people, auctions and bids in real-time. The data is not + * particularly sensible. + *

              + *

              See + * + * http://datalab.cs.pdx.edu/niagaraST/NEXMark/ + */ +public class NexmarkDriver { + + /** + * Entry point. + */ + public void runAll(OptionT options, NexmarkRunner runner) { + Instant start = Instant.now(); + Map baseline = loadBaseline(options.getBaselineFilename()); + Map actual = new LinkedHashMap<>(); + Iterable configurations = options.getSuite().getConfigurations(options); + + boolean successful = true; + try { + // Run all the configurations. + for (NexmarkConfiguration configuration : configurations) { + NexmarkPerf perf = runner.run(configuration); + if (perf != null) { + if (perf.errors == null || perf.errors.size() > 0) { + successful = false; + } + appendPerf(options.getPerfFilename(), configuration, perf); + actual.put(configuration, perf); + // Summarize what we've run so far. + saveSummary(null, configurations, actual, baseline, start); + } + } + } finally { + if (options.getMonitorJobs()) { + // Report overall performance. + saveSummary(options.getSummaryFilename(), configurations, actual, baseline, start); + saveJavascript(options.getJavascriptFilename(), configurations, actual, baseline, start); + } + } + + if (!successful) { + System.exit(1); + } + } + + /** + * Append the pair of {@code configuration} and {@code perf} to perf file. + */ + private void appendPerf( + @Nullable String perfFilename, NexmarkConfiguration configuration, + NexmarkPerf perf) { + if (perfFilename == null) { + return; + } + List lines = new ArrayList<>(); + lines.add(""); + lines.add(String.format("# %s", Instant.now())); + lines.add(String.format("# %s", configuration.toShortString())); + lines.add(configuration.toString()); + lines.add(perf.toString()); + try { + Files.write(Paths.get(perfFilename), lines, StandardCharsets.UTF_8, StandardOpenOption.CREATE, + StandardOpenOption.APPEND); + } catch (IOException e) { + throw new RuntimeException("Unable to write perf file: ", e); + } + NexmarkUtils.console("appended results to perf file %s.", perfFilename); + } + + /** + * Load the baseline perf. + */ + @Nullable + private static Map loadBaseline( + @Nullable String baselineFilename) { + if (baselineFilename == null) { + return null; + } + Map baseline = new LinkedHashMap<>(); + List lines; + try { + lines = Files.readAllLines(Paths.get(baselineFilename), StandardCharsets.UTF_8); + } catch (IOException e) { + throw new RuntimeException("Unable to read baseline perf file: ", e); + } + for (int i = 0; i < lines.size(); i++) { + if (lines.get(i).startsWith("#") || lines.get(i).trim().isEmpty()) { + continue; + } + NexmarkConfiguration configuration = NexmarkConfiguration.fromString(lines.get(i++)); + NexmarkPerf perf = NexmarkPerf.fromString(lines.get(i)); + baseline.put(configuration, perf); + } + NexmarkUtils.console("loaded %d entries from baseline file %s.", baseline.size(), + baselineFilename); + return baseline; + } + + private static final String LINE = + "=========================================================================================="; + + /** + * Print summary of {@code actual} vs (if non-null) {@code baseline}. + * + * @throws IOException + */ + private static void saveSummary( + @Nullable String summaryFilename, + Iterable configurations, Map actual, + @Nullable Map baseline, Instant start) { + List lines = new ArrayList<>(); + + lines.add(""); + lines.add(LINE); + + lines.add( + String.format("Run started %s and ran for %s", start, new Duration(start, Instant.now()))); + lines.add(""); + + lines.add("Default configuration:"); + lines.add(NexmarkConfiguration.DEFAULT.toString()); + lines.add(""); + + lines.add("Configurations:"); + lines.add(" Conf Description"); + int conf = 0; + for (NexmarkConfiguration configuration : configurations) { + lines.add(String.format(" %04d %s", conf++, configuration.toShortString())); + NexmarkPerf actualPerf = actual.get(configuration); + if (actualPerf != null && actualPerf.jobId != null) { + lines.add(String.format(" %4s [Ran as job %s]", "", actualPerf.jobId)); + } + } + + lines.add(""); + lines.add("Performance:"); + lines.add(String.format(" %4s %12s %12s %12s %12s %12s %12s", "Conf", "Runtime(sec)", + "(Baseline)", "Events(/sec)", "(Baseline)", "Results", "(Baseline)")); + conf = 0; + for (NexmarkConfiguration configuration : configurations) { + String line = String.format(" %04d ", conf++); + NexmarkPerf actualPerf = actual.get(configuration); + if (actualPerf == null) { + line += "*** not run ***"; + } else { + NexmarkPerf baselinePerf = baseline == null ? null : baseline.get(configuration); + double runtimeSec = actualPerf.runtimeSec; + line += String.format("%12.1f ", runtimeSec); + if (baselinePerf == null) { + line += String.format("%12s ", ""); + } else { + double baselineRuntimeSec = baselinePerf.runtimeSec; + double diff = ((runtimeSec - baselineRuntimeSec) / baselineRuntimeSec) * 100.0; + line += String.format("%+11.2f%% ", diff); + } + + double eventsPerSec = actualPerf.eventsPerSec; + line += String.format("%12.1f ", eventsPerSec); + if (baselinePerf == null) { + line += String.format("%12s ", ""); + } else { + double baselineEventsPerSec = baselinePerf.eventsPerSec; + double diff = ((eventsPerSec - baselineEventsPerSec) / baselineEventsPerSec) * 100.0; + line += String.format("%+11.2f%% ", diff); + } + + long numResults = actualPerf.numResults; + line += String.format("%12d ", numResults); + if (baselinePerf == null) { + line += String.format("%12s", ""); + } else { + long baselineNumResults = baselinePerf.numResults; + long diff = numResults - baselineNumResults; + line += String.format("%+12d", diff); + } + } + lines.add(line); + + if (actualPerf != null) { + List errors = actualPerf.errors; + if (errors == null) { + errors = new ArrayList(); + errors.add("NexmarkGoogleRunner returned null errors list"); + } + for (String error : errors) { + lines.add(String.format(" %4s *** %s ***", "", error)); + } + } + } + + lines.add(LINE); + lines.add(""); + + for (String line : lines) { + System.out.println(line); + } + + if (summaryFilename != null) { + try { + Files.write(Paths.get(summaryFilename), lines, StandardCharsets.UTF_8, + StandardOpenOption.CREATE, StandardOpenOption.APPEND); + } catch (IOException e) { + throw new RuntimeException("Unable to save summary file: ", e); + } + NexmarkUtils.console("appended summary to summary file %s.", summaryFilename); + } + } + + /** + * Write all perf data and any baselines to a javascript file which can be used by + * graphing page etc. + */ + private static void saveJavascript( + @Nullable String javascriptFilename, + Iterable configurations, Map actual, + @Nullable Map baseline, Instant start) { + if (javascriptFilename == null) { + return; + } + + List lines = new ArrayList<>(); + lines.add(String.format( + "// Run started %s and ran for %s", start, new Duration(start, Instant.now()))); + lines.add("var all = ["); + + for (NexmarkConfiguration configuration : configurations) { + lines.add(" {"); + lines.add(String.format(" config: %s", configuration)); + NexmarkPerf actualPerf = actual.get(configuration); + if (actualPerf != null) { + lines.add(String.format(" ,perf: %s", actualPerf)); + } + NexmarkPerf baselinePerf = baseline == null ? null : baseline.get(configuration); + if (baselinePerf != null) { + lines.add(String.format(" ,baseline: %s", baselinePerf)); + } + lines.add(" },"); + } + + lines.add("];"); + + try { + Files.write(Paths.get(javascriptFilename), lines, StandardCharsets.UTF_8, + StandardOpenOption.CREATE, StandardOpenOption.TRUNCATE_EXISTING); + } catch (IOException e) { + throw new RuntimeException("Unable to save javascript file: ", e); + } + NexmarkUtils.console("saved javascript to file %s.", javascriptFilename); + } +} diff --git a/integration/java/src/main/java/org/apache/beam/integration/nexmark/NexmarkFlinkDriver.java b/integration/java/src/main/java/org/apache/beam/integration/nexmark/NexmarkFlinkDriver.java new file mode 100644 index 0000000000000..0029a3653bbe8 --- /dev/null +++ b/integration/java/src/main/java/org/apache/beam/integration/nexmark/NexmarkFlinkDriver.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.integration.nexmark; + +import org.apache.beam.runners.flink.FlinkPipelineOptions; +import org.apache.beam.runners.flink.FlinkPipelineRunner; +import org.apache.beam.sdk.options.PipelineOptionsFactory; + +/** + * Run NexMark queries using Beam-on-Flink runner. + */ +public class NexmarkFlinkDriver extends NexmarkDriver { + /** + * Command line flags. + */ + public interface NexmarkFlinkOptions extends Options, FlinkPipelineOptions { + } + + /** + * Entry point. + */ + public static void main(String[] args) { + // Gather command line args, baseline, configurations, etc. + NexmarkFlinkOptions options = PipelineOptionsFactory.fromArgs(args) + .withValidation() + .as(NexmarkFlinkOptions.class); + options.setRunner(FlinkPipelineRunner.class); + NexmarkFlinkRunner runner = new NexmarkFlinkRunner(options); + new NexmarkFlinkDriver().runAll(options, runner); + } +} + + diff --git a/integration/java/src/main/java/org/apache/beam/integration/nexmark/NexmarkFlinkRunner.java b/integration/java/src/main/java/org/apache/beam/integration/nexmark/NexmarkFlinkRunner.java new file mode 100644 index 0000000000000..569aef66d6388 --- /dev/null +++ b/integration/java/src/main/java/org/apache/beam/integration/nexmark/NexmarkFlinkRunner.java @@ -0,0 +1,67 @@ +/* + * 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.integration.nexmark; + +import javax.annotation.Nullable; + +/** + * Run a specific Nexmark query using the Bean-on-Flink runner. + */ +public class NexmarkFlinkRunner extends NexmarkRunner { + @Override + protected boolean isStreaming() { + return options.isStreaming(); + } + + @Override + protected int coresPerWorker() { + return 4; + } + + @Override + protected int maxNumWorkers() { + return 5; + } + + @Override + protected boolean canMonitor() { + return false; + } + + @Override + protected void invokeBuilderForPublishOnlyPipeline( + PipelineBuilder builder) { + builder.build(options); + } + + @Override + protected void waitForPublisherPreload() { + throw new UnsupportedOperationException(); + } + + @Override + @Nullable + protected NexmarkPerf monitor(NexmarkQuery query) { + return null; + } + + public NexmarkFlinkRunner(NexmarkFlinkDriver.NexmarkFlinkOptions options) { + super(options); + } +} diff --git a/integration/java/src/main/java/org/apache/beam/integration/nexmark/NexmarkGoogleDriver.java b/integration/java/src/main/java/org/apache/beam/integration/nexmark/NexmarkGoogleDriver.java new file mode 100644 index 0000000000000..253415522ebbb --- /dev/null +++ b/integration/java/src/main/java/org/apache/beam/integration/nexmark/NexmarkGoogleDriver.java @@ -0,0 +1,90 @@ +/* + * 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.integration.nexmark; + +import org.apache.beam.runners.dataflow.DataflowPipelineRunner; +import org.apache.beam.runners.dataflow.options.DataflowPipelineOptions; +import org.apache.beam.sdk.options.Description; +import org.apache.beam.sdk.options.PipelineOptionsFactory; +import javax.annotation.Nullable; + +/** + * An implementation of the 'NEXMark queries' for Google Dataflow. + * These are 11 queries over a three table schema representing on online auction system: + *

                + *
              • {@link Person} represents a person submitting an item for auction and/or making a bid + * on an auction. + *
              • {@link Auction} represents an item under auction. + *
              • {@link Bid} represents a bid for an item under auction. + *
              + * The queries exercise many aspects of streaming dataflow. + *

              + *

              We synthesize the creation of people, auctions and bids in real-time. The data is not + * particularly sensible. + *

              + *

              See + * + * http://datalab.cs.pdx.edu/niagaraST/NEXMark/ + */ +class NexmarkGoogleDriver extends NexmarkDriver { + /** + * Command line flags. + */ + public interface NexmarkGoogleOptions extends Options, DataflowPipelineOptions { + @Description("If set, cancel running pipelines after this long") + @Nullable + Long getRunningTimeMinutes(); + + void setRunningTimeMinutes(Long value); + + @Description("If set and --monitorJobs is true, check that the system watermark is never more " + + "than this far behind real time") + @Nullable + Long getMaxSystemLagSeconds(); + + void setMaxSystemLagSeconds(Long value); + + @Description("If set and --monitorJobs is true, check that the data watermark is never more " + + "than this far behind real time") + @Nullable + Long getMaxDataLagSeconds(); + + void setMaxDataLagSeconds(Long value); + + @Description("Only start validating watermarks after this many seconds") + @Nullable + Long getWatermarkValidationDelaySeconds(); + + void setWatermarkValidationDelaySeconds(Long value); + } + + /** + * Entry point. + */ + public static void main(String[] args) { + // Gather command line args, baseline, configurations, etc. + NexmarkGoogleOptions options = PipelineOptionsFactory.fromArgs(args) + .withValidation() + .as(NexmarkGoogleOptions.class); + options.setRunner(DataflowPipelineRunner.class); + NexmarkGoogleRunner runner = new NexmarkGoogleRunner(options); + new NexmarkGoogleDriver().runAll(options, runner); + } +} + diff --git a/integration/java/src/main/java/org/apache/beam/integration/nexmark/NexmarkGoogleRunner.java b/integration/java/src/main/java/org/apache/beam/integration/nexmark/NexmarkGoogleRunner.java new file mode 100644 index 0000000000000..4b735922ef76b --- /dev/null +++ b/integration/java/src/main/java/org/apache/beam/integration/nexmark/NexmarkGoogleRunner.java @@ -0,0 +1,660 @@ +/* + * 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.integration.nexmark; + +import org.apache.beam.runners.dataflow.DataflowPipelineJob; +import org.apache.beam.sdk.PipelineResult; +import org.apache.beam.sdk.runners.AggregatorRetrievalException; +import org.apache.beam.sdk.transforms.Aggregator; + +import com.google.api.services.dataflow.model.JobMetrics; +import com.google.api.services.dataflow.model.MetricUpdate; +import com.google.common.base.Preconditions; +import com.google.common.collect.Iterables; + +import org.joda.time.Duration; +import org.joda.time.Instant; + +import java.io.IOException; +import java.math.BigDecimal; +import java.util.ArrayList; +import java.util.Collection; +import java.util.List; +import java.util.concurrent.TimeUnit; +import javax.annotation.Nullable; + +/** + * Run a singe Nexmark query using a given configuration on Google Dataflow. + */ +class NexmarkGoogleRunner extends NexmarkRunner { + /** + * How long to let streaming pipeline run after all events have been generated and we've + * seen no activity. + */ + private static final Duration DONE_DELAY = Duration.standardMinutes(1); + + /** + * How long to allow no activity without warning. + */ + private static final Duration STUCK_WARNING_DELAY = Duration.standardMinutes(10); + + /** + * How long to let streaming pipeline run after we've + * seen no activity, even if all events have not been generated. + */ + private static final Duration STUCK_TERMINATE_DELAY = Duration.standardDays(3); + + /** + * Delay between perf samples. + */ + private static final Duration PERF_DELAY = Duration.standardSeconds(15); + + /** + * Minimum number of samples needed for 'stead-state' rate calculation. + */ + private static final int MIN_SAMPLES = 9; + + /** + * Minimum length of time over which to consider samples for 'steady-state' rate calculation. + */ + private static final Duration MIN_WINDOW = Duration.standardMinutes(2); + + public NexmarkGoogleRunner(NexmarkGoogleDriver.NexmarkGoogleOptions options) { + super(options); + } + + @Override + protected boolean isStreaming() { + return options.isStreaming(); + } + + @Override + protected int coresPerWorker() { + String machineType = options.getWorkerMachineType(); + if (machineType == null || machineType.isEmpty()) { + return 1; + } + String[] split = machineType.split("-"); + if (split.length != 3) { + return 1; + } + try { + return Integer.parseInt(split[2]); + } catch (NumberFormatException ex) { + return 1; + } + } + + @Override + protected int maxNumWorkers() { + return Math.max(options.getNumWorkers(), options.getMaxNumWorkers()); + } + + @Override + protected boolean canMonitor() { + return true; + } + + @Override + protected void invokeBuilderForPublishOnlyPipeline(PipelineBuilder builder) { + String jobName = options.getJobName(); + String appName = options.getAppName(); + options.setJobName("p-" + jobName); + options.setAppName("p-" + appName); + int coresPerWorker = coresPerWorker(); + int eventGeneratorWorkers = (configuration.numEventGenerators + coresPerWorker - 1) + / coresPerWorker; + options.setMaxNumWorkers(Math.min(options.getMaxNumWorkers(), eventGeneratorWorkers)); + options.setNumWorkers(Math.min(options.getNumWorkers(), eventGeneratorWorkers)); + publisherMonitor = new Monitor(queryName, "publisher"); + try { + builder.build(options); + } finally { + options.setJobName(jobName); + options.setAppName(appName); + options.setMaxNumWorkers(options.getMaxNumWorkers()); + options.setNumWorkers(options.getNumWorkers()); + } + } + + /** + * Monitor the progress of the publisher job. Return when it has been generating events for + * at least {@code configuration.preloadSeconds}. + */ + @Override + protected void waitForPublisherPreload() { + Preconditions.checkNotNull(publisherMonitor); + Preconditions.checkNotNull(publisherResult); + if (!options.getMonitorJobs()) { + return; + } + if (!(publisherResult instanceof DataflowPipelineJob)) { + return; + } + if (configuration.preloadSeconds <= 0) { + return; + } + + NexmarkUtils.console("waiting for publisher to pre-load"); + + DataflowPipelineJob job = (DataflowPipelineJob) publisherResult; + + long numEvents = 0; + long startMsSinceEpoch = -1; + long endMsSinceEpoch = -1; + while (true) { + PipelineResult.State state = job.getState(); + switch (state) { + case UNKNOWN: + // Keep waiting. + NexmarkUtils.console("%s publisher (%d events)", state, numEvents); + break; + case STOPPED: + case DONE: + case CANCELLED: + case FAILED: + case UPDATED: + NexmarkUtils.console("%s publisher (%d events)", state, numEvents); + return; + case RUNNING: + numEvents = getLong(job, publisherMonitor.getElementCounter()); + if (startMsSinceEpoch < 0 && numEvents > 0) { + startMsSinceEpoch = System.currentTimeMillis(); + endMsSinceEpoch = startMsSinceEpoch + + Duration.standardSeconds(configuration.preloadSeconds).getMillis(); + } + if (endMsSinceEpoch < 0) { + NexmarkUtils.console("%s publisher (%d events)", state, numEvents); + } else { + long remainMs = endMsSinceEpoch - System.currentTimeMillis(); + if (remainMs > 0) { + NexmarkUtils.console("%s publisher (%d events, waiting for %ds)", state, numEvents, + remainMs / 1000); + } else { + NexmarkUtils.console("publisher preloaded %d events", numEvents); + return; + } + } + break; + } + + try { + Thread.sleep(PERF_DELAY.getMillis()); + } catch (InterruptedException e) { + Thread.interrupted(); + throw new RuntimeException("Interrupted: publisher still running."); + } + } + } + + /** + * Monitor the performance and progress of a running job. Return final performance if + * it was measured. + */ + @Override + @Nullable + protected NexmarkPerf monitor(NexmarkQuery query) { + if (!options.getMonitorJobs()) { + return null; + } + if (!(mainResult instanceof DataflowPipelineJob)) { + return null; + } + + if (configuration.debug) { + NexmarkUtils.console("Waiting for main pipeline to 'finish'"); + } else { + NexmarkUtils.console("--debug=false, so job will not self-cancel"); + } + + DataflowPipelineJob job = (DataflowPipelineJob) mainResult; + DataflowPipelineJob publisherJob = (DataflowPipelineJob) publisherResult; + List snapshots = new ArrayList<>(); + long startMsSinceEpoch = System.currentTimeMillis(); + long endMsSinceEpoch = -1; + if (options.getRunningTimeMinutes() != null) { + endMsSinceEpoch = startMsSinceEpoch + + Duration.standardMinutes(options.getRunningTimeMinutes()).getMillis() + - Duration.standardSeconds(configuration.preloadSeconds).getMillis(); + } + long lastActivityMsSinceEpoch = -1; + NexmarkPerf perf = null; + boolean waitingForShutdown = false; + boolean publisherCancelled = false; + List errors = new ArrayList<>(); + + while (true) { + long now = System.currentTimeMillis(); + if (endMsSinceEpoch >= 0 && now > endMsSinceEpoch && !waitingForShutdown) { + NexmarkUtils.console("Reached end of test, cancelling job"); + try { + job.cancel(); + } catch (IOException e) { + throw new RuntimeException("Unable to cancel main job: ", e); + } + if (publisherResult != null) { + try { + publisherJob.cancel(); + } catch (IOException e) { + throw new RuntimeException("Unable to cancel publisher job: ", e); + } + publisherCancelled = true; + } + waitingForShutdown = true; + } + + PipelineResult.State state = job.getState(); + NexmarkUtils.console("%s %s%s", state, queryName, + waitingForShutdown ? " (waiting for shutdown)" : ""); + + NexmarkPerf currPerf; + if (configuration.debug) { + currPerf = currentPerf(startMsSinceEpoch, now, job, snapshots, + query.eventMonitor, query.resultMonitor); + } else { + currPerf = null; + } + + if (perf == null || perf.anyActivity(currPerf)) { + lastActivityMsSinceEpoch = now; + } + + if (options.isStreaming() && !waitingForShutdown) { + Duration quietFor = new Duration(lastActivityMsSinceEpoch, now); + if (query.getFatalCount() != null && getLong(job, query.getFatalCount()) > 0) { + NexmarkUtils.console("job has fatal errors, cancelling."); + errors.add(String.format("Pipeline reported %s fatal errors", query.getFatalCount())); + waitingForShutdown = true; + } else if (configuration.debug && configuration.numEvents > 0 + && currPerf.numEvents == configuration.numEvents + && currPerf.numResults >= 0 && quietFor.isLongerThan(DONE_DELAY)) { + NexmarkUtils.console("streaming query appears to have finished, cancelling job."); + waitingForShutdown = true; + } else if (quietFor.isLongerThan(STUCK_TERMINATE_DELAY)) { + NexmarkUtils.console("streaming query appears to have gotten stuck, cancelling job."); + errors.add("Streaming job was cancelled since appeared stuck"); + waitingForShutdown = true; + } else if (quietFor.isLongerThan(STUCK_WARNING_DELAY)) { + NexmarkUtils.console("WARNING: streaming query appears to have been stuck for %d min.", + quietFor.getStandardMinutes()); + errors.add( + String.format("Streaming query was stuck for %d min", quietFor.getStandardMinutes())); + } + + errors.addAll(checkWatermarks(job, startMsSinceEpoch)); + + if (waitingForShutdown) { + try { + job.cancel(); + } catch (IOException e) { + throw new RuntimeException("Unable to cancel main job: ", e); + } + } + } + + perf = currPerf; + + boolean running = true; + switch (state) { + case UNKNOWN: + case STOPPED: + case RUNNING: + // Keep going. + break; + case DONE: + // All done. + running = false; + break; + case CANCELLED: + running = false; + if (!waitingForShutdown) { + errors.add("Job was unexpectedly cancelled"); + } + break; + case FAILED: + case UPDATED: + // Abnormal termination. + running = false; + errors.add("Job was unexpectedly updated"); + break; + } + + if (!running) { + break; + } + + if (lastActivityMsSinceEpoch == now) { + NexmarkUtils.console("new perf %s", perf); + } else { + NexmarkUtils.console("no activity"); + } + + try { + Thread.sleep(PERF_DELAY.getMillis()); + } catch (InterruptedException e) { + Thread.interrupted(); + NexmarkUtils.console("Interrupted: pipeline is still running"); + } + } + + perf.errors = errors; + perf.snapshots = snapshots; + + if (publisherResult != null) { + NexmarkUtils.console("Shutting down publisher pipeline."); + try { + if (!publisherCancelled) { + publisherJob.cancel(); + } + publisherJob.waitToFinish(5, TimeUnit.MINUTES, null); + } catch (IOException e) { + throw new RuntimeException("Unable to cancel publisher job: ", e); + } catch (InterruptedException e) { + Thread.interrupted(); + throw new RuntimeException("Interrupted: publish job still running.", e); + } + } + + return perf; + } + + enum MetricType { + SYSTEM_WATERMARK, + DATA_WATERMARK, + OTHER + } + + private MetricType getMetricType(MetricUpdate metric) { + String metricName = metric.getName().getName(); + if (metricName.endsWith("windmill-system-watermark")) { + return MetricType.SYSTEM_WATERMARK; + } else if (metricName.endsWith("windmill-data-watermark")) { + return MetricType.DATA_WATERMARK; + } else { + return MetricType.OTHER; + } + } + + /** + * Check that watermarks are not too far behind. + *

              + *

              Returns a list of errors detected. + */ + private List checkWatermarks(DataflowPipelineJob job, long startMsSinceEpoch) { + long now = System.currentTimeMillis(); + List errors = new ArrayList<>(); + try { + JobMetrics metricResponse = job.getDataflowClient() + .projects() + .jobs() + .getMetrics(job.getProjectId(), job.getJobId()) + .execute(); + List metrics = metricResponse.getMetrics(); + if (metrics != null) { + boolean foundWatermarks = false; + for (MetricUpdate metric : metrics) { + MetricType type = getMetricType(metric); + if (type == MetricType.OTHER) { + continue; + } + foundWatermarks = true; + @SuppressWarnings("unchecked") + BigDecimal scalar = (BigDecimal) metric.getScalar(); + if (scalar.signum() < 0) { + continue; + } + Instant value = + new Instant(scalar.divideToIntegralValue(new BigDecimal(1000)).longValueExact()); + Instant updateTime = Instant.parse(metric.getUpdateTime()); + + if (options.getWatermarkValidationDelaySeconds() == null + || now > startMsSinceEpoch + + Duration.standardSeconds(options.getWatermarkValidationDelaySeconds()) + .getMillis()) { + Duration threshold = null; + if (type == MetricType.SYSTEM_WATERMARK && options.getMaxSystemLagSeconds() != null) { + threshold = Duration.standardSeconds(options.getMaxSystemLagSeconds()); + } else if (type == MetricType.DATA_WATERMARK + && options.getMaxDataLagSeconds() != null) { + threshold = Duration.standardSeconds(options.getMaxDataLagSeconds()); + } + + if (threshold != null && value.isBefore(updateTime.minus(threshold))) { + String msg = String.format("High lag for %s: %s vs %s (allowed lag of %s)", + metric.getName().getName(), value, updateTime, threshold); + errors.add(msg); + NexmarkUtils.console(msg); + } + } + } + if (!foundWatermarks) { + NexmarkUtils.console("No known watermarks in update: " + metrics); + if (now > startMsSinceEpoch + Duration.standardMinutes(5).getMillis()) { + errors.add("No known watermarks found. Metrics were " + metrics); + } + } + } + } catch (IOException e) { + NexmarkUtils.console("Warning: failed to get JobMetrics: " + e); + } + + return errors; + } + + /** + * Return the current performance given {@code eventMonitor} and {@code resultMonitor}. + */ + private NexmarkPerf currentPerf( + long startMsSinceEpoch, long now, DataflowPipelineJob job, + List snapshots, Monitor eventMonitor, + Monitor resultMonitor) { + NexmarkPerf perf = new NexmarkPerf(); + + long numEvents = getLong(job, eventMonitor.getElementCounter()); + long numEventBytes = getLong(job, eventMonitor.getBytesCounter()); + long eventStart = getTimestamp(now, job, eventMonitor.getStartTime()); + long eventEnd = getTimestamp(now, job, eventMonitor.getEndTime()); + long numResults = getLong(job, resultMonitor.getElementCounter()); + long numResultBytes = getLong(job, resultMonitor.getBytesCounter()); + long resultStart = getTimestamp(now, job, resultMonitor.getStartTime()); + long resultEnd = getTimestamp(now, job, resultMonitor.getEndTime()); + long timestampStart = getTimestamp(now, job, resultMonitor.getStartTimestamp()); + long timestampEnd = getTimestamp(now, job, resultMonitor.getEndTimestamp()); + + long effectiveEnd = -1; + if (eventEnd >= 0 && resultEnd >= 0) { + // It is possible for events to be generated after the last result was emitted. + // (Eg Query 2, which only yields results for a small prefix of the event stream.) + // So use the max of last event and last result times. + effectiveEnd = Math.max(eventEnd, resultEnd); + } else if (resultEnd >= 0) { + effectiveEnd = resultEnd; + } else if (eventEnd >= 0) { + // During startup we may have no result yet, but we would still like to track how + // long the pipeline has been running. + effectiveEnd = eventEnd; + } + + if (effectiveEnd >= 0 && eventStart >= 0 && effectiveEnd >= eventStart) { + perf.runtimeSec = (effectiveEnd - eventStart) / 1000.0; + } + + if (numEvents >= 0) { + perf.numEvents = numEvents; + } + + if (numEvents >= 0 && perf.runtimeSec > 0.0) { + // For streaming we may later replace this with a 'steady-state' value calculated + // from the progress snapshots. + perf.eventsPerSec = numEvents / perf.runtimeSec; + } + + if (numEventBytes >= 0 && perf.runtimeSec > 0.0) { + perf.eventBytesPerSec = numEventBytes / perf.runtimeSec; + } + + if (numResults >= 0) { + perf.numResults = numResults; + } + + if (numResults >= 0 && perf.runtimeSec > 0.0) { + perf.resultsPerSec = numResults / perf.runtimeSec; + } + + if (numResultBytes >= 0 && perf.runtimeSec > 0.0) { + perf.resultBytesPerSec = numResultBytes / perf.runtimeSec; + } + + if (eventStart >= 0) { + perf.startupDelaySec = (eventStart - startMsSinceEpoch) / 1000.0; + } + + if (resultStart >= 0 && eventStart >= 0 && resultStart >= eventStart) { + perf.processingDelaySec = (resultStart - eventStart) / 1000.0; + } + + if (timestampStart >= 0 && timestampEnd >= 0 && perf.runtimeSec > 0.0) { + double eventRuntimeSec = (timestampEnd - timestampStart) / 1000.0; + perf.timeDilation = eventRuntimeSec / perf.runtimeSec; + } + + if (resultEnd >= 0) { + // Fill in the shutdown delay assuming the job has now finished. + perf.shutdownDelaySec = (now - resultEnd) / 1000.0; + } + + perf.jobId = job.getJobId(); + // As soon as available, try to capture cumulative cost at this point too. + + NexmarkPerf.ProgressSnapshot snapshot = new NexmarkPerf.ProgressSnapshot(); + snapshot.secSinceStart = (now - startMsSinceEpoch) / 1000.0; + snapshot.runtimeSec = perf.runtimeSec; + snapshot.numEvents = numEvents; + snapshot.numResults = numResults; + snapshots.add(snapshot); + + captureSteadyState(perf, snapshots); + + return perf; + } + + /** + * Find a 'steady state' events/sec from {@code snapshots} and + * store it in {@code perf} if found. + */ + private void captureSteadyState(NexmarkPerf perf, List snapshots) { + if (!options.isStreaming()) { + return; + } + + // Find the first sample with actual event and result counts. + int dataStart = 0; + for (; dataStart < snapshots.size(); dataStart++) { + if (snapshots.get(dataStart).numEvents >= 0 && snapshots.get(dataStart).numResults >= 0) { + break; + } + } + + // Find the last sample which demonstrated progress. + int dataEnd = snapshots.size() - 1; + for (; dataEnd > dataStart; dataEnd--) { + if (snapshots.get(dataEnd).anyActivity(snapshots.get(dataEnd - 1))) { + break; + } + } + + int numSamples = dataEnd - dataStart + 1; + if (numSamples < MIN_SAMPLES) { + // Not enough samples. + NexmarkUtils.console("%d samples not enough to calculate steady-state event rate", + numSamples); + return; + } + + // We'll look at only the middle third samples. + int sampleStart = dataStart + numSamples / 3; + int sampleEnd = dataEnd - numSamples / 3; + + double sampleSec = + snapshots.get(sampleEnd).secSinceStart - snapshots.get(sampleStart).secSinceStart; + if (sampleSec < MIN_WINDOW.getStandardSeconds()) { + // Not sampled over enough time. + NexmarkUtils.console( + "sample of %.1f sec not long enough to calculate steady-state event rate", + sampleSec); + return; + } + + // Find rate with least squares error. + double sumxx = 0.0; + double sumxy = 0.0; + long prevNumEvents = -1; + for (int i = sampleStart; i <= sampleEnd; i++) { + if (prevNumEvents == snapshots.get(i).numEvents) { + // Skip samples with no change in number of events since they contribute no data. + continue; + } + // Use the effective runtime instead of wallclock time so we can + // insulate ourselves from delays and stutters in the query manager. + double x = snapshots.get(i).runtimeSec; + prevNumEvents = snapshots.get(i).numEvents; + double y = prevNumEvents; + sumxx += x * x; + sumxy += x * y; + } + double eventsPerSec = sumxy / sumxx; + NexmarkUtils.console("revising events/sec from %.1f to %.1f", perf.eventsPerSec, eventsPerSec); + perf.eventsPerSec = eventsPerSec; + } + + /** + * Return the current value for a long counter, or -1 if can't be retrieved. + */ + private long getLong(DataflowPipelineJob job, Aggregator aggregator) { + try { + Collection values = job.getAggregatorValues(aggregator).getValues(); + if (values.size() != 1) { + return -1; + } + return Iterables.getOnlyElement(values); + } catch (AggregatorRetrievalException e) { + return -1; + } + } + + /** + * Return the current value for a time counter, or -1 if can't be retrieved. + */ + private long getTimestamp( + long now, DataflowPipelineJob job, Aggregator aggregator) { + try { + Collection values = job.getAggregatorValues(aggregator).getValues(); + if (values.size() != 1) { + return -1; + } + long value = Iterables.getOnlyElement(values); + if (Math.abs(value - now) > Duration.standardDays(10000).getMillis()) { + return -1; + } + return value; + } catch (AggregatorRetrievalException e) { + return -1; + } + } +} diff --git a/integration/java/src/main/java/org/apache/beam/integration/nexmark/NexmarkInProcessDriver.java b/integration/java/src/main/java/org/apache/beam/integration/nexmark/NexmarkInProcessDriver.java new file mode 100644 index 0000000000000..fe279c031f996 --- /dev/null +++ b/integration/java/src/main/java/org/apache/beam/integration/nexmark/NexmarkInProcessDriver.java @@ -0,0 +1,48 @@ +/* + * 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.integration.nexmark; + +import org.apache.beam.runners.dataflow.options.DataflowPipelineOptions; +import org.apache.beam.runners.direct.InProcessPipelineRunner; +import org.apache.beam.sdk.options.PipelineOptionsFactory; + +/** + * An implementation of the 'NEXMark queries' using the in-process runner. + */ +class NexmarkInProcessDriver extends NexmarkDriver { + /** + * Command line flags. + */ + public interface NexmarkInProcessOptions extends Options, DataflowPipelineOptions { + } + + /** + * Entry point. + */ + public static void main(String[] args) { + NexmarkInProcessOptions options = + PipelineOptionsFactory.fromArgs(args) + .withValidation() + .as(NexmarkInProcessOptions.class); + options.setRunner(InProcessPipelineRunner.class); + NexmarkInProcessRunner runner = new NexmarkInProcessRunner(options); + new NexmarkInProcessDriver().runAll(options, runner); + } +} + diff --git a/integration/java/src/main/java/org/apache/beam/integration/nexmark/NexmarkInProcessRunner.java b/integration/java/src/main/java/org/apache/beam/integration/nexmark/NexmarkInProcessRunner.java new file mode 100644 index 0000000000000..ba141f947042c --- /dev/null +++ b/integration/java/src/main/java/org/apache/beam/integration/nexmark/NexmarkInProcessRunner.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.integration.nexmark; + +import javax.annotation.Nullable; + +/** + * Run a singe Nexmark query using a given configuration on Google Dataflow. + */ +class NexmarkInProcessRunner extends NexmarkRunner { + public NexmarkInProcessRunner(NexmarkInProcessDriver.NexmarkInProcessOptions options) { + super(options); + } + + @Override + protected boolean isStreaming() { + return options.isStreaming(); + } + + @Override + protected int coresPerWorker() { + return 4; + } + + @Override + protected int maxNumWorkers() { + return 1; + } + + @Override + protected boolean canMonitor() { + return false; + } + + @Override + protected void invokeBuilderForPublishOnlyPipeline(PipelineBuilder builder) { + throw new UnsupportedOperationException( + "Cannot use --pubSubMode=COMBINED with InProcessPipelineRunner"); + } + + /** + * Monitor the progress of the publisher job. Return when it has been generating events for + * at least {@code configuration.preloadSeconds}. + */ + @Override + protected void waitForPublisherPreload() { + throw new UnsupportedOperationException( + "Cannot use --pubSubMode=COMBINED with InProcessPipelineRunner"); + } + + /** + * Monitor the performance and progress of a running job. Return final performance if + * it was measured. + */ + @Override + @Nullable + protected NexmarkPerf monitor(NexmarkQuery query) { + throw new UnsupportedOperationException( + "Cannot use --monitorJobs=true with InProcessPipelineRunner"); + } +} diff --git a/integration/java/src/main/java/org/apache/beam/integration/nexmark/NexmarkPerf.java b/integration/java/src/main/java/org/apache/beam/integration/nexmark/NexmarkPerf.java new file mode 100644 index 0000000000000..6eb7267b7eaef --- /dev/null +++ b/integration/java/src/main/java/org/apache/beam/integration/nexmark/NexmarkPerf.java @@ -0,0 +1,212 @@ +/* + * 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.integration.nexmark; + +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.core.JsonProcessingException; + +import java.io.IOException; +import java.util.List; + +import javax.annotation.Nullable; + +/** + * Summary of performance for a particular run of a configuration. + */ +class NexmarkPerf { + /** + * A sample of the number of events and number of results (if known) generated at + * a particular time. + */ + public static class ProgressSnapshot { + /** Seconds since job was started (in wallclock time). */ + @JsonProperty + double secSinceStart; + + /** Job runtime in seconds (time from first event to last generated event or output result). */ + @JsonProperty + double runtimeSec; + + /** Cumulative number of events generated. -1 if not known. */ + @JsonProperty + long numEvents; + + /** Cumulative number of results emitted. -1 if not known. */ + @JsonProperty + long numResults; + + /** + * Return true if there looks to be activity between {@code this} and {@code that} + * snapshots. + */ + public boolean anyActivity(ProgressSnapshot that) { + if (runtimeSec != that.runtimeSec) { + // An event or result end timestamp looks to have changed. + return true; + } + if (numEvents != that.numEvents) { + // Some more events were generated. + return true; + } + if (numResults != that.numResults) { + // Some more results were emitted. + return true; + } + return false; + } + } + + /** + * Progess snapshots. Null if not yet calculated. + */ + @JsonProperty + @Nullable + public List snapshots = null; + + /** + * Effective runtime, in seconds. Measured from timestamp of first generated event to latest of + * timestamp of last generated event and last emitted result. -1 if not known. + */ + @JsonProperty + public double runtimeSec = -1.0; + + /** + * Number of events generated. -1 if not known. + */ + @JsonProperty + public long numEvents = -1; + + /** + * Number of events generated per second of runtime. For batch this is number of events + * over the above runtime. For streaming this is the 'steady-state' event generation rate sampled + * over the lifetime of the job. -1 if not known. + */ + @JsonProperty + public double eventsPerSec = -1.0; + + /** + * Number of event bytes generated per second of runtime. -1 if not known. + */ + @JsonProperty + public double eventBytesPerSec = -1.0; + + /** + * Number of results emitted. -1 if not known. + */ + @JsonProperty + public long numResults = -1; + + /** + * Number of results generated per second of runtime. -1 if not known. + */ + @JsonProperty + public double resultsPerSec = -1.0; + + /** + * Number of result bytes generated per second of runtime. -1 if not known. + */ + @JsonProperty + public double resultBytesPerSec = -1.0; + + /** + * Delay between start of job and first event in second. -1 if not known. + */ + @JsonProperty + public double startupDelaySec = -1.0; + + /** + * Delay between first event and first result in seconds. -1 if not known. + */ + @JsonProperty + public double processingDelaySec = -1.0; + + /** + * Delay between last result and job completion in seconds. -1 if not known. + */ + @JsonProperty + public double shutdownDelaySec = -1.0; + + /** + * Time-dilation factor. Calculate as event time advancement rate relative to real time. + * Greater than one implies we processed events faster than they would have been generated + * in real time. Less than one implies we could not keep up with events in real time. + * -1 if not known. + */ + @JsonProperty + double timeDilation = -1.0; + + /** + * List of errors encountered during job execution. + */ + @JsonProperty + @Nullable + public List errors = null; + + /** + * The job id this perf was drawn from. Null if not known. + */ + @JsonProperty + @Nullable + public String jobId = null; + + /** + * Return a JSON representation of performance. + */ + @Override + public String toString() { + try { + return NexmarkUtils.MAPPER.writeValueAsString(this); + } catch (JsonProcessingException e) { + throw new RuntimeException(e); + } + } + + /** + * Parse a {@link NexmarkPerf} object from JSON {@code string}. + * + * @throws IOException + */ + public static NexmarkPerf fromString(String string) { + try { + return NexmarkUtils.MAPPER.readValue(string, NexmarkPerf.class); + } catch (IOException e) { + throw new RuntimeException("Unable to parse nexmark perf: ", e); + } + } + + /** + * Return true if there looks to be activity between {@code this} and {@code that} + * perf values. + */ + public boolean anyActivity(NexmarkPerf that) { + if (runtimeSec != that.runtimeSec) { + // An event or result end timestamp looks to have changed. + return true; + } + if (numEvents != that.numEvents) { + // Some more events were generated. + return true; + } + if (numResults != that.numResults) { + // Some more results were emitted. + return true; + } + return false; + } +} diff --git a/integration/java/src/main/java/org/apache/beam/integration/nexmark/NexmarkQuery.java b/integration/java/src/main/java/org/apache/beam/integration/nexmark/NexmarkQuery.java new file mode 100644 index 0000000000000..462660948bd00 --- /dev/null +++ b/integration/java/src/main/java/org/apache/beam/integration/nexmark/NexmarkQuery.java @@ -0,0 +1,276 @@ +/* + * 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.integration.nexmark; + +import org.apache.beam.sdk.transforms.Aggregator; +import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.transforms.Filter; +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.values.KV; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.TimestampedValue; +import org.apache.beam.sdk.values.TupleTag; + +import org.joda.time.Instant; + +import javax.annotation.Nullable; + +/** + * Base class for the eight 'NEXMark' queries. Supplies some fragments common to + * multiple queries. + */ +public abstract class NexmarkQuery + extends PTransform, PCollection>> { + protected static final TupleTag AUCTION_TAG = new TupleTag<>("auctions"); + protected static final TupleTag BID_TAG = new TupleTag<>("bids"); + protected static final TupleTag PERSON_TAG = new TupleTag<>("person"); + + /** Predicate to detect a new person event. */ + protected static final SerializableFunction IS_NEW_PERSON = + new SerializableFunction() { + @Override + public Boolean apply(Event event) { + return event.newPerson != null; + } + }; + + /** DoFn to convert a new person event to a person. */ + protected static final DoFn AS_PERSON = new DoFn() { + @Override + public void processElement(ProcessContext c) { + c.output(c.element().newPerson); + } + }; + + /** Predicate to detect a new auction event. */ + protected static final SerializableFunction IS_NEW_AUCTION = + new SerializableFunction() { + @Override + public Boolean apply(Event event) { + return event.newAuction != null; + } + }; + + /** DoFn to convert a new auction event to an auction. */ + protected static final DoFn AS_AUCTION = new DoFn() { + @Override + public void processElement(ProcessContext c) { + c.output(c.element().newAuction); + } + }; + + /** Predicate to detect a new bid event. */ + protected static final SerializableFunction IS_BID = + new SerializableFunction() { + @Override + public Boolean apply(Event event) { + return event.bid != null; + } + }; + + /** DoFn to convert a bid event to a bid. */ + protected static final DoFn AS_BID = new DoFn() { + @Override + public void processElement(ProcessContext c) { + c.output(c.element().bid); + } + }; + + /** Transform to key each person by their id. */ + protected static final ParDo.Bound> PERSON_BY_ID = + ParDo.named("PersonById") + .of(new DoFn>() { + @Override + public void processElement(ProcessContext c) { + c.output(KV.of(c.element().id, c.element())); + } + }); + + /** Transform to key each auction by its id. */ + protected static final ParDo.Bound> AUCTION_BY_ID = + ParDo.named("AuctionById") + .of(new DoFn>() { + @Override + public void processElement(ProcessContext c) { + c.output(KV.of(c.element().id, c.element())); + } + }); + + /** Transform to key each auction by its seller id. */ + protected static final ParDo.Bound> AUCTION_BY_SELLER = + ParDo.named("AuctionBySeller") + .of(new DoFn>() { + @Override + public void processElement(ProcessContext c) { + c.output(KV.of(c.element().seller, c.element())); + } + }); + + /** Transform to key each bid by it's auction id. */ + protected static final ParDo.Bound> BID_BY_AUCTION = + ParDo.named("BidByAuction") + .of(new DoFn>() { + @Override + public void processElement(ProcessContext c) { + c.output(KV.of(c.element().auction, c.element())); + } + }); + + /** Transform to project the auction id from each bid. */ + protected static final ParDo.Bound BID_TO_AUCTION = + ParDo.named("BidToAuction") + .of(new DoFn() { + @Override + public void processElement(ProcessContext c) { + c.output(c.element().auction); + } + }); + + /** Transform to project the price from each bid. */ + protected static final ParDo.Bound BID_TO_PRICE = + ParDo.named("BidToPrice") + .of(new DoFn() { + @Override + public void processElement(ProcessContext c) { + c.output(c.element().price); + } + }); + + /** Transform to emit each event with the timestamp embedded within it. */ + public static final ParDo.Bound EVENT_TIMESTAMP_FROM_DATA = + ParDo.named("OutputWithTimestamp") + .of(new DoFn() { + @Override + public void processElement(ProcessContext c) { + Event e = c.element(); + if (e.bid != null) { + c.outputWithTimestamp(e, new Instant(e.bid.dateTime)); + } else if (e.newPerson != null) { + c.outputWithTimestamp(e, new Instant(e.newPerson.dateTime)); + } else if (e.newAuction != null) { + c.outputWithTimestamp(e, new Instant(e.newAuction.dateTime)); + } + } + }); + + /** + * Transform to filter for just the new auction events. + */ + protected static final PTransform, PCollection> JUST_NEW_AUCTIONS = + new PTransform, PCollection>("justNewAuctions") { + @Override + public PCollection apply(PCollection input) { + return input.apply(Filter.byPredicate(IS_NEW_AUCTION).named("IsAuction")) + .apply(ParDo.named("AsAuction").of(AS_AUCTION)); + } + }; + + /** + * Transform to filter for just the new person events. + */ + protected static final PTransform, PCollection> JUST_NEW_PERSONS = + new PTransform, PCollection>("justNewPersons") { + @Override + public PCollection apply(PCollection input) { + return input.apply(Filter.byPredicate(IS_NEW_PERSON).named("IsPerson")) + .apply(ParDo.named("AsPerson").of(AS_PERSON)); + } + }; + + /** + * Transform to filter for just the bid events. + */ + protected static final PTransform, PCollection> JUST_BIDS = + new PTransform, PCollection>("justBids") { + @Override + public PCollection apply(PCollection input) { + return input.apply(Filter.byPredicate(IS_BID).named("IsBid")) + .apply(ParDo.named("AsBid").of(AS_BID)); + } + }; + + protected final NexmarkConfiguration configuration; + public final Monitor eventMonitor; + public final Monitor resultMonitor; + public final Monitor endOfStreamMonitor; + + protected NexmarkQuery(NexmarkConfiguration configuration, String name) { + super(name); + this.configuration = configuration; + if (configuration.debug) { + eventMonitor = new Monitor<>(name + ".Events", "event"); + resultMonitor = new Monitor<>(name + ".Results", "result"); + endOfStreamMonitor = new Monitor<>(name + ".EndOfStream", "end"); + } else { + eventMonitor = null; + resultMonitor = null; + endOfStreamMonitor = null; + } + } + + /** + * Return the aggregator which counts fatal errors in this query. Return null if no such + * aggregator. + */ + @Nullable + public Aggregator getFatalCount() { + return null; + } + + /** + * Implement the actual query. All we know about the result is it has a known encoded size. + */ + protected abstract PCollection applyPrim(PCollection events); + + @Override + public PCollection> apply(PCollection events) { + + if (configuration.debug) { + events = + events + // Monitor events as they go by. + .apply(eventMonitor.getTransform()) + // Count each type of event. + .apply(NexmarkUtils.snoop(name)); + } + + if (configuration.cpuDelayMs > 0) { + // Slow down by pegging one core at 100%. + events = events.apply(NexmarkUtils.cpuDelay(name, configuration.cpuDelayMs)); + } + + if (configuration.diskBusyBytes > 0) { + // Slow down by forcing bytes to durable store. + events = events.apply(NexmarkUtils.diskBusy(name, configuration.diskBusyBytes)); + } + + // Run the query. + PCollection queryResults = applyPrim(events); + + if (configuration.debug) { + // Monitor results as they go by. + queryResults = queryResults.apply(resultMonitor.getTransform()); + } + + // Timestamp the query results. + return queryResults.apply(NexmarkUtils.stamp(name)); + } +} diff --git a/integration/java/src/main/java/org/apache/beam/integration/nexmark/NexmarkQueryModel.java b/integration/java/src/main/java/org/apache/beam/integration/nexmark/NexmarkQueryModel.java new file mode 100644 index 0000000000000..b42042f0de8e8 --- /dev/null +++ b/integration/java/src/main/java/org/apache/beam/integration/nexmark/NexmarkQueryModel.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.integration.nexmark; + +import org.apache.beam.sdk.transforms.SerializableFunction; +import org.apache.beam.sdk.values.TimestampedValue; + +import org.joda.time.Duration; +import org.joda.time.Instant; +import org.junit.Assert; + +import java.io.Serializable; +import java.util.ArrayList; +import java.util.Collection; +import java.util.HashSet; +import java.util.Iterator; +import java.util.List; +import java.util.Set; + +/** + * Base class for models of the eight NEXMark queries. Provides an assertion + * function which can be applied against the actual query results to check their consistency + * with the model. + */ +public abstract class NexmarkQueryModel implements Serializable { + /** + * Return the start of the most recent window of {@code size} and {@code period} which ends + * strictly before {@code timestamp}. + */ + public static Instant windowStart(Duration size, Duration period, Instant timestamp) { + long ts = timestamp.getMillis(); + long p = period.getMillis(); + long lim = ts - ts % p; + long s = size.getMillis(); + return new Instant(lim - s); + } + + protected final NexmarkConfiguration configuration; + + public NexmarkQueryModel(NexmarkConfiguration configuration) { + this.configuration = configuration; + } + + /** + * Convert {@code itr} to strings capturing values, timestamps and order. + */ + protected static List toValueTimestampOrder(Iterator> itr) { + List strings = new ArrayList<>(); + while (itr.hasNext()) { + strings.add(itr.next().toString()); + } + return strings; + } + + /** + * Convert {@code itr} to strings capturing values and order. + */ + protected static List toValueOrder(Iterator> itr) { + List strings = new ArrayList<>(); + while (itr.hasNext()) { + strings.add(itr.next().getValue().toString()); + } + return strings; + } + + /** + * Convert {@code itr} to strings capturing values only. + */ + protected static Set toValue(Iterator> itr) { + Set strings = new HashSet<>(); + while (itr.hasNext()) { + strings.add(itr.next().getValue().toString()); + } + return strings; + } + + /** Return simulator for query. */ + protected abstract AbstractSimulator simulator(); + + /** Return sub-sequence of results which are significant for model. */ + protected Iterable> relevantResults( + Iterable> results) { + return results; + } + + /** + * Convert iterator of elements to collection of strings to use when testing coherence + * of model against actual query results. + */ + protected abstract Collection toCollection(Iterator> itr); + + /** + * Return assertion to use on results of pipeline for this query. + */ + public SerializableFunction>, Void> assertionFor() { + final Collection expectedStrings = toCollection(simulator().results()); + + return new SerializableFunction>, Void>() { + @Override + public Void apply(Iterable> actual) { + Collection actualStrings = toCollection(relevantResults(actual).iterator()); + Assert.assertEquals(expectedStrings, actualStrings); + return null; + } + }; + } +} diff --git a/integration/java/src/main/java/org/apache/beam/integration/nexmark/NexmarkRunner.java b/integration/java/src/main/java/org/apache/beam/integration/nexmark/NexmarkRunner.java new file mode 100644 index 0000000000000..b7151f87a143c --- /dev/null +++ b/integration/java/src/main/java/org/apache/beam/integration/nexmark/NexmarkRunner.java @@ -0,0 +1,746 @@ +/* + * 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.integration.nexmark; + +import org.apache.beam.sdk.Pipeline; +import org.apache.beam.sdk.PipelineResult; +import org.apache.beam.sdk.io.AvroIO; +import org.apache.beam.sdk.io.BigQueryIO; +import org.apache.beam.sdk.io.PubsubIO; +import org.apache.beam.sdk.io.TextIO; +import org.apache.beam.sdk.testing.PAssert; +import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.transforms.ParDo; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.PCollectionTuple; +import org.apache.beam.sdk.values.TimestampedValue; +import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.TupleTagList; + +import com.google.api.services.bigquery.model.TableFieldSchema; +import com.google.api.services.bigquery.model.TableRow; +import com.google.api.services.bigquery.model.TableSchema; +import com.google.common.base.Preconditions; +import com.google.common.base.Strings; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.Lists; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.concurrent.ThreadLocalRandom; +import javax.annotation.Nullable; + +/** + * Run a single Nexmark query using a given configuration. + */ +public abstract class NexmarkRunner { + /** + * Options shared by all runs. + */ + protected final OptionT options; + + /** + * Which configuration we are running. + */ + @Nullable + protected NexmarkConfiguration configuration; + + /** + * Accumulate the pub/sub subscriptions etc which should be cleaned up on end of run. + */ + @Nullable + protected PubsubHelper pubsub; + + /** + * If in --pubsubMode=COMBINED, the event monitor for the publisher pipeline. Otherwise null. + */ + @Nullable + protected Monitor publisherMonitor; + + /** + * If in --pubsubMode=COMBINED, the pipeline result for the publisher pipeline. Otherwise null. + */ + @Nullable + protected PipelineResult publisherResult; + + /** + * Result for the main pipeline. + */ + @Nullable + protected PipelineResult mainResult; + + /** + * Query name we are running. + */ + @Nullable + protected String queryName; + + public NexmarkRunner(OptionT options) { + this.options = options; + } + + /** + * Return a Pubsub helper. + */ + private PubsubHelper getPubsub() { + if (pubsub == null) { + pubsub = PubsubHelper.create(options); + } + return pubsub; + } + + // ================================================================================ + // Overridden by each runner. + // ================================================================================ + + /** + * Is this query running in streaming mode? + */ + protected abstract boolean isStreaming(); + + /** + * Return number of cores per worker. + */ + protected abstract int coresPerWorker(); + + /** + * Return maximum number of workers. + */ + protected abstract int maxNumWorkers(); + + /** + * Return true if runner can monitor running jobs. + */ + protected abstract boolean canMonitor(); + + /** + * Build and run a pipeline using specified options. + */ + protected interface PipelineBuilder { + void build(OptionT publishOnlyOptions); + } + + /** + * Invoke the builder with options suitable for running a publish-only child pipeline. + */ + protected abstract void invokeBuilderForPublishOnlyPipeline(PipelineBuilder builder); + + /** + * If monitoring, wait until the publisher pipeline has run long enough to establish + * a backlog on the Pubsub topic. Otherwise, return immediately. + */ + protected abstract void waitForPublisherPreload(); + + /** + * If monitoring, print stats on the main pipeline and return the final perf + * when it has run long enough. Otherwise, return {@literal null} immediately. + */ + @Nullable + protected abstract NexmarkPerf monitor(NexmarkQuery query); + + // ================================================================================ + // Basic sources and sinks + // ================================================================================ + + /** + * Return a topic name. + */ + private String shortTopic(long now) { + String baseTopic = options.getPubsubTopic(); + if (Strings.isNullOrEmpty(baseTopic)) { + throw new RuntimeException("Missing --pubsubTopic"); + } + switch (options.getResourceNameMode()) { + case VERBATIM: + return baseTopic; + case QUERY: + return String.format("%s_%s_source", baseTopic, queryName); + case QUERY_AND_SALT: + return String.format("%s_%s_%d_source", baseTopic, queryName, now); + } + throw new RuntimeException("Unrecognized enum " + options.getResourceNameMode()); + } + + /** + * Return a subscription name. + */ + private String shortSubscription(long now) { + String baseSubscription = options.getPubsubSubscription(); + if (Strings.isNullOrEmpty(baseSubscription)) { + throw new RuntimeException("Missing --pubsubSubscription"); + } + switch (options.getResourceNameMode()) { + case VERBATIM: + return baseSubscription; + case QUERY: + return String.format("%s_%s_source", baseSubscription, queryName); + case QUERY_AND_SALT: + return String.format("%s_%s_%d_source", baseSubscription, queryName, now); + } + throw new RuntimeException("Unrecognized enum " + options.getResourceNameMode()); + } + + /** + * Return a file name for plain text. + */ + private String textFilename(long now) { + String baseFilename = options.getOutputPath(); + if (Strings.isNullOrEmpty(baseFilename)) { + throw new RuntimeException("Missing --outputPath"); + } + switch (options.getResourceNameMode()) { + case VERBATIM: + return baseFilename; + case QUERY: + return String.format("%s/nexmark_%s.txt", baseFilename, queryName); + case QUERY_AND_SALT: + return String.format("%s/nexmark_%s_%d.txt", baseFilename, queryName, now); + } + throw new RuntimeException("Unrecognized enum " + options.getResourceNameMode()); + } + + /** + * Return a BigQuery table spec. + */ + private String tableSpec(long now, String version) { + String baseTableName = options.getBigQueryTable(); + if (Strings.isNullOrEmpty(baseTableName)) { + throw new RuntimeException("Missing --bigQueryTable"); + } + switch (options.getResourceNameMode()) { + case VERBATIM: + return String.format("%s:nexmark.%s_%s", + options.getProject(), baseTableName, version); + case QUERY: + return String.format("%s:nexmark.%s_%s_%s", + options.getProject(), baseTableName, queryName, version); + case QUERY_AND_SALT: + return String.format("%s:nexmark.%s_%s_%s_%d", + options.getProject(), baseTableName, queryName, version, now); + } + throw new RuntimeException("Unrecognized enum " + options.getResourceNameMode()); + } + + /** + * Return a directory for logs. + */ + private String logsDir(long now) { + String baseFilename = options.getOutputPath(); + if (Strings.isNullOrEmpty(baseFilename)) { + throw new RuntimeException("Missing --outputPath"); + } + switch (options.getResourceNameMode()) { + case VERBATIM: + return baseFilename; + case QUERY: + return String.format("%s/logs_%s", baseFilename, queryName); + case QUERY_AND_SALT: + return String.format("%s/logs_%s_%d", baseFilename, queryName, now); + } + throw new RuntimeException("Unrecognized enum " + options.getResourceNameMode()); + } + + /** + * Return a source of synthetic events. + */ + private PCollection sourceEventsFromSynthetic(Pipeline p) { + if (isStreaming()) { + NexmarkUtils.console("Generating %d events in streaming mode", configuration.numEvents); + return p.apply(NexmarkUtils.streamEventsSource(queryName, configuration)); + } else { + NexmarkUtils.console("Generating %d events in batch mode", configuration.numEvents); + return p.apply(NexmarkUtils.batchEventsSource(queryName, configuration)); + } + } + + /** + * Return source of events from Pubsub. + */ + private PCollection sourceEventsFromPubsub(Pipeline p, long now) { + String shortTopic = shortTopic(now); + String shortSubscription = shortSubscription(now); + + // Create/confirm the subscription. + String subscription = null; + if (!options.getManageResources()) { + // The subscription should already have been created by the user. + subscription = getPubsub().reuseSubscription(shortTopic, shortSubscription).getPath(); + } else { + subscription = getPubsub().createSubscription(shortTopic, shortSubscription).getPath(); + } + NexmarkUtils.console("Reading events from Pubsub %s", subscription); + PubsubIO.Read.Bound io = + PubsubIO.Read.named(queryName + ".ReadPubsubEvents") + .subscription(subscription) + .idLabel(NexmarkUtils.PUBSUB_ID) + .withCoder(Event.CODER); + if (!configuration.usePubsubPublishTime) { + io = io.timestampLabel(NexmarkUtils.PUBSUB_TIMESTAMP); + } + return p.apply(io); + } + + /** + * Return Avro source of events from {@code options.getInputFilePrefix}. + */ + private PCollection sourceEventsFromAvro(Pipeline p) { + String filename = options.getInputPath(); + if (Strings.isNullOrEmpty(filename)) { + throw new RuntimeException("Missing --inputPath"); + } + NexmarkUtils.console("Reading events from Avro files at %s", filename); + return p + .apply(AvroIO.Read.named(queryName + ".ReadAvroEvents") + .from(filename + "*.avro") + .withSchema(Event.class)) + .apply(NexmarkQuery.EVENT_TIMESTAMP_FROM_DATA); + } + + /** + * Send {@code events} to Pubsub. + */ + private void sinkEventsToPubsub(PCollection events, long now) { + String shortTopic = shortTopic(now); + + // Create/confirm the topic. + String topic; + if (!options.getManageResources() + || configuration.pubSubMode == NexmarkUtils.PubSubMode.SUBSCRIBE_ONLY) { + // The topic should already have been created by the user or + // a companion 'PUBLISH_ONLY' process. + topic = getPubsub().reuseTopic(shortTopic).getPath(); + } else { + // Create a fresh topic to loopback via. It will be destroyed when the + // (necessarily blocking) job is done. + topic = getPubsub().createTopic(shortTopic).getPath(); + } + NexmarkUtils.console("Writing events to Pubsub %s", topic); + PubsubIO.Write.Bound io = + PubsubIO.Write.named(queryName + ".WritePubsubEvents") + .topic(topic) + .idLabel(NexmarkUtils.PUBSUB_ID) + .withCoder(Event.CODER); + if (!configuration.usePubsubPublishTime) { + io = io.timestampLabel(NexmarkUtils.PUBSUB_TIMESTAMP); + } + events.apply(io); + } + + /** + * Send {@code formattedResults} to Pubsub. + */ + private void sinkResultsToPubsub(PCollection formattedResults, long now) { + String shortTopic = shortTopic(now); + String topic; + if (!options.getManageResources()) { + topic = getPubsub().reuseTopic(shortTopic).getPath(); + } else { + topic = getPubsub().createTopic(shortTopic).getPath(); + } + NexmarkUtils.console("Writing results to Pubsub %s", topic); + PubsubIO.Write.Bound io = + PubsubIO.Write.named(queryName + ".WritePubsubResults") + .topic(topic) + .idLabel(NexmarkUtils.PUBSUB_ID); + if (!configuration.usePubsubPublishTime) { + io = io.timestampLabel(NexmarkUtils.PUBSUB_TIMESTAMP); + } + formattedResults.apply(io); + } + + /** + * Sink all raw Events in {@code source} to {@code options.getOutputPath}. + * This will configure the job to write the following files: + *

                + *
              • {@code $outputPath/event*.avro} All Event entities. + *
              • {@code $outputPath/auction*.avro} Auction entities. + *
              • {@code $outputPath/bid*.avro} Bid entities. + *
              • {@code $outputPath/person*.avro} Person entities. + *
              + * + * @param source A PCollection of events. + */ + private void sinkEventsToAvro(PCollection source) { + String filename = options.getOutputPath(); + if (Strings.isNullOrEmpty(filename)) { + throw new RuntimeException("Missing --outputPath"); + } + NexmarkUtils.console("Writing events to Avro files at %s", filename); + source.apply(AvroIO.Write.named(queryName + ".WriteAvroEvents") + .to(filename + "/event") + .withSuffix(".avro") + .withSchema(Event.class)); + source.apply(NexmarkQuery.JUST_BIDS) + .apply(AvroIO.Write.named(queryName + ".WriteAvroBids") + .to(filename + "/bid") + .withSuffix(".avro") + .withSchema(Bid.class)); + source.apply(NexmarkQuery.JUST_NEW_AUCTIONS) + .apply(AvroIO.Write.named( + queryName + ".WriteAvroAuctions") + .to(filename + "/auction") + .withSuffix(".avro") + .withSchema(Auction.class)); + source.apply(NexmarkQuery.JUST_NEW_PERSONS) + .apply(AvroIO.Write.named(queryName + ".WriteAvroPeople") + .to(filename + "/person") + .withSuffix(".avro") + .withSchema(Person.class)); + } + + /** + * Send {@code formattedResults} to text files. + */ + private void sinkResultsToText(PCollection formattedResults, long now) { + String filename = textFilename(now); + NexmarkUtils.console("Writing results to text files at %s", filename); + formattedResults.apply( + TextIO.Write.named(queryName + ".WriteTextResults") + .to(filename)); + } + + private static class StringToTableRow extends DoFn { + @Override + public void processElement(ProcessContext c) { + int n = ThreadLocalRandom.current().nextInt(10); + List records = new ArrayList<>(n); + for (int i = 0; i < n; i++) { + records.add(new TableRow().set("index", i).set("value", Integer.toString(i))); + } + c.output(new TableRow().set("result", c.element()).set("records", records)); + } + } + + /** + * Send {@code formattedResults} to BigQuery. + */ + private void sinkResultsToBigQuery( + PCollection formattedResults, long now, + String version) { + String tableSpec = tableSpec(now, version); + TableSchema tableSchema = + new TableSchema().setFields(ImmutableList.of( + new TableFieldSchema().setName("result").setType("STRING"), + new TableFieldSchema().setName("records").setMode("REPEATED").setType("RECORD") + .setFields(ImmutableList.of( + new TableFieldSchema().setName("index").setType("INTEGER"), + new TableFieldSchema().setName("value").setType("STRING"))))); + NexmarkUtils.console("Writing results to BigQuery table %s", tableSpec); + BigQueryIO.Write.Bound io = + BigQueryIO.Write.named(queryName + ".WriteBigQueryResults") + .to(tableSpec) + .withSchema(tableSchema) + .withCreateDisposition(BigQueryIO.Write.CreateDisposition.CREATE_IF_NEEDED) + .withWriteDisposition(BigQueryIO.Write.WriteDisposition.WRITE_APPEND); + formattedResults + .apply(ParDo.named(queryName + ".StringToTableRow") + .of(new StringToTableRow())) + .apply(io); + } + + // ================================================================================ + // Construct overall pipeline + // ================================================================================ + + /** + * Return source of events for this run, or null if we are simply publishing events + * to Pubsub. + */ + private PCollection createSource(Pipeline p, final long now) { + PCollection source = null; + switch (configuration.sourceType) { + case DIRECT: + source = sourceEventsFromSynthetic(p); + break; + case AVRO: + source = sourceEventsFromAvro(p); + break; + case PUBSUB: + // Setup the sink for the publisher. + switch (configuration.pubSubMode) { + case SUBSCRIBE_ONLY: + // Nothing to publish. + break; + case PUBLISH_ONLY: + // Send synthesized events to Pubsub in this job. + sinkEventsToPubsub(sourceEventsFromSynthetic(p).apply(NexmarkUtils.snoop(queryName)), + now); + break; + case COMBINED: + // Send synthesized events to Pubsub in separate publisher job. + // We won't start the main pipeline until the publisher has sent the pre-load events. + // We'll shutdown the publisher job when we notice the main job has finished. + invokeBuilderForPublishOnlyPipeline(new PipelineBuilder() { + @Override + public void build(Options publishOnlyOptions) { + Pipeline sp = Pipeline.create(options); + NexmarkUtils.setupPipeline(configuration.coderStrategy, sp); + publisherMonitor = new Monitor(queryName, "publisher"); + sinkEventsToPubsub( + sourceEventsFromSynthetic(sp).apply(publisherMonitor.getTransform()), + now); + publisherResult = sp.run(); + } + }); + break; + } + + // Setup the source for the consumer. + switch (configuration.pubSubMode) { + case PUBLISH_ONLY: + // Nothing to consume. Leave source null. + break; + case SUBSCRIBE_ONLY: + case COMBINED: + // Read events from pubsub. + source = sourceEventsFromPubsub(p, now); + break; + } + break; + } + return source; + } + + private static final TupleTag MAIN = new TupleTag(){}; + private static final TupleTag SIDE = new TupleTag(){}; + + private static class PartitionDoFn extends DoFn { + @Override + public void processElement(ProcessContext c) { + if (c.element().hashCode() % 2 == 0) { + c.output(c.element()); + } else { + c.sideOutput(SIDE, c.element()); + } + } + } + + /** + * Consume {@code results}. + */ + private void sink(PCollection> results, long now) { + if (configuration.sinkType == NexmarkUtils.SinkType.COUNT_ONLY) { + // Avoid the cost of formatting the results. + results.apply(NexmarkUtils.devNull(queryName)); + return; + } + + PCollection formattedResults = results.apply(NexmarkUtils.format(queryName)); + if (options.getLogResults()) { + formattedResults = formattedResults.apply(NexmarkUtils.log(queryName + ".Results")); + } + + switch (configuration.sinkType) { + case DEVNULL: + // Discard all results + formattedResults.apply(NexmarkUtils.devNull(queryName)); + break; + case PUBSUB: + sinkResultsToPubsub(formattedResults, now); + break; + case TEXT: + sinkResultsToText(formattedResults, now); + break; + case AVRO: + NexmarkUtils.console( + "WARNING: with --sinkType=AVRO, actual query results will be discarded."); + break; + case BIGQUERY: + // Multiple BigQuery backends to mimic what most customers do. + PCollectionTuple res = formattedResults.apply( + ParDo.named(queryName + ".Partition") + .withOutputTags(MAIN, TupleTagList.of(SIDE)) + .of(new PartitionDoFn())); + sinkResultsToBigQuery(res.get(MAIN), now, "main"); + sinkResultsToBigQuery(res.get(SIDE), now, "side"); + sinkResultsToBigQuery(formattedResults, now, "copy"); + break; + case COUNT_ONLY: + // Short-circuited above. + throw new RuntimeException(); + } + } + + // ================================================================================ + // Entry point + // ================================================================================ + + /** + * Calculate the distribution of the expected rate of results per minute (in event time, not + * wallclock time). + */ + private void modelResultRates(NexmarkQueryModel model) { + List counts = Lists.newArrayList(model.simulator().resultsPerWindow()); + Collections.sort(counts); + int n = counts.size(); + if (n < 5) { + NexmarkUtils.console("Query%d: only %d samples", model.configuration.query, n); + } else { + NexmarkUtils.console("Query%d: N:%d; min:%d; 1st%%:%d; mean:%d; 3rd%%:%d; max:%d", + model.configuration.query, n, counts.get(0), counts.get(n / 4), + counts.get(n / 2), + counts.get(n - 1 - n / 4), counts.get(n - 1)); + } + } + + /** + * Run {@code configuration} and return its performance if possible. + */ + @Nullable + public NexmarkPerf run(NexmarkConfiguration runConfiguration) { + if (options.getMonitorJobs() && !canMonitor()) { + throw new RuntimeException("Cannot use --monitorJobs with this runner since it does not " + + "support monitoring."); + } + if (options.getManageResources() && !options.getMonitorJobs()) { + throw new RuntimeException("If using --manageResources then must also use --monitorJobs."); + } + + // + // Setup per-run state. + // + Preconditions.checkState(configuration == null); + Preconditions.checkState(pubsub == null); + Preconditions.checkState(queryName == null); + configuration = runConfiguration; + + // GCS URI patterns to delete on exit. + List pathsToDelete = new ArrayList<>(); + + try { + NexmarkUtils.console("Running %s", configuration.toShortString()); + + if (configuration.numEvents < 0) { + NexmarkUtils.console("skipping since configuration is disabled"); + return null; + } + + List queries = Arrays.asList(new Query0(configuration), + new Query1(configuration), + new Query2(configuration), + new Query3(configuration), + new Query4(configuration), + new Query5(configuration), + new Query6(configuration), + new Query7(configuration), + new Query8(configuration), + new Query9(configuration), + new Query10(configuration), + new Query11(configuration), + new Query12(configuration)); + NexmarkQuery query = queries.get(configuration.query); + queryName = query.getName(); + + List models = Arrays.asList( + new Query0Model(configuration), + new Query1Model(configuration), + new Query2Model(configuration), + new Query3Model(configuration), + new Query4Model(configuration), + new Query5Model(configuration), + new Query6Model(configuration), + new Query7Model(configuration), + new Query8Model(configuration), + new Query9Model(configuration), + null, + null, + null); + NexmarkQueryModel model = models.get(configuration.query); + + if (options.getJustModelResultRate()) { + if (model == null) { + throw new RuntimeException(String.format("No model for %s", queryName)); + } + modelResultRates(model); + return null; + } + + long now = System.currentTimeMillis(); + Pipeline p = Pipeline.create(options); + NexmarkUtils.setupPipeline(configuration.coderStrategy, p); + + // Generate events. + PCollection source = createSource(p, now); + + if (options.getLogEvents()) { + source = source.apply(NexmarkUtils.log(queryName + ".Events")); + } + + // Source will be null if source type is PUBSUB and mode is PUBLISH_ONLY. + // In that case there's nothing more to add to pipeline. + if (source != null) { + // Optionally sink events in Avro format. + // (Query results are ignored). + if (configuration.sinkType == NexmarkUtils.SinkType.AVRO) { + sinkEventsToAvro(source); + } + + // Special hacks for Query 10 (big logger). + if (configuration.query == 10) { + String path = null; + if (options.getOutputPath() != null && !options.getOutputPath().isEmpty()) { + path = logsDir(now); + } + ((Query10) query).setOutputPath(path); + ((Query10) query).setMaxNumWorkers(maxNumWorkers()); + if (path != null && options.getManageResources()) { + pathsToDelete.add(path + "/**"); + } + } + + // Apply query. + PCollection> results = source.apply(query); + + if (options.getAssertCorrectness()) { + if (model == null) { + throw new RuntimeException(String.format("No model for %s", queryName)); + } + // We know all our streams have a finite number of elements. + results.setIsBoundedInternal(PCollection.IsBounded.BOUNDED); + // If we have a finite number of events then assert our pipeline's + // results match those of a model using the same sequence of events. + PAssert.that(results).satisfies(model.assertionFor()); + } + + // Output results. + sink(results, now); + } + + if (publisherResult != null) { + waitForPublisherPreload(); + } + mainResult = p.run(); + return monitor(query); + } finally { + // + // Cleanup per-run state. + // + if (pubsub != null) { + // Delete any subscriptions and topics we created. + pubsub.close(); + pubsub = null; + } + configuration = null; + queryName = null; + // TODO: Cleanup pathsToDelete + } + } +} diff --git a/integration/java/src/main/java/org/apache/beam/integration/nexmark/NexmarkSuite.java b/integration/java/src/main/java/org/apache/beam/integration/nexmark/NexmarkSuite.java new file mode 100644 index 0000000000000..cccaeb146723c --- /dev/null +++ b/integration/java/src/main/java/org/apache/beam/integration/nexmark/NexmarkSuite.java @@ -0,0 +1,112 @@ +/* + * 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.integration.nexmark; + +import java.util.ArrayList; +import java.util.LinkedHashSet; +import java.util.List; +import java.util.Set; + +/** + * A set of {@link NexmarkConfiguration}s. + */ +public enum NexmarkSuite { + /** + * The default. + */ + DEFAULT(defaultConf()), + + /** + * Sweep through all 11 queries using the default configuration. + * 100k/10k events (depending on query). + */ + SMOKE(smoke()), + + /** + * As for SMOKE, but with 10m/1m events. + */ + STRESS(stress()), + + /** + * As for SMOKE, but with 1b/100m events. + */ + FULL_THROTTLE(fullThrottle()); + + private static List defaultConf() { + List configurations = new ArrayList<>(); + configurations.add(new NexmarkConfiguration()); + return configurations; + } + + private static List smoke() { + List configurations = new ArrayList<>(); + for (int query = 0; query <= 12; query++) { + NexmarkConfiguration configuration = NexmarkConfiguration.DEFAULT.clone(); + configuration.query = query; + configuration.numEvents = 100_000; + if (query == 4 || query == 6 || query == 9) { + // Scale back so overall runtimes are reasonably close across all queries. + configuration.numEvents /= 10; + } + configurations.add(configuration); + } + return configurations; + } + + private static List stress() { + List configurations = smoke(); + for (NexmarkConfiguration configuration : configurations) { + if (configuration.numEvents >= 0) { + configuration.numEvents *= 1000; + } + } + return configurations; + } + + private static List fullThrottle() { + List configurations = smoke(); + for (NexmarkConfiguration configuration : configurations) { + if (configuration.numEvents >= 0) { + configuration.numEvents *= 1000; + } + } + return configurations; + } + + private final List configurations; + + NexmarkSuite(List configurations) { + this.configurations = configurations; + } + + /** + * Return the configurations corresponding to this suite. We'll override each configuration + * with any set command line flags, except for --isStreaming which is only respected for + * the {@link #DEFAULT} suite. + */ + public Iterable getConfigurations(Options options) { + Set results = new LinkedHashSet<>(); + for (NexmarkConfiguration configuration : configurations) { + NexmarkConfiguration result = configuration.clone(); + result.overrideFromOptions(options); + results.add(result); + } + return results; + } +} diff --git a/integration/java/src/main/java/org/apache/beam/integration/nexmark/NexmarkUtils.java b/integration/java/src/main/java/org/apache/beam/integration/nexmark/NexmarkUtils.java new file mode 100644 index 0000000000000..13ed5808617d5 --- /dev/null +++ b/integration/java/src/main/java/org/apache/beam/integration/nexmark/NexmarkUtils.java @@ -0,0 +1,681 @@ +/* + * 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.integration.nexmark; + +import org.apache.beam.sdk.Pipeline; +import org.apache.beam.sdk.coders.AvroCoder; +import org.apache.beam.sdk.coders.ByteArrayCoder; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.coders.CoderException; +import org.apache.beam.sdk.coders.CoderRegistry; +import org.apache.beam.sdk.coders.CustomCoder; +import org.apache.beam.sdk.coders.SerializableCoder; +import org.apache.beam.sdk.io.Read; +import org.apache.beam.sdk.runners.DirectPipelineRunner; +import org.apache.beam.sdk.runners.PipelineRunner; +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.PTransform; +import org.apache.beam.sdk.transforms.ParDo; +import org.apache.beam.sdk.transforms.Sum.SumLongFn; +import org.apache.beam.sdk.transforms.windowing.AfterPane; +import org.apache.beam.sdk.transforms.windowing.BoundedWindow; +import org.apache.beam.sdk.transforms.windowing.GlobalWindows; +import org.apache.beam.sdk.transforms.windowing.Window; +import org.apache.beam.sdk.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.values.PCollection; +import org.apache.beam.sdk.values.PInput; +import org.apache.beam.sdk.values.TimestampedValue; +import com.google.common.collect.ImmutableList; +import com.google.common.hash.Hashing; + +import com.fasterxml.jackson.databind.ObjectMapper; + +import org.joda.time.Duration; +import org.joda.time.Instant; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.nio.charset.StandardCharsets; +import java.util.Iterator; +import java.util.List; + +/** + * Odd's 'n Ends used throughout queries and driver. + */ +public class NexmarkUtils { + private static final Logger LOG = LoggerFactory.getLogger(NexmarkGoogleDriver.class.getName()); + + /** + * Mapper for (de)serializing JSON. + */ + static final ObjectMapper MAPPER = new ObjectMapper(); + + /** + * Possible sources for events. + */ + public enum SourceType { + /** + * Produce events directly. + */ + DIRECT, + /** + * Read events from an Avro file. + */ + AVRO, + /** + * Read from a PubSub topic. It will be fed the same synthetic events by this pipeline. + */ + PUBSUB + } + + /** + * Possible sinks for query results. + */ + public enum SinkType { + /** + * Discard all results. + */ + COUNT_ONLY, + /** + * Discard all results after converting them to strings. + */ + DEVNULL, + /** + * Write to a PubSub topic. It will be drained by this pipeline. + */ + PUBSUB, + /** + * Write to a text file. Only works in batch mode. + */ + TEXT, + /** + * Write raw Events to Avro. Only works in batch mode. + */ + AVRO, + /** + * Write raw Events to BigQuery. + */ + BIGQUERY, + } + + /** + * Pub/sub mode to run in. + */ + public enum PubSubMode { + /** + * Publish events to pub/sub, but don't run the query. + */ + PUBLISH_ONLY, + /** + * Consume events from pub/sub and run the query, but don't publish. + */ + SUBSCRIBE_ONLY, + /** + * Both publish and consume, but as separate jobs. + */ + COMBINED + } + + /** + * Coder strategies. + */ + public enum CoderStrategy { + /** + * Hand-written. + */ + HAND, + /** + * Avro. + */ + AVRO, + /** + * Java serialization. + */ + JAVA + } + + /** + * How to determine resource names. + */ + public enum ResourceNameMode { + /** Names are used as provided. */ + VERBATIM, + /** Names are suffixed with the query being run. */ + QUERY, + /** Names are suffixed with the query being run and a random number. */ + QUERY_AND_SALT; + } + + /** + * Units for rates. + */ + public enum RateUnit { + PER_SECOND(1_000_000L), + PER_MINUTE(60_000_000L); + + RateUnit(long usPerUnit) { + this.usPerUnit = usPerUnit; + } + + /** + * Number of microseconds per unit. + */ + private final long usPerUnit; + + /** + * Number of microseconds between events at given rate. + */ + public long rateToPeriodUs(long rate) { + return (usPerUnit + rate / 2) / rate; + } + } + + /** + * Shape of event rate. + */ + public static enum RateShape { + SQUARE, + SINE; + + /** + * Number of steps used to approximate sine wave. + */ + private static final int N = 10; + + /** + * Return inter-event delay, in microseconds, for each generator + * to follow in order to achieve {@code rate} at {@code unit} using {@code numGenerators}. + */ + public long interEventDelayUs(int rate, RateUnit unit, int numGenerators) { + return unit.rateToPeriodUs(rate) * numGenerators; + } + + /** + * Return array of successive inter-event delays, in microseconds, for each generator + * to follow in order to achieve this shape with {@code firstRate/nextRate} at + * {@code unit} using {@code numGenerators}. + */ + public long[] interEventDelayUs( + int firstRate, int nextRate, RateUnit unit, int numGenerators) { + if (firstRate == nextRate) { + long[] interEventDelayUs = new long[1]; + interEventDelayUs[0] = unit.rateToPeriodUs(firstRate) * numGenerators; + return interEventDelayUs; + } + + switch (this) { + case SQUARE: { + long[] interEventDelayUs = new long[2]; + interEventDelayUs[0] = unit.rateToPeriodUs(firstRate) * numGenerators; + interEventDelayUs[1] = unit.rateToPeriodUs(nextRate) * numGenerators; + return interEventDelayUs; + } + case SINE: { + double mid = (firstRate + nextRate) / 2.0; + double amp = (firstRate - nextRate) / 2.0; // may be -ve + long[] interEventDelayUs = new long[N]; + for (int i = 0; i < N; i++) { + double r = (2.0 * Math.PI * i) / N; + double rate = mid + amp * Math.cos(r); + interEventDelayUs[i] = unit.rateToPeriodUs(Math.round(rate)) * numGenerators; + } + return interEventDelayUs; + } + } + throw new RuntimeException(); // switch should be exhaustive + } + + /** + * Return delay between steps, in seconds, for result of {@link #interEventDelayUs}, so + * as to cycle through the entire sequence every {@code ratePeriodSec}. + */ + public int stepLengthSec(int ratePeriodSec) { + int n = 0; + switch (this) { + case SQUARE: + n = 2; + break; + case SINE: + n = N; + break; + } + return (ratePeriodSec + n - 1) / n; + } + } + + /** + * Set to true to capture all info messages. The logging level flags don't currently work. + */ + private static final boolean LOG_INFO = false; + + /** + * Set to true to capture all error messages. The logging level flags don't currently work. + */ + private static final boolean LOG_ERROR = true; + + /** + * Set to true to log directly to stdout on VM. You can watch the results in real-time with: + * tail -f /var/log/dataflow/streaming-harness/harness-stdout.log + */ + private static final boolean LOG_TO_CONSOLE = false; + + /** + * Log info message. + */ + public static void info(String format, Object... args) { + if (LOG_INFO) { + LOG.info(String.format(format, args)); + if (LOG_TO_CONSOLE) { + System.out.println(String.format(format, args)); + } + } + } + + /** + * Log error message. + */ + public static void error(String format, Object... args) { + if (LOG_ERROR) { + LOG.error(String.format(format, args)); + if (LOG_TO_CONSOLE) { + System.out.println(String.format(format, args)); + } + } + } + + /** + * Log message to console. For client side only. + */ + public static void console(String format, Object... args) { + System.out.printf("%s %s\n", Instant.now(), String.format(format, args)); + } + + /** + * Label to use for timestamps on pub/sub messages. + */ + public static final String PUBSUB_TIMESTAMP = "timestamp"; + + /** + * Label to use for windmill ids on pub/sub messages. + */ + public static final String PUBSUB_ID = "id"; + + /** + * All events will be given a timestamp relative to this time (ms since epoch). + */ + public static final long BASE_TIME = Instant.parse("2015-07-15T00:00:00.000Z").getMillis(); + + /** + * Instants guaranteed to be strictly before and after all event timestamps, and which won't + * be subject to underflow/overflow. + */ + public static final Instant BEGINNING_OF_TIME = new Instant(0).plus(Duration.standardDays(365)); + public static final Instant END_OF_TIME = + BoundedWindow.TIMESTAMP_MAX_VALUE.minus(Duration.standardDays(365)); + + /** + * Setup pipeline with codes and some other options. + */ + public static void setupPipeline(CoderStrategy coderStrategy, Pipeline p) { + PipelineRunner runner = p.getRunner(); + if (runner instanceof DirectPipelineRunner) { + // Disable randomization of output since we want to check batch and streaming match the + // model both locally and on the cloud. + ((DirectPipelineRunner) runner).withUnorderednessTesting(false); + } + + CoderRegistry registry = p.getCoderRegistry(); + switch (coderStrategy) { + case HAND: + registry.registerCoder(Auction.class, Auction.CODER); + registry.registerCoder(AuctionBid.class, AuctionBid.CODER); + registry.registerCoder(AuctionCount.class, AuctionCount.CODER); + registry.registerCoder(AuctionPrice.class, AuctionPrice.CODER); + registry.registerCoder(Bid.class, Bid.CODER); + registry.registerCoder(CategoryPrice.class, CategoryPrice.CODER); + registry.registerCoder(Event.class, Event.CODER); + registry.registerCoder(IdNameReserve.class, IdNameReserve.CODER); + registry.registerCoder(NameCityStateId.class, NameCityStateId.CODER); + registry.registerCoder(Person.class, Person.CODER); + registry.registerCoder(SellerPrice.class, SellerPrice.CODER); + registry.registerCoder(Done.class, Done.CODER); + registry.registerCoder(BidsPerSession.class, BidsPerSession.CODER); + break; + case AVRO: + registry.setFallbackCoderProvider(AvroCoder.PROVIDER); + break; + case JAVA: + registry.setFallbackCoderProvider(SerializableCoder.PROVIDER); + break; + } + } + + /** + * Return a generator config to match the given {@code options}. + */ + public static GeneratorConfig standardGeneratorConfig(NexmarkConfiguration configuration) { + return new GeneratorConfig(configuration, + configuration.useWallclockEventTime ? System.currentTimeMillis() + : BASE_TIME, 0, + configuration.numEvents, 0); + } + + /** + * Return an iterator of events using the 'standard' generator config. + */ + public static Iterator> standardEventIterator( + NexmarkConfiguration configuration) { + return new Generator(standardGeneratorConfig(configuration)); + } + + /** + * Return a transform which yields a finite number of synthesized events generated + * as a batch. + */ + public static PTransform> batchEventsSource( + String name, NexmarkConfiguration configuration) { + return Read + .from(new BoundedEventSource( + NexmarkUtils.standardGeneratorConfig(configuration), configuration.numEventGenerators)) + .named(name + ".ReadBounded"); + } + + /** + * Return a transform which yields a finite number of synthesized events generated + * on-the-fly in real time. + */ + public static PTransform> streamEventsSource( + String name, NexmarkConfiguration configuration) { + return Read.from(new UnboundedEventSource(NexmarkUtils.standardGeneratorConfig(configuration), + configuration.numEventGenerators, + configuration.watermarkHoldbackSec, + configuration.isRateLimited)) + .named(name + ".ReadUnbounded"); + } + + /** + * Return a transform to pass-through events, but count them as they go by. + */ + public static ParDo.Bound snoop(final String name) { + return ParDo.named(name + ".Snoop") + .of(new DoFn() { + final Aggregator eventCounter = + createAggregator("events", new SumLongFn()); + final Aggregator newPersonCounter = + createAggregator("newPersons", new SumLongFn()); + final Aggregator newAuctionCounter = + createAggregator("newAuctions", new SumLongFn()); + final Aggregator bidCounter = + createAggregator("bids", new SumLongFn()); + final Aggregator endOfStreamCounter = + createAggregator("endOfStream", new SumLongFn()); + + @Override + public void processElement(ProcessContext c) { + eventCounter.addValue(1L); + if (c.element().newPerson != null) { + newPersonCounter.addValue(1L); + } else if (c.element().newAuction != null) { + newAuctionCounter.addValue(1L); + } else if (c.element().bid != null) { + bidCounter.addValue(1L); + } else { + endOfStreamCounter.addValue(1L); + } + info("%s snooping element %s", name, c.element()); + c.output(c.element()); + } + }); + } + + /** + * Return a transform to count and discard each element. + */ + public static ParDo.Bound devNull(String name) { + return ParDo.named(name + ".DevNull") + .of(new DoFn() { + final Aggregator discardCounter = + createAggregator("discarded", new SumLongFn()); + + @Override + public void processElement(ProcessContext c) { + discardCounter.addValue(1L); + } + }); + } + + /** + * Return a transform to log each element, passing it through unchanged. + */ + public static ParDo.Bound log(final String name) { + return ParDo.named(name + ".Log") + .of(new DoFn() { + @Override + public void processElement(ProcessContext c) { + error("%s: %s", name, c.element()); + c.output(c.element()); + } + }); + } + + /** + * Return a transform to format each element as a string. + */ + public static ParDo.Bound format(String name) { + return ParDo.named(name + ".Format") + .of(new DoFn() { + final Aggregator recordCounter = + createAggregator("records", new SumLongFn()); + + @Override + public void processElement(ProcessContext c) { + recordCounter.addValue(1L); + c.output(c.element().toString()); + } + }); + } + + /** + * Return a transform to make explicit the timestamp of each element. + */ + public static ParDo.Bound> stamp(String name) { + return ParDo.named(name + ".Stamp") + .of(new DoFn>() { + @Override + public void processElement(ProcessContext c) { + c.output(TimestampedValue.of(c.element(), c.timestamp())); + } + }); + } + + /** + * Return a transform to reduce a stream to a single, order-invariant long hash. + */ + public static PTransform, PCollection> hash( + final long numEvents, String name) { + return new PTransform, PCollection>(name) { + @Override + public PCollection apply(PCollection input) { + return input.apply(Window.into(new GlobalWindows()) + .triggering(AfterPane.elementCountAtLeast((int) numEvents)) + .withAllowedLateness(Duration.standardDays(1)) + .discardingFiredPanes()) + + .apply(ParDo.named(name + ".Hash").of(new DoFn() { + @Override + public void processElement(ProcessContext c) { + long hash = + Hashing.murmur3_128() + .newHasher() + .putLong(c.timestamp().getMillis()) + .putString(c.element().toString(), StandardCharsets.UTF_8) + .hash() + .asLong(); + c.output(hash); + } + })) + + .apply(Combine.globally(new Combine.BinaryCombineFn() { + @Override + public Long apply(Long left, Long right) { + return left ^ right; + } + })); + } + }; + } + + private static final long MASK = (1L << 16) - 1L; + private static final long HASH = 0x243F6A8885A308D3L; + private static final long INIT_PLAINTEXT = 50000L; + + /** + * Return a transform to keep the CPU busy for given milliseconds on every record. + */ + public static ParDo.Bound cpuDelay(String name, final long delayMs) { + return ParDo.named(name + ".CpuDelay") + .of(new DoFn() { + @Override + public void processElement(ProcessContext c) { + long now = System.currentTimeMillis(); + long end = now + delayMs; + while (now < end) { + // Find plaintext which hashes to HASH in lowest MASK bits. + // Values chosen to roughly take 1ms on typical workstation. + long p = INIT_PLAINTEXT; + while (true) { + long t = Hashing.murmur3_128().hashLong(p).asLong(); + if ((t & MASK) == (HASH & MASK)) { + break; + } + p++; + } + long next = System.currentTimeMillis(); + now = next; + } + c.output(c.element()); + } + }); + } + + private static final StateTag> DUMMY_TAG = + StateTags.value("dummy", ByteArrayCoder.of()); + private static final int MAX_BUFFER_SIZE = 1 << 24; + + /** + * Return a transform to write given number of bytes to durable store on every record. + */ + public static ParDo.Bound diskBusy(String name, final long bytes) { + return ParDo.named(name + ".DiskBusy") + .of(new DoFn() { + @Override + public void processElement(ProcessContext c) { + long remain = bytes; + long start = System.currentTimeMillis(); + long now = start; + while (remain > 0) { + long thisBytes = Math.min(remain, MAX_BUFFER_SIZE); + remain -= thisBytes; + byte[] arr = new byte[(int) thisBytes]; + for (int i = 0; i < thisBytes; i++) { + arr[i] = (byte) now; + } + ValueState state = c.windowingInternals().stateInternals().state( + StateNamespaces.global(), DUMMY_TAG); + state.write(arr); + now = System.currentTimeMillis(); + } + c.output(c.element()); + } + }); + } + + /** + * Return a transform to cast each element to {@link KnownSize}. + */ + private static ParDo.Bound castToKnownSize( + final String name) { + return ParDo.named(name + ".Forget") + .of(new DoFn() { + @Override + public void processElement(ProcessContext c) { + c.output(c.element()); + } + }); + } + + /** + * A coder for instances of {@code T} cast up to {@link KnownSize}. + * + * @param True type of object. + */ + private static class CastingCoder extends CustomCoder { + private final Coder trueCoder; + + public CastingCoder(Coder trueCoder) { + this.trueCoder = trueCoder; + } + + @Override + public void encode(KnownSize value, OutputStream outStream, Context context) + throws CoderException, IOException { + @SuppressWarnings("unchecked") + T typedValue = (T) value; + trueCoder.encode(typedValue, outStream, context); + } + + @Override + public KnownSize decode(InputStream inStream, Context context) + throws CoderException, IOException { + return trueCoder.decode(inStream, context); + } + + @Override + public List> getComponents() { + return ImmutableList.of(trueCoder); + } + } + + /** + * Return a coder for {@code KnownSize} that are known to be exactly of type {@code T}. + */ + private static Coder makeCastingCoder(Coder trueCoder) { + return new CastingCoder<>(trueCoder); + } + + /** + * Return {@code elements} as {@code KnownSize}s. + */ + public static PCollection castToKnownSize( + final String name, PCollection elements) { + return elements.apply(castToKnownSize(name)).setCoder(makeCastingCoder(elements.getCoder())); + } + + // Do not instantiate. + private NexmarkUtils() { + } +} diff --git a/integration/java/src/main/java/org/apache/beam/integration/nexmark/Options.java b/integration/java/src/main/java/org/apache/beam/integration/nexmark/Options.java new file mode 100644 index 0000000000000..4f5304d5a1599 --- /dev/null +++ b/integration/java/src/main/java/org/apache/beam/integration/nexmark/Options.java @@ -0,0 +1,360 @@ +/* + * 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.integration.nexmark; + +import org.apache.beam.sdk.options.Default; +import org.apache.beam.sdk.options.Description; +import org.apache.beam.sdk.options.PubsubOptions; +import javax.annotation.Nullable; + +/** + * Command line flags. + */ +public interface Options extends PubsubOptions { + @Description("Which suite to run. Default is to use command line arguments for one job.") + @Default.Enum("DEFAULT") + NexmarkSuite getSuite(); + + void setSuite(NexmarkSuite suite); + + @Description("If true, and using the DataflowPipelineRunner, monitor the jobs as they run.") + @Default.Boolean(false) + boolean getMonitorJobs(); + + void setMonitorJobs(boolean monitorJobs); + + @Description("Where the events come from.") + @Nullable + NexmarkUtils.SourceType getSourceType(); + + void setSourceType(NexmarkUtils.SourceType sourceType); + + @Description("Prefix for input files if using avro input") + @Nullable + String getInputPath(); + + void setInputPath(String inputPath); + + @Description("Where results go.") + @Nullable + NexmarkUtils.SinkType getSinkType(); + + void setSinkType(NexmarkUtils.SinkType sinkType); + + @Description("Which mode to run in when source is PUBSUB.") + @Nullable + NexmarkUtils.PubSubMode getPubSubMode(); + + void setPubSubMode(NexmarkUtils.PubSubMode pubSubMode); + + @Description("Which query to run.") + @Nullable + Integer getQuery(); + + void setQuery(Integer query); + + @Description("Prefix for output files if using text output for results or running Query 10.") + @Nullable + String getOutputPath(); + + void setOutputPath(String outputPath); + + @Description("Base name of pubsub topic to publish to in streaming mode.") + @Nullable + @Default.String("nexmark") + String getPubsubTopic(); + + void setPubsubTopic(String pubsubTopic); + + @Description("Base name of pubsub subscription to read from in streaming mode.") + @Nullable + @Default.String("nexmark") + String getPubsubSubscription(); + + void setPubsubSubscription(String pubsubSubscription); + + @Description("Base name of BigQuery table name if using BigQuery output.") + @Nullable + @Default.String("nexmark") + String getBigQueryTable(); + + void setBigQueryTable(String bigQueryTable); + + @Description("Approximate number of events to generate. " + + "Zero for effectively unlimited in streaming mode.") + @Nullable + Long getNumEvents(); + + void setNumEvents(Long numEvents); + + @Description("Time in seconds to preload the subscription with data, at the initial input rate " + + "of the pipeline.") + @Nullable + Integer getPreloadSeconds(); + + void setPreloadSeconds(Integer preloadSeconds); + + @Description("Number of unbounded sources to create events.") + @Nullable + Integer getNumEventGenerators(); + + void setNumEventGenerators(Integer numEventGenerators); + + @Description("Shape of event rate curve.") + @Nullable + NexmarkUtils.RateShape getRateShape(); + + void setRateShape(NexmarkUtils.RateShape rateShape); + + @Description("Initial overall event rate (in --rateUnit).") + @Nullable + Integer getFirstEventRate(); + + void setFirstEventRate(Integer firstEventRate); + + @Description("Next overall event rate (in --rateUnit).") + @Nullable + Integer getNextEventRate(); + + void setNextEventRate(Integer nextEventRate); + + @Description("Unit for rates.") + @Nullable + NexmarkUtils.RateUnit getRateUnit(); + + void setRateUnit(NexmarkUtils.RateUnit rateUnit); + + @Description("Overall period of rate shape, in seconds.") + @Nullable + Integer getRatePeriodSec(); + + void setRatePeriodSec(Integer ratePeriodSec); + + @Description("If true, relay events in real time in streaming mode.") + @Nullable + Boolean getIsRateLimited(); + + void setIsRateLimited(Boolean isRateLimited); + + @Description("If true, use wallclock time as event time. Otherwise, use a deterministic" + + " time in the past so that multiple runs will see exactly the same event streams" + + " and should thus have exactly the same results.") + @Nullable + Boolean getUseWallclockEventTime(); + + void setUseWallclockEventTime(Boolean useWallclockEventTime); + + @Description("Assert pipeline results match model results.") + @Nullable + boolean getAssertCorrectness(); + + void setAssertCorrectness(boolean assertCorrectness); + + @Description("Log all input events.") + @Nullable + boolean getLogEvents(); + + void setLogEvents(boolean logEvents); + + @Description("Log all query results.") + @Nullable + boolean getLogResults(); + + void setLogResults(boolean logResults); + + @Description("Average size in bytes for a person record.") + @Nullable + Integer getAvgPersonByteSize(); + + void setAvgPersonByteSize(Integer avgPersonByteSize); + + @Description("Average size in bytes for an auction record.") + @Nullable + Integer getAvgAuctionByteSize(); + + void setAvgAuctionByteSize(Integer avgAuctionByteSize); + + @Description("Average size in bytes for a bid record.") + @Nullable + Integer getAvgBidByteSize(); + + void setAvgBidByteSize(Integer avgBidByteSize); + + @Description("Ratio of bids for 'hot' auctions above the background.") + @Nullable + Integer getHotAuctionRatio(); + + void setHotAuctionRatio(Integer hotAuctionRatio); + + @Description("Ratio of auctions for 'hot' sellers above the background.") + @Nullable + Integer getHotSellersRatio(); + + void setHotSellersRatio(Integer hotSellersRatio); + + @Description("Ratio of auctions for 'hot' bidders above the background.") + @Nullable + Integer getHotBiddersRatio(); + + void setHotBiddersRatio(Integer hotBiddersRatio); + + @Description("Window size in seconds.") + @Nullable + Long getWindowSizeSec(); + + void setWindowSizeSec(Long windowSizeSec); + + @Description("Window period in seconds.") + @Nullable + Long getWindowPeriodSec(); + + void setWindowPeriodSec(Long windowPeriodSec); + + @Description("If in streaming mode, the holdback for watermark in seconds.") + @Nullable + Long getWatermarkHoldbackSec(); + + void setWatermarkHoldbackSec(Long watermarkHoldbackSec); + + @Description("Roughly how many auctions should be in flight for each generator.") + @Nullable + Integer getNumInFlightAuctions(); + + void setNumInFlightAuctions(Integer numInFlightAuctions); + + + @Description("Maximum number of people to consider as active for placing auctions or bids.") + @Nullable + Integer getNumActivePeople(); + + void setNumActivePeople(Integer numActivePeople); + + @Description("Filename of perf data to append to.") + @Nullable + String getPerfFilename(); + + void setPerfFilename(String perfFilename); + + @Description("Filename of baseline perf data to read from.") + @Nullable + String getBaselineFilename(); + + void setBaselineFilename(String baselineFilename); + + @Description("Filename of summary perf data to append to.") + @Nullable + String getSummaryFilename(); + + void setSummaryFilename(String summaryFilename); + + @Description("Filename for javascript capturing all perf data and any baselines.") + @Nullable + String getJavascriptFilename(); + + void setJavascriptFilename(String javascriptFilename); + + @Description("If true, don't run the actual query. Instead, calculate the distribution " + + "of number of query results per (event time) minute according to the query model.") + @Nullable + boolean getJustModelResultRate(); + + void setJustModelResultRate(boolean justModelResultRate); + + @Description("Coder strategy to use.") + @Nullable + NexmarkUtils.CoderStrategy getCoderStrategy(); + + void setCoderStrategy(NexmarkUtils.CoderStrategy coderStrategy); + + @Description("Delay, in milliseconds, for each event. We will peg one core for this " + + "number of milliseconds to simulate CPU-bound computation.") + @Nullable + Long getCpuDelayMs(); + + void setCpuDelayMs(Long cpuDelayMs); + + @Description("Extra data, in bytes, to save to persistent state for each event. " + + "This will force I/O all the way to durable storage to simulate an " + + "I/O-bound computation.") + @Nullable + Long getDiskBusyBytes(); + + void setDiskBusyBytes(Long diskBusyBytes); + + @Description("Skip factor for query 2. We select bids for every {@code auctionSkip}'th auction") + @Nullable + Integer getAuctionSkip(); + + void setAuctionSkip(Integer auctionSkip); + + @Description("Fanout for queries 4 (groups by category id) and 7 (finds a global maximum).") + @Nullable + Integer getFanout(); + + void setFanout(Integer fanout); + + @Description("Length of occasional delay to impose on events (in seconds).") + @Nullable + Long getOccasionalDelaySec(); + + void setOccasionalDelaySec(Long occasionalDelaySec); + + @Description("Probability that an event will be delayed by delayS.") + @Nullable + Double getProbDelayedEvent(); + + void setProbDelayedEvent(Double probDelayedEvent); + + @Description("Maximum size of each log file (in events). For Query10 only.") + @Nullable + Integer getMaxLogEvents(); + + void setMaxLogEvents(Integer maxLogEvents); + + @Description("How to derive names of resources.") + @Default.Enum("QUERY_AND_SALT") + NexmarkUtils.ResourceNameMode getResourceNameMode(); + + void setResourceNameMode(NexmarkUtils.ResourceNameMode mode); + + @Description("If true, manage the creation and cleanup of topics, subscriptions and gcs files.") + @Default.Boolean(true) + boolean getManageResources(); + + void setManageResources(boolean manageResources); + + @Description("If true, use pub/sub publish time instead of event time.") + @Nullable + Boolean getUsePubsubPublishTime(); + + void setUsePubsubPublishTime(Boolean usePubsubPublishTime); + + @Description("Number of events in out-of-order groups. 1 implies no out-of-order events. " + + "1000 implies every 1000 events per generator are emitted in pseudo-random order.") + @Nullable + Long getOutOfOrderGroupSize(); + + void setOutOfOrderGroupSize(Long outOfOrderGroupSize); + + @Description("If false, do not add the Monitor and Snoop transforms.") + @Nullable + Boolean getDebug(); + + void setDebug(Boolean value); +} diff --git a/integration/java/src/main/java/org/apache/beam/integration/nexmark/Person.java b/integration/java/src/main/java/org/apache/beam/integration/nexmark/Person.java new file mode 100644 index 0000000000000..6fcf388b72724 --- /dev/null +++ b/integration/java/src/main/java/org/apache/beam/integration/nexmark/Person.java @@ -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. + */ + +package org.apache.beam.integration.nexmark; + +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.StringUtf8Coder; +import org.apache.beam.sdk.coders.VarLongCoder; + +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.core.JsonProcessingException; + +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.io.Serializable; + +/** + * A person either creating an auction or making a bid. + */ +public class Person implements KnownSize, Serializable { + private static final Coder LONG_CODER = VarLongCoder.of(); + private static final Coder STRING_CODER = StringUtf8Coder.of(); + public static final Coder CODER = new AtomicCoder() { + @Override + public void encode(Person value, OutputStream outStream, + Coder.Context context) + throws CoderException, IOException { + LONG_CODER.encode(value.id, outStream, Context.NESTED); + STRING_CODER.encode(value.name, outStream, Context.NESTED); + STRING_CODER.encode(value.emailAddress, outStream, Context.NESTED); + STRING_CODER.encode(value.creditCard, outStream, Context.NESTED); + STRING_CODER.encode(value.city, outStream, Context.NESTED); + STRING_CODER.encode(value.state, outStream, Context.NESTED); + LONG_CODER.encode(value.dateTime, outStream, Context.NESTED); + STRING_CODER.encode(value.extra, outStream, Context.NESTED); + } + + @Override + public Person decode( + InputStream inStream, Coder.Context context) + throws CoderException, IOException { + long id = LONG_CODER.decode(inStream, Context.NESTED); + String name = STRING_CODER.decode(inStream, Context.NESTED); + String emailAddress = STRING_CODER.decode(inStream, Context.NESTED); + String creditCard = STRING_CODER.decode(inStream, Context.NESTED); + String city = STRING_CODER.decode(inStream, Context.NESTED); + String state = STRING_CODER.decode(inStream, Context.NESTED); + long dateTime = LONG_CODER.decode(inStream, Context.NESTED); + String extra = STRING_CODER.decode(inStream, Context.NESTED); + return new Person(id, name, emailAddress, creditCard, city, state, dateTime, extra); + } + }; + + /** Id of person. */ + @JsonProperty + public final long id; // primary key + + /** Extra person properties. */ + @JsonProperty + public final String name; + + @JsonProperty + public final String emailAddress; + + @JsonProperty + public final String creditCard; + + @JsonProperty + public final String city; + + @JsonProperty + public final String state; + + @JsonProperty + public final long dateTime; + + /** Additional arbitrary payload for performance testing. */ + @JsonProperty + public final String extra; + + // For Avro only. + @SuppressWarnings("unused") + private Person() { + id = 0; + name = null; + emailAddress = null; + creditCard = null; + city = null; + state = null; + dateTime = 0; + extra = null; + } + + public Person(long id, String name, String emailAddress, String creditCard, String city, + String state, long dateTime, String extra) { + this.id = id; + this.name = name; + this.emailAddress = emailAddress; + this.creditCard = creditCard; + this.city = city; + this.state = state; + this.dateTime = dateTime; + this.extra = extra; + } + + /** + * Return a copy of person which capture the given annotation. + * (Used for debugging). + */ + public Person withAnnotation(String annotation) { + return new Person(id, name, emailAddress, creditCard, city, state, dateTime, + annotation + ": " + extra); + } + + /** + * Does person have {@code annotation}? (Used for debugging.) + */ + public boolean hasAnnotation(String annotation) { + return extra.startsWith(annotation + ": "); + } + + /** + * Remove {@code annotation} from person. (Used for debugging.) + */ + public Person withoutAnnotation(String annotation) { + if (hasAnnotation(annotation)) { + return new Person(id, name, emailAddress, creditCard, city, state, dateTime, + extra.substring(annotation.length() + 2)); + } else { + return this; + } + } + + @Override + public long sizeInBytes() { + return 8 + name.length() + 1 + emailAddress.length() + 1 + creditCard.length() + 1 + + city.length() + 1 + state.length() + 8 + 1 + extra.length() + 1; + } + + @Override + public String toString() { + try { + return NexmarkUtils.MAPPER.writeValueAsString(this); + } catch (JsonProcessingException e) { + throw new RuntimeException(e); + } + } +} diff --git a/integration/java/src/main/java/org/apache/beam/integration/nexmark/PubsubHelper.java b/integration/java/src/main/java/org/apache/beam/integration/nexmark/PubsubHelper.java new file mode 100644 index 0000000000000..1255154dc7676 --- /dev/null +++ b/integration/java/src/main/java/org/apache/beam/integration/nexmark/PubsubHelper.java @@ -0,0 +1,217 @@ +/* + * 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.integration.nexmark; + +import org.apache.beam.sdk.options.PubsubOptions; +import org.apache.beam.sdk.util.PubsubClient; +import org.apache.beam.sdk.util.PubsubJsonClient; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collection; +import java.util.List; + +/** + * Helper for working with pubsub. + */ +public class PubsubHelper implements AutoCloseable { + /** + * Underlying pub/sub client. + */ + private final PubsubClient pubsubClient; + + /** + * Project id. + */ + private final String projectId; + + /** + * Topics we should delete on close. + */ + private final List createdTopics; + + /** + * Subscriptions we should delete on close. + */ + private final List createdSubscriptions; + + private PubsubHelper(PubsubClient pubsubClient, String projectId) { + this.pubsubClient = pubsubClient; + this.projectId = projectId; + createdTopics = new ArrayList<>(); + createdSubscriptions = new ArrayList<>(); + } + + /** + * Create a helper. + */ + public static PubsubHelper create(PubsubOptions options) { + try { + return new PubsubHelper( + PubsubJsonClient.FACTORY.newClient(null, null, options), + options.getProject()); + } catch (IOException e) { + throw new RuntimeException("Unable to create Pubsub client: ", e); + } + } + + /** + * Create a topic from short name. Delete it if it already exists. Ensure the topic will be + * deleted on cleanup. Return full topic name. + */ + public PubsubClient.TopicPath createTopic(String shortTopic) { + PubsubClient.TopicPath topic = PubsubClient.topicPathFromName(projectId, shortTopic); + try { + if (topicExists(shortTopic)) { + NexmarkUtils.console("attempting to cleanup topic %s", topic); + pubsubClient.deleteTopic(topic); + } + NexmarkUtils.console("create topic %s", topic); + pubsubClient.createTopic(topic); + createdTopics.add(topic); + return topic; + } catch (IOException e) { + throw new RuntimeException("Unable to create Pubsub topic " + topic + ": ", e); + } + } + + /** + * Create a topic from short name if it does not already exist. The topic will not be + * deleted on cleanup. Return full topic name. + */ + public PubsubClient.TopicPath createOrReuseTopic(String shortTopic) { + PubsubClient.TopicPath topic = PubsubClient.topicPathFromName(projectId, shortTopic); + try { + if (topicExists(shortTopic)) { + NexmarkUtils.console("topic %s already exists", topic); + return topic; + } + NexmarkUtils.console("create topic %s", topic); + pubsubClient.createTopic(topic); + return topic; + } catch (IOException e) { + throw new RuntimeException("Unable to create or reuse Pubsub topic " + topic + ": ", e); + } + } + + /** + * Check a topic corresponding to short name exists, and throw exception if not. The + * topic will not be deleted on cleanup. Return full topic name. + */ + public PubsubClient.TopicPath reuseTopic(String shortTopic) { + PubsubClient.TopicPath topic = PubsubClient.topicPathFromName(projectId, shortTopic); + if (topicExists(shortTopic)) { + NexmarkUtils.console("reusing existing topic %s", topic); + return topic; + } + throw new RuntimeException("topic '" + topic + "' does not already exist"); + } + + /** + * Does topic corresponding to short name exist? + */ + public boolean topicExists(String shortTopic) { + PubsubClient.ProjectPath project = PubsubClient.projectPathFromId(projectId); + PubsubClient.TopicPath topic = PubsubClient.topicPathFromName(projectId, shortTopic); + try { + Collection existingTopics = pubsubClient.listTopics(project); + return existingTopics.contains(topic); + } catch (IOException e) { + throw new RuntimeException("Unable to check Pubsub topic " + topic + ": ", e); + } + } + + /** + * Create subscription from short name. Delete subscription if it already exists. Ensure the + * subscription will be deleted on cleanup. Return full subscription name. + */ + public PubsubClient.SubscriptionPath createSubscription( + String shortTopic, String shortSubscription) { + PubsubClient.TopicPath topic = PubsubClient.topicPathFromName(projectId, shortTopic); + PubsubClient.SubscriptionPath subscription = + PubsubClient.subscriptionPathFromName(projectId, shortSubscription); + try { + if (subscriptionExists(shortTopic, shortSubscription)) { + NexmarkUtils.console("attempting to cleanup subscription %s", subscription); + pubsubClient.deleteSubscription(subscription); + } + NexmarkUtils.console("create subscription %s", subscription); + pubsubClient.createSubscription(topic, subscription, 60); + createdSubscriptions.add(subscription); + } catch (IOException e) { + throw new RuntimeException("Unable to create Pubsub subscription " + subscription + ": ", e); + } + return subscription; + } + + /** + * Check a subscription corresponding to short name exists, and throw exception if not. The + * subscription will not be deleted on cleanup. Return full topic name. + */ + public PubsubClient.SubscriptionPath reuseSubscription( + String shortTopic, String shortSubscription) { + PubsubClient.SubscriptionPath subscription = + PubsubClient.subscriptionPathFromName(projectId, shortSubscription); + if (subscriptionExists(shortTopic, shortSubscription)) { + NexmarkUtils.console("reusing existing subscription %s", subscription); + return subscription; + } + throw new RuntimeException("subscription'" + subscription + "' does not already exist"); + } + + /** + * Does subscription corresponding to short name exist? + */ + public boolean subscriptionExists(String shortTopic, String shortSubscription) { + PubsubClient.ProjectPath project = PubsubClient.projectPathFromId(projectId); + PubsubClient.TopicPath topic = PubsubClient.topicPathFromName(projectId, shortTopic); + PubsubClient.SubscriptionPath subscription = + PubsubClient.subscriptionPathFromName(projectId, shortSubscription); + try { + Collection existingSubscriptions = + pubsubClient.listSubscriptions(project, topic); + return existingSubscriptions.contains(subscription); + } catch (IOException e) { + throw new RuntimeException("Unable to check Pubsub subscription" + subscription + ": ", e); + } + } + + /** + * Delete all the subscriptions and topics we created. + */ + @Override + public void close() { + for (PubsubClient.SubscriptionPath subscription : createdSubscriptions) { + try { + NexmarkUtils.console("delete subscription %s", subscription); + pubsubClient.deleteSubscription(subscription); + } catch (IOException ex) { + NexmarkUtils.console("could not delete subscription %s", subscription); + } + } + for (PubsubClient.TopicPath topic : createdTopics) { + try { + NexmarkUtils.console("delete topic %s", topic); + pubsubClient.deleteTopic(topic); + } catch (IOException ex) { + NexmarkUtils.console("could not delete topic %s", topic); + } + } + } +} diff --git a/integration/java/src/main/java/org/apache/beam/integration/nexmark/Query0.java b/integration/java/src/main/java/org/apache/beam/integration/nexmark/Query0.java new file mode 100644 index 0000000000000..ea0d7ca200668 --- /dev/null +++ b/integration/java/src/main/java/org/apache/beam/integration/nexmark/Query0.java @@ -0,0 +1,72 @@ +/* + * 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.integration.nexmark; + +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.coders.CoderException; +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.SumLongFn; +import org.apache.beam.sdk.values.PCollection; + + +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.IOException; + +/** + * Query 0: Pass events through unchanged. However, force them to do a round trip through + * serialization so that we measure the impact of the choice of coders. + */ +public class Query0 extends NexmarkQuery { + public Query0(NexmarkConfiguration configuration) { + super(configuration, "Query0"); + } + + private PCollection applyTyped(PCollection events) { + final Coder coder = events.getCoder(); + + return events + + // Force round trip through coder. + .apply( + ParDo.named(name + ".Serialize") + .of(new DoFn() { + private final Aggregator bytes = + createAggregator("bytes", new SumLongFn()); + + @Override + public void processElement(ProcessContext c) throws CoderException, IOException { + ByteArrayOutputStream outStream = new ByteArrayOutputStream(); + coder.encode(c.element(), outStream, Coder.Context.OUTER); + byte[] byteArray = outStream.toByteArray(); + bytes.addValue((long) byteArray.length); + ByteArrayInputStream inStream = new ByteArrayInputStream(byteArray); + Event event = coder.decode(inStream, Coder.Context.OUTER); + c.output(event); + } + })); + } + + @Override + protected PCollection applyPrim(PCollection events) { + return NexmarkUtils.castToKnownSize(name, applyTyped(events)); + } +} diff --git a/integration/java/src/main/java/org/apache/beam/integration/nexmark/Query0Model.java b/integration/java/src/main/java/org/apache/beam/integration/nexmark/Query0Model.java new file mode 100644 index 0000000000000..f3ceca267a05c --- /dev/null +++ b/integration/java/src/main/java/org/apache/beam/integration/nexmark/Query0Model.java @@ -0,0 +1,62 @@ +/* + * 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.integration.nexmark; + +import org.apache.beam.sdk.values.TimestampedValue; + +import java.util.Collection; +import java.util.Iterator; + +/** + * A direct implementation of {@link Query0}. + */ +public class Query0Model extends NexmarkQueryModel { + /** + * Simulator for query 0. + */ + private class Simulator extends AbstractSimulator { + public Simulator(NexmarkConfiguration configuration) { + super(NexmarkUtils.standardEventIterator(configuration)); + } + + @Override + protected void run() { + TimestampedValue timestampedEvent = nextInput(); + if (timestampedEvent == null) { + allDone(); + return; + } + addResult(timestampedEvent); + } + } + + public Query0Model(NexmarkConfiguration configuration) { + super(configuration); + } + + @Override + protected AbstractSimulator simulator() { + return new Simulator(configuration); + } + + @Override + protected Collection toCollection(Iterator> itr) { + return toValueTimestampOrder(itr); + } +} diff --git a/integration/java/src/main/java/org/apache/beam/integration/nexmark/Query1.java b/integration/java/src/main/java/org/apache/beam/integration/nexmark/Query1.java new file mode 100644 index 0000000000000..7e60b9c783dbb --- /dev/null +++ b/integration/java/src/main/java/org/apache/beam/integration/nexmark/Query1.java @@ -0,0 +1,64 @@ +/* + * 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.integration.nexmark; + +import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.transforms.ParDo; +import org.apache.beam.sdk.values.PCollection; + +/** + * Query 1, 'Currency Conversion'. Convert each bid value from dollars to euros. + * In CQL syntax: + * + *
              + * SELECT Istream(auction, DOLTOEUR(price), bidder, datetime)
              + * FROM bid [ROWS UNBOUNDED];
              + * 
              + * + *

              To make things more interesting, allow the 'currency conversion' to be arbitrarily + * slowed down. + */ +class Query1 extends NexmarkQuery { + public Query1(NexmarkConfiguration configuration) { + super(configuration, "Query1"); + } + + private PCollection applyTyped(PCollection events) { + return events + // Only want the bid events. + .apply(JUST_BIDS) + + // Map the conversion function over all bids. + .apply( + ParDo.named(name + ".ToEuros") + .of(new DoFn() { + @Override + public void processElement(ProcessContext c) { + Bid bid = c.element(); + c.output(new Bid( + bid.auction, bid.bidder, (bid.price * 89) / 100, bid.dateTime, bid.extra)); + } + })); + } + + @Override + protected PCollection applyPrim(PCollection events) { + return NexmarkUtils.castToKnownSize(name, applyTyped(events)); + } +} diff --git a/integration/java/src/main/java/org/apache/beam/integration/nexmark/Query10.java b/integration/java/src/main/java/org/apache/beam/integration/nexmark/Query10.java new file mode 100644 index 0000000000000..74fb28c3cc775 --- /dev/null +++ b/integration/java/src/main/java/org/apache/beam/integration/nexmark/Query10.java @@ -0,0 +1,378 @@ +/* + * 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.integration.nexmark; + +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.options.GcsOptions; +import org.apache.beam.sdk.transforms.Aggregator; +import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.transforms.DoFnWithContext; +import org.apache.beam.sdk.transforms.GroupByKey; +import org.apache.beam.sdk.transforms.ParDo; +import org.apache.beam.sdk.transforms.Sum.SumLongFn; +import org.apache.beam.sdk.transforms.windowing.AfterEach; +import org.apache.beam.sdk.transforms.windowing.AfterFirst; +import org.apache.beam.sdk.transforms.windowing.AfterPane; +import org.apache.beam.sdk.transforms.windowing.AfterProcessingTime; +import org.apache.beam.sdk.transforms.windowing.AfterWatermark; +import org.apache.beam.sdk.transforms.windowing.BoundedWindow; +import org.apache.beam.sdk.transforms.windowing.FixedWindows; +import org.apache.beam.sdk.transforms.windowing.PaneInfo; +import org.apache.beam.sdk.transforms.windowing.PaneInfo.Timing; +import org.apache.beam.sdk.transforms.windowing.Repeatedly; +import org.apache.beam.sdk.transforms.windowing.Window; +import org.apache.beam.sdk.util.GcsIOChannelFactory; +import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.PCollection; + +import com.google.cloud.hadoop.gcsio.GoogleCloudStorageWriteChannel; +import com.google.common.base.Preconditions; + +import org.joda.time.Duration; +import org.joda.time.Instant; + +import java.io.IOException; +import java.io.OutputStream; +import java.io.Serializable; +import java.nio.channels.Channels; +import java.nio.channels.WritableByteChannel; +import java.util.concurrent.ThreadLocalRandom; +import javax.annotation.Nullable; + +/** + * Query "10", 'Log to sharded files' (Not in original suite.) + * + *

              Every windowSizeSec, save all events from the last period into 2*maxWorkers log files. + */ +class Query10 extends NexmarkQuery { + private static final int CHANNEL_BUFFER = 8 << 20; // 8MB + private static final int NUM_SHARDS_PER_WORKER = 5; + private static final Duration LATE_BATCHING_PERIOD = Duration.standardSeconds(10); + + /** + * Capture everything we need to know about the records in a single output file. + */ + private static class OutputFile implements Serializable { + /** Maximum possible timestamp of records in file. */ + private final Instant maxTimestamp; + /** Shard within window. */ + private final String shard; + /** Index of file in all files in shard. */ + private final long index; + /** Timing of records in this file. */ + private final PaneInfo.Timing timing; + /** Path to file containing records, or {@literal null} if no output required. */ + @Nullable + private final String filename; + + public OutputFile( + Instant maxTimestamp, + String shard, + long index, + PaneInfo.Timing timing, + @Nullable String filename) { + this.maxTimestamp = maxTimestamp; + this.shard = shard; + this.index = index; + this.timing = timing; + this.filename = filename; + } + + @Override + public String toString() { + return String.format("%s %s %d %s %s\n", maxTimestamp, shard, index, timing, filename); + } + } + + /** + * GCS uri prefix for all log and 'finished' files. If null they won't be written. + */ + @Nullable + private String outputPath; + + /** + * Maximum number of workers, used to determine log sharding factor. + */ + private int maxNumWorkers; + + public Query10(NexmarkConfiguration configuration) { + super(configuration, "Query10"); + } + + public void setOutputPath(@Nullable String outputPath) { + this.outputPath = outputPath; + } + + public void setMaxNumWorkers(int maxNumWorkers) { + this.maxNumWorkers = maxNumWorkers; + } + + /** + * Return channel for writing bytes to GCS. + * + * @throws IOException + */ + private WritableByteChannel openWritableGcsFile(GcsOptions options, String filename) + throws IOException { + WritableByteChannel channel = new GcsIOChannelFactory(options).create(filename, "text/plain"); + Preconditions.checkState(channel instanceof GoogleCloudStorageWriteChannel); + ((GoogleCloudStorageWriteChannel) channel).setUploadBufferSize(CHANNEL_BUFFER); + return channel; + } + + /** Return a short string to describe {@code timing}. */ + private String timingToString(PaneInfo.Timing timing) { + switch (timing) { + case EARLY: + return "E"; + case ON_TIME: + return "O"; + case LATE: + return "L"; + } + throw new RuntimeException(); // cases are exhaustive + } + + /** Construct an {@link OutputFile} for {@code pane} in {@code window} for {@code shard}. */ + private OutputFile outputFileFor(BoundedWindow window, String shard, PaneInfo pane) { + @Nullable String filename = + outputPath == null + ? null + : String.format("%s/LOG-%s-%s-%03d-%s-%x", + outputPath, window.maxTimestamp(), shard, pane.getIndex(), + timingToString(pane.getTiming()), + ThreadLocalRandom.current().nextLong()); + return new OutputFile(window.maxTimestamp(), shard, pane.getIndex(), + pane.getTiming(), filename); + } + + /** + * Return path to which we should write the index for {@code window}, or {@literal null} + * if no output required. + */ + @Nullable + private String indexPathFor(BoundedWindow window) { + if (outputPath == null) { + return null; + } + return String.format("%s/INDEX-%s", outputPath, window.maxTimestamp()); + } + + private PCollection applyTyped(PCollection events) { + final int numLogShards = maxNumWorkers * NUM_SHARDS_PER_WORKER; + + return events + .apply(ParDo.named(name + ".ShardEvents") + .of(new DoFn>() { + final Aggregator lateCounter = + createAggregator("actuallyLateEvent", new SumLongFn()); + final Aggregator onTimeCounter = + createAggregator("actuallyOnTimeEvent", new SumLongFn()); + + @Override + public void processElement(ProcessContext c) { + if (c.element().hasAnnotation("LATE")) { + lateCounter.addValue(1L); + NexmarkUtils.error("Observed late: %s", c.element()); + } else { + onTimeCounter.addValue(1L); + } + int shardNum = (int) Math.abs((long) c.element().hashCode() % numLogShards); + String shard = String.format("shard-%05d-of-%05d", shardNum, numLogShards); + c.output(KV.of(shard, c.element())); + } + })) + .apply(Window.>into( + FixedWindows.of(Duration.standardSeconds(configuration.windowSizeSec))) + .named(name + ".WindowEvents") + .triggering(AfterEach.inOrder( + Repeatedly + .forever(AfterPane.elementCountAtLeast(configuration.maxLogEvents)) + .orFinally(AfterWatermark.pastEndOfWindow()), + Repeatedly.forever( + AfterFirst.of(AfterPane.elementCountAtLeast(configuration.maxLogEvents), + AfterProcessingTime.pastFirstElementInPane() + .plusDelayOf(LATE_BATCHING_PERIOD))))) + .discardingFiredPanes() + // Use a 1 day allowed lateness so that any forgotten hold will stall the + // pipeline for that period and be very noticeable. + .withAllowedLateness(Duration.standardDays(1))) + .apply(GroupByKey.create()) + .apply( + ParDo.named(name + ".CheckForLateEvents") + .of(new DoFnWithContext>, + KV>>() { + final Aggregator earlyCounter = + createAggregator("earlyShard", new SumLongFn()); + final Aggregator onTimeCounter = + createAggregator("onTimeShard", new SumLongFn()); + final Aggregator lateCounter = + createAggregator("lateShard", new SumLongFn()); + final Aggregator unexpectedLatePaneCounter = + createAggregator("ERROR_unexpectedLatePane", new SumLongFn()); + final Aggregator unexpectedOnTimeElementCounter = + createAggregator("ERROR_unexpectedOnTimeElement", new SumLongFn()); + + @ProcessElement + public void processElement(ProcessContext c, BoundedWindow window) { + int numLate = 0; + int numOnTime = 0; + for (Event event : c.element().getValue()) { + if (event.hasAnnotation("LATE")) { + numLate++; + } else { + numOnTime++; + } + } + String shard = c.element().getKey(); + NexmarkUtils.error( + "%s with timestamp %s has %d actually late and %d on-time " + + "elements in pane %s for window %s", + shard, c.timestamp(), numLate, numOnTime, c.pane(), + window.maxTimestamp()); + if (c.pane().getTiming() == PaneInfo.Timing.LATE) { + if (numLate == 0) { + NexmarkUtils.error( + "ERROR! No late events in late pane for %s", shard); + unexpectedLatePaneCounter.addValue(1L); + } + if (numOnTime > 0) { + NexmarkUtils.error( + "ERROR! Have %d on-time events in late pane for %s", + numOnTime, shard); + unexpectedOnTimeElementCounter.addValue(1L); + } + lateCounter.addValue(1L); + } else if (c.pane().getTiming() == PaneInfo.Timing.EARLY) { + if (numOnTime + numLate < configuration.maxLogEvents) { + NexmarkUtils.error( + "ERROR! Only have %d events in early pane for %s", + numOnTime + numLate, shard); + } + earlyCounter.addValue(1L); + } else { + onTimeCounter.addValue(1L); + } + c.output(c.element()); + } + })) + .apply( + ParDo.named(name + ".UploadEvents") + .of(new DoFnWithContext>, + KV>() { + final Aggregator savedFileCounter = + createAggregator("savedFile", new SumLongFn()); + final Aggregator writtenRecordsCounter = + createAggregator("writtenRecords", new SumLongFn()); + + @ProcessElement + public void process(ProcessContext c, BoundedWindow window) throws IOException { + String shard = c.element().getKey(); + GcsOptions options = c.getPipelineOptions().as(GcsOptions.class); + OutputFile outputFile = outputFileFor(window, shard, c.pane()); + NexmarkUtils.error( + "Writing %s with record timestamp %s, window timestamp %s, pane %s", + shard, c.timestamp(), window.maxTimestamp(), c.pane()); + if (outputFile.filename != null) { + NexmarkUtils.error("Beginning write to '%s'", outputFile.filename); + int n = 0; + try (OutputStream output = + Channels.newOutputStream(openWritableGcsFile(options, outputFile + .filename))) { + for (Event event : c.element().getValue()) { + Event.CODER.encode(event, output, Coder.Context.OUTER); + writtenRecordsCounter.addValue(1L); + if (++n % 10000 == 0) { + NexmarkUtils.error("So far written %d records to '%s'", n, + outputFile.filename); + } + } + } + NexmarkUtils.error("Written all %d records to '%s'", n, outputFile.filename); + } + savedFileCounter.addValue(1L); + c.output(KV.of(null, outputFile)); + } + })) + // Clear fancy triggering from above. + .apply(Window.>into( + FixedWindows.of(Duration.standardSeconds(configuration.windowSizeSec))) + .named(name + ".WindowLogFiles") + .triggering(AfterWatermark.pastEndOfWindow()) + // We expect no late data here, but we'll assume the worst so we can detect any. + .withAllowedLateness(Duration.standardDays(1)) + .discardingFiredPanes()) + .apply(GroupByKey.create()) + .apply( + ParDo.named(name + ".Index") + .of(new DoFnWithContext>, Done>() { + final Aggregator unexpectedLateCounter = + createAggregator("ERROR_unexpectedLate", new SumLongFn()); + final Aggregator unexpectedEarlyCounter = + createAggregator("ERROR_unexpectedEarly", new SumLongFn()); + final Aggregator unexpectedIndexCounter = + createAggregator("ERROR_unexpectedIndex", new SumLongFn()); + final Aggregator finalizedCounter = + createAggregator("indexed", new SumLongFn()); + + @ProcessElement + public void process(ProcessContext c, BoundedWindow window) throws IOException { + if (c.pane().getTiming() == Timing.LATE) { + unexpectedLateCounter.addValue(1L); + NexmarkUtils.error("ERROR! Unexpected LATE pane: %s", c.pane()); + } else if (c.pane().getTiming() == Timing.EARLY) { + unexpectedEarlyCounter.addValue(1L); + NexmarkUtils.error("ERROR! Unexpected EARLY pane: %s", c.pane()); + } else if (c.pane().getTiming() == Timing.ON_TIME + && c.pane().getIndex() != 0) { + unexpectedIndexCounter.addValue(1L); + NexmarkUtils.error("ERROR! Unexpected ON_TIME pane index: %s", c.pane()); + } else { + GcsOptions options = c.getPipelineOptions().as(GcsOptions.class); + NexmarkUtils.error( + "Index with record timestamp %s, window timestamp %s, pane %s", + c.timestamp(), window.maxTimestamp(), c.pane()); + + @Nullable String filename = indexPathFor(window); + if (filename != null) { + NexmarkUtils.error("Beginning write to '%s'", filename); + int n = 0; + try (OutputStream output = + Channels.newOutputStream( + openWritableGcsFile(options, filename))) { + for (OutputFile outputFile : c.element().getValue()) { + output.write(outputFile.toString().getBytes()); + n++; + } + } + NexmarkUtils.error("Written all %d lines to '%s'", n, filename); + } + c.output( + new Done("written for timestamp " + window.maxTimestamp())); + finalizedCounter.addValue(1L); + } + } + })); + } + + @Override + protected PCollection applyPrim(PCollection events) { + return NexmarkUtils.castToKnownSize(name, applyTyped(events)); + } +} diff --git a/integration/java/src/main/java/org/apache/beam/integration/nexmark/Query11.java b/integration/java/src/main/java/org/apache/beam/integration/nexmark/Query11.java new file mode 100644 index 0000000000000..9841421ee15e7 --- /dev/null +++ b/integration/java/src/main/java/org/apache/beam/integration/nexmark/Query11.java @@ -0,0 +1,76 @@ +/* + * 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.integration.nexmark; + +import org.apache.beam.sdk.transforms.Count; +import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.transforms.ParDo; +import org.apache.beam.sdk.transforms.windowing.AfterPane; +import org.apache.beam.sdk.transforms.windowing.Repeatedly; +import org.apache.beam.sdk.transforms.windowing.Sessions; +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; + +/** + * Query "11", 'User sessions' (Not in original suite.) + * + *

              Group bids by the same user into sessions with {@code windowSizeSec} max gap. + * However limit the session to at most {@code maxLogEvents}. Emit the number of + * bids per session. + */ +class Query11 extends NexmarkQuery { + public Query11(NexmarkConfiguration configuration) { + super(configuration, "Query11"); + } + + private PCollection applyTyped(PCollection events) { + return events.apply(JUST_BIDS) + .apply( + ParDo.named(name + ".Rekey") + .of(new DoFn>() { + @Override + public void processElement(ProcessContext c) { + Bid bid = c.element(); + c.output(KV.of(bid.bidder, (Void) null)); + } + })) + .apply(Window.>into( + Sessions.withGapDuration(Duration.standardSeconds(configuration.windowSizeSec))) + .triggering(Repeatedly.forever(AfterPane.elementCountAtLeast(configuration.maxLogEvents))) + .discardingFiredPanes() + .withAllowedLateness(Duration.standardSeconds(configuration.occasionalDelaySec / 2))) + .apply(Count.perKey()) + .apply( + ParDo.named(name + ".ToResult") + .of(new DoFn, BidsPerSession>() { + @Override + public void processElement(ProcessContext c) { + c.output(new BidsPerSession(c.element().getKey(), c.element().getValue())); + } + })); + } + + @Override + protected PCollection applyPrim(PCollection events) { + return NexmarkUtils.castToKnownSize(name, applyTyped(events)); + } +} diff --git a/integration/java/src/main/java/org/apache/beam/integration/nexmark/Query12.java b/integration/java/src/main/java/org/apache/beam/integration/nexmark/Query12.java new file mode 100644 index 0000000000000..dd39971654fb9 --- /dev/null +++ b/integration/java/src/main/java/org/apache/beam/integration/nexmark/Query12.java @@ -0,0 +1,79 @@ +/* + * 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.integration.nexmark; + +import org.apache.beam.sdk.transforms.Count; +import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.transforms.ParDo; +import org.apache.beam.sdk.transforms.windowing.AfterProcessingTime; +import org.apache.beam.sdk.transforms.windowing.GlobalWindows; +import org.apache.beam.sdk.transforms.windowing.Repeatedly; +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; + +/** + * Query "12", 'Processing time windows' (Not in original suite.) + *

              + *

              Group bids by the same user into processing time windows of windowSize. Emit the count + * of bids per window. + */ +class Query12 extends NexmarkQuery { + public Query12(NexmarkConfiguration configuration) { + super(configuration, "Query12"); + } + + private PCollection applyTyped(PCollection events) { + return events + .apply(JUST_BIDS) + .apply( + ParDo.named(name + ".Rekey") + .of(new DoFn>() { + @Override + public void processElement(ProcessContext c) { + Bid bid = c.element(); + c.output(KV.of(bid.bidder, (Void) null)); + } + })) + .apply(Window.>into(new GlobalWindows()) + .triggering( + Repeatedly.forever( + AfterProcessingTime.pastFirstElementInPane() + .plusDelayOf( + Duration.standardSeconds(configuration.windowSizeSec)))) + .discardingFiredPanes() + .withAllowedLateness(Duration.ZERO)) + .apply(Count.perKey()) + .apply( + ParDo.named(name + ".ToResult") + .of(new DoFn, BidsPerSession>() { + @Override + public void processElement(ProcessContext c) { + c.output( + new BidsPerSession(c.element().getKey(), c.element().getValue())); + } + })); + } + + @Override + protected PCollection applyPrim(PCollection events) { + return NexmarkUtils.castToKnownSize(name, applyTyped(events)); + } +} diff --git a/integration/java/src/main/java/org/apache/beam/integration/nexmark/Query1Model.java b/integration/java/src/main/java/org/apache/beam/integration/nexmark/Query1Model.java new file mode 100644 index 0000000000000..462d4262237a4 --- /dev/null +++ b/integration/java/src/main/java/org/apache/beam/integration/nexmark/Query1Model.java @@ -0,0 +1,73 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.beam.integration.nexmark; + +import org.apache.beam.sdk.values.TimestampedValue; + +import java.io.Serializable; +import java.util.Collection; +import java.util.Iterator; + +/** + * A direct implementation of {@link Query1}. + */ +public class Query1Model extends NexmarkQueryModel implements Serializable { + /** + * Simulator for query 1. + */ + private class Simulator extends AbstractSimulator { + public Simulator(NexmarkConfiguration configuration) { + super(NexmarkUtils.standardEventIterator(configuration)); + } + + @Override + protected void run() { + TimestampedValue timestampedEvent = nextInput(); + if (timestampedEvent == null) { + allDone(); + return; + } + Event event = timestampedEvent.getValue(); + if (event.bid == null) { + // Ignore non-bid events. + return; + } + Bid bid = event.bid; + Bid resultBid = + new Bid(bid.auction, bid.bidder, bid.price * 89 / 100, bid.dateTime, bid.extra); + TimestampedValue result = + TimestampedValue.of(resultBid, timestampedEvent.getTimestamp()); + addResult(result); + } + } + + public Query1Model(NexmarkConfiguration configuration) { + super(configuration); + } + + @Override + public AbstractSimulator simulator() { + return new Simulator(configuration); + } + + @Override + protected Collection toCollection(Iterator> itr) { + return toValueTimestampOrder(itr); + } +} diff --git a/integration/java/src/main/java/org/apache/beam/integration/nexmark/Query2.java b/integration/java/src/main/java/org/apache/beam/integration/nexmark/Query2.java new file mode 100644 index 0000000000000..cede2f36eb7c0 --- /dev/null +++ b/integration/java/src/main/java/org/apache/beam/integration/nexmark/Query2.java @@ -0,0 +1,75 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.beam.integration.nexmark; + +import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.transforms.Filter; +import org.apache.beam.sdk.transforms.ParDo; +import org.apache.beam.sdk.transforms.SerializableFunction; +import org.apache.beam.sdk.values.PCollection; + +/** + * Query 2, 'Filtering. Find bids with specific auction ids and show their bid price. + * In CQL syntax: + * + *

              + * SELECT Rstream(auction, price)
              + * FROM Bid [NOW]
              + * WHERE auction = 1007 OR auction = 1020 OR auction = 2001 OR auction = 2019 OR auction = 2087;
              + * 
              + * + *

              As written that query will only yield a few hundred results over event streams of + * arbitrary size. To make it more interesting we instead choose bids for every + * {@code auctionSkip}'th auction. + */ +class Query2 extends NexmarkQuery { + public Query2(NexmarkConfiguration configuration) { + super(configuration, "Query2"); + } + + private PCollection applyTyped(PCollection events) { + return events + // Only want the bid events. + .apply(JUST_BIDS) + + // Select just the bids for the auctions we care about. + .apply(Filter.byPredicate(new SerializableFunction() { + @Override + public Boolean apply(Bid bid) { + return bid.auction % configuration.auctionSkip == 0; + } + })) + + // Project just auction id and price. + .apply( + ParDo.named(name + ".Project") + .of(new DoFn() { + @Override + public void processElement(ProcessContext c) { + Bid bid = c.element(); + c.output(new AuctionPrice(bid.auction, bid.price)); + } + })); + } + + @Override + protected PCollection applyPrim(PCollection events) { + return NexmarkUtils.castToKnownSize(name, applyTyped(events)); + } +} diff --git a/integration/java/src/main/java/org/apache/beam/integration/nexmark/Query2Model.java b/integration/java/src/main/java/org/apache/beam/integration/nexmark/Query2Model.java new file mode 100644 index 0000000000000..6ccfeeb9d7ba3 --- /dev/null +++ b/integration/java/src/main/java/org/apache/beam/integration/nexmark/Query2Model.java @@ -0,0 +1,76 @@ +/* + * 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.integration.nexmark; + +import org.apache.beam.sdk.values.TimestampedValue; + +import java.io.Serializable; +import java.util.Collection; +import java.util.Iterator; + +/** + * A direct implementation of {@link Query2}. + */ +public class Query2Model extends NexmarkQueryModel implements Serializable { + /** + * Simulator for query 2. + */ + private class Simulator extends AbstractSimulator { + public Simulator(NexmarkConfiguration configuration) { + super(NexmarkUtils.standardEventIterator(configuration)); + } + + @Override + protected void run() { + TimestampedValue timestampedEvent = nextInput(); + if (timestampedEvent == null) { + allDone(); + return; + } + Event event = timestampedEvent.getValue(); + if (event.bid == null) { + // Ignore non bid events. + return; + } + Bid bid = event.bid; + if (bid.auction % configuration.auctionSkip != 0) { + // Ignore bids for auctions we don't care about. + return; + } + AuctionPrice auctionPrice = new AuctionPrice(bid.auction, bid.price); + TimestampedValue result = + TimestampedValue.of(auctionPrice, timestampedEvent.getTimestamp()); + addResult(result); + } + } + + public Query2Model(NexmarkConfiguration configuration) { + super(configuration); + } + + @Override + public AbstractSimulator simulator() { + return new Simulator(configuration); + } + + @Override + protected Collection toCollection(Iterator> itr) { + return toValueTimestampOrder(itr); + } +} diff --git a/integration/java/src/main/java/org/apache/beam/integration/nexmark/Query3.java b/integration/java/src/main/java/org/apache/beam/integration/nexmark/Query3.java new file mode 100644 index 0000000000000..5b9b17bc6a794 --- /dev/null +++ b/integration/java/src/main/java/org/apache/beam/integration/nexmark/Query3.java @@ -0,0 +1,248 @@ +/* + * 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.integration.nexmark; + +import org.apache.beam.sdk.coders.ListCoder; +import org.apache.beam.sdk.transforms.Aggregator; +import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.transforms.Filter; +import org.apache.beam.sdk.transforms.ParDo; +import org.apache.beam.sdk.transforms.SerializableFunction; +import org.apache.beam.sdk.transforms.Sum.SumLongFn; +import org.apache.beam.sdk.transforms.join.CoGbkResult; +import org.apache.beam.sdk.transforms.join.CoGroupByKey; +import org.apache.beam.sdk.transforms.join.KeyedPCollectionTuple; +import org.apache.beam.sdk.transforms.windowing.FixedWindows; +import org.apache.beam.sdk.transforms.windowing.Window; +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.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.joda.time.Duration; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; + +import javax.annotation.Nullable; + +/** + * Query 3, 'Local Item Suggestion'. Who is selling in OR, ID or CA in category 10, and for what + * auction ids? In CQL syntax: + * + *

              + * SELECT Istream(P.name, P.city, P.state, A.id)
              + * FROM Auction A [ROWS UNBOUNDED], Person P [ROWS UNBOUNDED]
              + * WHERE A.seller = P.id AND (P.state = `OR' OR P.state = `ID' OR P.state = `CA') AND A.category
              + * = 10;
              + * 
              + * + *

              We'll implement this query to allow 'new auction' events to come before the 'new person' + * events for the auction seller. Those auctions will be stored until the matching person is + * seen. Then all subsequent auctions for a person will use the stored person record. + * + *

              A real system would use an external system to maintain the id-to-person association. + */ +class Query3 extends NexmarkQuery { + private static final StateNamespace GLOBAL_NAMESPACE = StateNamespaces.global(); + private static final StateTag>> AUCTION_LIST_CODED_TAG = + StateTags.value("left", ListCoder.of(Auction.CODER)); + private static final StateTag> PERSON_CODED_TAG = + StateTags.value("right", Person.CODER); + + /** + * Join {@code auctions} and {@code people} by person id and emit their cross-product one pair + * at a time. + * + *

              We know a person may submit any number of auctions. Thus new person event must have the + * person record stored in persistent state in order to match future auctions by that person. + * + *

              However we know that each auction is associated with at most one person, so only need + * to store auction records in persistent state until we have seen the corresponding person + * record. And of course may have already seen that record. + */ + private static class JoinDoFn extends DoFn, KV> { + private final Aggregator newAuctionCounter = + createAggregator("newAuction", new SumLongFn()); + private final Aggregator newPersonCounter = + createAggregator("newPerson", new SumLongFn()); + private final Aggregator newNewOutputCounter = + createAggregator("newNewOutput", new SumLongFn()); + private final Aggregator newOldOutputCounter = + createAggregator("newOldOutput", new SumLongFn()); + private final Aggregator oldNewOutputCounter = + createAggregator("oldNewOutput", new SumLongFn()); + public final Aggregator fatalCounter = createAggregator("fatal", new SumLongFn()); + + @Override + public void processElement(ProcessContext c) throws IOException { + // TODO: This is using the internal state API. Rework to use the + // We would *almost* implement this by rewindowing into the global window and + // running a combiner over the result. The combiner's accumulator would be the + // state we use below. However, combiners cannot emit intermediate results, thus + // we need to wait for the pending ReduceFn API. + StateInternals stateInternals = c.windowingInternals().stateInternals(); + ValueState personState = stateInternals.state(GLOBAL_NAMESPACE, PERSON_CODED_TAG); + Person existingPerson = personState.read(); + if (existingPerson != null) { + // We've already seen the new person event for this person id. + // We can join with any new auctions on-the-fly without needing any + // additional persistent state. + for (Auction newAuction : c.element().getValue().getAll(AUCTION_TAG)) { + newAuctionCounter.addValue(1L); + newOldOutputCounter.addValue(1L); + c.output(KV.of(newAuction, existingPerson)); + } + return; + } + + ValueState> auctionsState = + stateInternals.state(GLOBAL_NAMESPACE, AUCTION_LIST_CODED_TAG); + Person theNewPerson = null; + for (Person newPerson : c.element().getValue().getAll(PERSON_TAG)) { + if (theNewPerson == null) { + theNewPerson = newPerson; + } else { + if (theNewPerson.equals(newPerson)) { + NexmarkUtils.error("**** duplicate person %s ****", theNewPerson); + } else { + NexmarkUtils.error("**** conflicting persons %s and %s ****", theNewPerson, newPerson); + } + fatalCounter.addValue(1L); + continue; + } + newPersonCounter.addValue(1L); + // We've now seen the person for this person id so can flush any + // pending auctions for the same seller id. + List pendingAuctions = auctionsState.read(); + if (pendingAuctions != null) { + for (Auction pendingAuction : pendingAuctions) { + oldNewOutputCounter.addValue(1L); + c.output(KV.of(pendingAuction, newPerson)); + } + auctionsState.clear(); + } + // Also deal with any new auctions. + for (Auction newAuction : c.element().getValue().getAll(AUCTION_TAG)) { + newAuctionCounter.addValue(1L); + newNewOutputCounter.addValue(1L); + c.output(KV.of(newAuction, newPerson)); + } + // Remember this person for any future auctions. + personState.write(newPerson); + } + if (theNewPerson != null) { + return; + } + + // We'll need to remember the auctions until we see the corresponding + // new person event. + List pendingAuctions = auctionsState.read(); + if (pendingAuctions == null) { + pendingAuctions = new ArrayList<>(); + } + for (Auction newAuction : c.element().getValue().getAll(AUCTION_TAG)) { + newAuctionCounter.addValue(1L); + pendingAuctions.add(newAuction); + } + auctionsState.write(pendingAuctions); + } + } + + private final JoinDoFn joinDoFn = new JoinDoFn(); + + public Query3(NexmarkConfiguration configuration) { + super(configuration, "Query3"); + } + + @Override + @Nullable + public Aggregator getFatalCount() { + return joinDoFn.fatalCounter; + } + + private PCollection applyTyped(PCollection events) { + // Batch into incremental results windows. + events = events.apply( + Window.into(FixedWindows.of(Duration.standardSeconds(configuration.windowSizeSec)))); + + PCollection> auctionsBySellerId = + events + // Only want the new auction events. + .apply(JUST_NEW_AUCTIONS) + + // We only want auctions in category 10. + .apply(Filter.byPredicate(new SerializableFunction() { + @Override + public Boolean apply(Auction auction) { + return auction.category == 10; + } + }).named(name + ".InCategory")) + + // Key auctions by their seller id. + .apply(AUCTION_BY_SELLER); + + PCollection> personsById = + events + // Only want the new people events. + .apply(JUST_NEW_PERSONS) + + // We only want people in OR, ID, CA. + .apply(Filter.byPredicate(new SerializableFunction() { + @Override + public Boolean apply(Person person) { + return person.state.equals("OR") || person.state.equals("ID") + || person.state.equals("CA"); + } + }).named(name + ".InState")) + + // Key people by their id. + .apply(PERSON_BY_ID); + + return + // Join auctions and people. + KeyedPCollectionTuple.of(AUCTION_TAG, auctionsBySellerId) + .and(PERSON_TAG, personsById) + .apply(CoGroupByKey.create()) + .apply(ParDo.named(name + ".Join").of(joinDoFn)) + + // Project what we want. + .apply( + ParDo.named(name + ".Project") + .of(new DoFn, NameCityStateId>() { + @Override + public void processElement(ProcessContext c) { + Auction auction = c.element().getKey(); + Person person = c.element().getValue(); + c.output(new NameCityStateId( + person.name, person.city, person.state, auction.id)); + } + })); + } + + @Override + protected PCollection applyPrim(PCollection events) { + return NexmarkUtils.castToKnownSize(name, applyTyped(events)); + } +} diff --git a/integration/java/src/main/java/org/apache/beam/integration/nexmark/Query3Model.java b/integration/java/src/main/java/org/apache/beam/integration/nexmark/Query3Model.java new file mode 100644 index 0000000000000..b865eda613374 --- /dev/null +++ b/integration/java/src/main/java/org/apache/beam/integration/nexmark/Query3Model.java @@ -0,0 +1,119 @@ +/* + * 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.integration.nexmark; + +import org.apache.beam.sdk.values.TimestampedValue; +import com.google.common.collect.ArrayListMultimap; +import com.google.common.collect.Multimap; + +import org.joda.time.Instant; + +import java.io.Serializable; +import java.util.Collection; +import java.util.HashMap; +import java.util.Iterator; +import java.util.Map; + +/** + * A direct implementation of {@link Query3}. + */ +public class Query3Model extends NexmarkQueryModel implements Serializable { + /** + * Simulator for query 3. + */ + private class Simulator extends AbstractSimulator { + /** Auctions, indexed by seller id. */ + private final Multimap newAuctions; + + /** Persons, indexed by id. */ + private final Map newPersons; + + public Simulator(NexmarkConfiguration configuration) { + super(NexmarkUtils.standardEventIterator(configuration)); + newPersons = new HashMap<>(); + newAuctions = ArrayListMultimap.create(); + } + + /** + * Capture new result. + */ + private void addResult(Auction auction, Person person, Instant timestamp) { + TimestampedValue result = TimestampedValue.of( + new NameCityStateId(person.name, person.city, person.state, auction.id), timestamp); + addResult(result); + } + + @Override + protected void run() { + TimestampedValue timestampedEvent = nextInput(); + if (timestampedEvent == null) { + allDone(); + return; + } + Event event = timestampedEvent.getValue(); + if (event.bid != null) { + // Ignore bid events. + return; + } + + Instant timestamp = timestampedEvent.getTimestamp(); + + if (event.newAuction != null) { + // Only want auctions in category 10. + if (event.newAuction.category == 10) { + // Join new auction with existing person, if any. + Person person = newPersons.get(event.newAuction.seller); + if (person != null) { + addResult(event.newAuction, person, timestamp); + } else { + // Remember auction for future new person event. + newAuctions.put(event.newAuction.seller, event.newAuction); + } + } + } else { + // Only want people in OR, ID or CA. + if (event.newPerson.state.equals("OR") || event.newPerson.state.equals("ID") + || event.newPerson.state.equals("CA")) { + // Join new person with existing auctions. + for (Auction auction : newAuctions.get(event.newPerson.id)) { + addResult(auction, event.newPerson, timestamp); + } + // We'll never need these auctions again. + newAuctions.removeAll(event.newPerson.id); + // Remember person for future auctions. + newPersons.put(event.newPerson.id, event.newPerson); + } + } + } + } + + public Query3Model(NexmarkConfiguration configuration) { + super(configuration); + } + + @Override + public AbstractSimulator simulator() { + return new Simulator(configuration); + } + + @Override + protected Collection toCollection(Iterator> itr) { + return toValue(itr); + } +} diff --git a/integration/java/src/main/java/org/apache/beam/integration/nexmark/Query4.java b/integration/java/src/main/java/org/apache/beam/integration/nexmark/Query4.java new file mode 100644 index 0000000000000..bc695b7fafb3e --- /dev/null +++ b/integration/java/src/main/java/org/apache/beam/integration/nexmark/Query4.java @@ -0,0 +1,110 @@ +/* + * 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.integration.nexmark; + +import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.transforms.Mean; +import org.apache.beam.sdk.transforms.ParDo; +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.joda.time.Duration; + +/** + * Query 4, 'Average Price for a Category'. Select the average of the wining bid prices for all + * closed auctions in each category. In CQL syntax: + * + *

              + * SELECT Istream(AVG(Q.final))
              + * FROM Category C, (SELECT Rstream(MAX(B.price) AS final, A.category)
              + *                   FROM Auction A [ROWS UNBOUNDED], Bid B [ROWS UNBOUNDED]
              + *                   WHERE A.id=B.auction AND B.datetime < A.expires AND A.expires < CURRENT_TIME
              + *                   GROUP BY A.id, A.category) Q
              + * WHERE Q.category = C.id
              + * GROUP BY C.id;
              + * 
              + * + *

              For extra spiciness our implementation differs slightly from the above: + *

                + *
              • We select both the average winning price and the category. + *
              • We don't bother joining with a static category table, since it's contents are never used. + *
              • We only consider bids which are above the auction's reserve price. + *
              • We accept the highest-price, earliest valid bid as the winner. + *
              • We calculate the averages oven a sliding window of size {@code windowSizeSec} and + * period {@code windowPeriodSec}. + *
              + */ +class Query4 extends NexmarkQuery { + private final Monitor winningBidsMonitor; + + public Query4(NexmarkConfiguration configuration) { + super(configuration, "Query4"); + winningBidsMonitor = new Monitor<>(name + ".WinningBids", "winning"); + } + + private PCollection applyTyped(PCollection events) { + PCollection winningBids = + events + // Find the winning bid for each closed auction. + .apply(new WinningBids(name + ".WinningBids", configuration)); + + // Monitor winning bids + winningBids = winningBids.apply(winningBidsMonitor.getTransform()); + + return winningBids + // Key the winning bid price by the auction category. + .apply( + ParDo.named(name + ".Rekey") + .of(new DoFn>() { + @Override + public void processElement(ProcessContext c) { + Auction auction = c.element().auction; + Bid bid = c.element().bid; + c.output(KV.of(auction.category, bid.price)); + } + })) + + // Re-window so we can calculate a sliding average + .apply(Window.>into( + SlidingWindows.of(Duration.standardSeconds(configuration.windowSizeSec)) + .every(Duration.standardSeconds(configuration.windowPeriodSec)))) + + // Find the average of the winning bids for each category. + // Make sure we share the work for each category between workers. + .apply(Mean.perKey().withHotKeyFanout(configuration.fanout)) + + // For testing against Query4Model, capture which results are 'final'. + .apply( + ParDo.named(name + ".Project") + .of(new DoFn, CategoryPrice>() { + @Override + public void processElement(ProcessContext c) { + c.output(new CategoryPrice(c.element().getKey(), + Math.round(c.element().getValue()), c.pane().isLast())); + } + })); + } + + @Override + protected PCollection applyPrim(PCollection events) { + return NexmarkUtils.castToKnownSize(name, applyTyped(events)); + } +} diff --git a/integration/java/src/main/java/org/apache/beam/integration/nexmark/Query4Model.java b/integration/java/src/main/java/org/apache/beam/integration/nexmark/Query4Model.java new file mode 100644 index 0000000000000..24103067020de --- /dev/null +++ b/integration/java/src/main/java/org/apache/beam/integration/nexmark/Query4Model.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.integration.nexmark; + +import org.apache.beam.sdk.transforms.windowing.BoundedWindow; +import org.apache.beam.sdk.values.TimestampedValue; + +import org.joda.time.Duration; +import org.joda.time.Instant; +import org.junit.Assert; + +import java.io.Serializable; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.TreeMap; + +/** + * A direct implementation of {@link Query4}. + */ +public class Query4Model extends NexmarkQueryModel implements Serializable { + /** + * Simulator for query 4. + */ + private class Simulator extends AbstractSimulator { + /** The prices and categories for all winning bids in the last window size. */ + private final List> winningPricesByCategory; + + /** Timestamp of last result (ms since epoch). */ + private Instant lastTimestamp; + + /** When oldest active window starts. */ + private Instant windowStart; + + /** The last seen result for each category. */ + private final Map> lastSeenResults; + + public Simulator(NexmarkConfiguration configuration) { + super(new WinningBidsSimulator(configuration).results()); + winningPricesByCategory = new ArrayList<>(); + lastTimestamp = BoundedWindow.TIMESTAMP_MIN_VALUE; + windowStart = NexmarkUtils.BEGINNING_OF_TIME; + lastSeenResults = new TreeMap<>(); + } + + /** + * Calculate the average bid price for each category for all winning bids + * which are strictly before {@code end}. + */ + private void averages(Instant end) { + Map counts = new TreeMap<>(); + Map totals = new TreeMap<>(); + for (TimestampedValue value : winningPricesByCategory) { + if (!value.getTimestamp().isBefore(end)) { + continue; + } + long category = value.getValue().category; + long price = value.getValue().price; + Long count = counts.get(category); + if (count == null) { + count = 1L; + } else { + count += 1; + } + counts.put(category, count); + Long total = totals.get(category); + if (total == null) { + total = price; + } else { + total += price; + } + totals.put(category, total); + } + for (long category : counts.keySet()) { + long count = counts.get(category); + long total = totals.get(category); + TimestampedValue result = TimestampedValue.of( + new CategoryPrice(category, Math.round((double) total / count), true), lastTimestamp); + addIntermediateResult(result); + lastSeenResults.put(category, result); + } + } + + /** + * Calculate averages for any windows which can now be retired. Also prune entries + * which can no longer contribute to any future window. + */ + private void prune(Instant newWindowStart) { + while (!newWindowStart.equals(windowStart)) { + averages(windowStart.plus(Duration.standardSeconds(configuration.windowSizeSec))); + windowStart = windowStart.plus(Duration.standardSeconds(configuration.windowPeriodSec)); + Iterator> itr = winningPricesByCategory.iterator(); + while (itr.hasNext()) { + if (itr.next().getTimestamp().isBefore(windowStart)) { + itr.remove(); + } + } + if (winningPricesByCategory.isEmpty()) { + windowStart = newWindowStart; + } + } + } + + /** + * Capture the winning bid. + */ + private void captureWinningBid(Auction auction, Bid bid, Instant timestamp) { + winningPricesByCategory.add( + TimestampedValue.of(new CategoryPrice(auction.category, bid.price, false), timestamp)); + } + + @Override + protected void run() { + TimestampedValue timestampedWinningBid = nextInput(); + if (timestampedWinningBid == null) { + prune(NexmarkUtils.END_OF_TIME); + for (TimestampedValue result : lastSeenResults.values()) { + addResult(result); + } + allDone(); + return; + } + lastTimestamp = timestampedWinningBid.getTimestamp(); + Instant newWindowStart = windowStart(Duration.standardSeconds(configuration.windowSizeSec), + Duration.standardSeconds(configuration.windowPeriodSec), lastTimestamp); + prune(newWindowStart); + captureWinningBid(timestampedWinningBid.getValue().auction, + timestampedWinningBid.getValue().bid, lastTimestamp); + } + } + + public Query4Model(NexmarkConfiguration configuration) { + super(configuration); + } + + @Override + public AbstractSimulator simulator() { + return new Simulator(configuration); + } + + @Override + protected Iterable> relevantResults( + Iterable> results) { + // Find the last (in processing time) reported average price for each category. + Map> finalAverages = new TreeMap<>(); + for (TimestampedValue obj : results) { + Assert.assertTrue("have CategoryPrice", obj.getValue() instanceof CategoryPrice); + CategoryPrice categoryPrice = (CategoryPrice) obj.getValue(); + if (categoryPrice.isLast) { + finalAverages.put( + categoryPrice.category, + TimestampedValue.of((KnownSize) categoryPrice, obj.getTimestamp())); + } + } + + return finalAverages.values(); + } + + @Override + protected Collection toCollection(Iterator> itr) { + return toValue(itr); + } +} diff --git a/integration/java/src/main/java/org/apache/beam/integration/nexmark/Query5.java b/integration/java/src/main/java/org/apache/beam/integration/nexmark/Query5.java new file mode 100644 index 0000000000000..91a4a28c67c42 --- /dev/null +++ b/integration/java/src/main/java/org/apache/beam/integration/nexmark/Query5.java @@ -0,0 +1,127 @@ +/* + * 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.integration.nexmark; + +import org.apache.beam.sdk.transforms.Combine; +import org.apache.beam.sdk.transforms.Count; +import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.transforms.ParDo; +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.joda.time.Duration; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; + +/** + * Query 5, 'Hot Items'. Which auctions have seen the most bids in the last hour (updated every + * minute). In CQL syntax: + * + *
              + * SELECT Rstream(auction)
              + * FROM (SELECT B1.auction, count(*) AS num
              + *       FROM Bid [RANGE 60 MINUTE SLIDE 1 MINUTE] B1
              + *       GROUP BY B1.auction)
              + * WHERE num >= ALL (SELECT count(*)
              + *                   FROM Bid [RANGE 60 MINUTE SLIDE 1 MINUTE] B2
              + *                   GROUP BY B2.auction);
              + * 
              + * + *

              To make things a bit more dynamic and easier to test we use much shorter windows, and + * we'll also preserve the bid counts. + */ +class Query5 extends NexmarkQuery { + public Query5(NexmarkConfiguration configuration) { + super(configuration, "Query5"); + } + + private PCollection applyTyped(PCollection events) { + return events + // Only want the bid events. + .apply(JUST_BIDS) + // Window the bids into sliding windows. + .apply(Window.into( + SlidingWindows.of(Duration.standardSeconds(configuration.windowSizeSec)) + .every(Duration.standardSeconds(configuration.windowPeriodSec)))) + // Project just the auction id. + .apply(BID_TO_AUCTION) + + // Count the number of bids per auction id. + .apply(Count.perElement()) + + // We'll want to keep all auctions with the maximal number of bids. + // Start by lifting each into a singleton list. + .apply( + ParDo.named(name + ".ToSingletons") + .of(new DoFn, KV, Long>>() { + @Override + public void processElement(ProcessContext c) { + c.output(KV.of(Arrays.asList(c.element().getKey()), c.element().getValue())); + } + })) + + // Keep only the auction ids with the most bids. + .apply( + Combine + .globally(new Combine.BinaryCombineFn, Long>>() { + @Override + public KV, Long> apply( + KV, Long> left, KV, Long> right) { + List leftBestAuctions = left.getKey(); + long leftCount = left.getValue(); + List rightBestAuctions = right.getKey(); + long rightCount = right.getValue(); + if (leftCount > rightCount) { + return left; + } else if (leftCount < rightCount) { + return right; + } else { + List newBestAuctions = new ArrayList<>(); + newBestAuctions.addAll(leftBestAuctions); + newBestAuctions.addAll(rightBestAuctions); + return KV.of(newBestAuctions, leftCount); + } + } + }) + .withoutDefaults() + .withFanout(configuration.fanout)) + + // Project into result. + .apply( + ParDo.named(name + ".Select") + .of(new DoFn, Long>, AuctionCount>() { + @Override + public void processElement(ProcessContext c) { + long count = c.element().getValue(); + for (long auction : c.element().getKey()) { + c.output(new AuctionCount(auction, count)); + } + } + })); + } + + @Override + protected PCollection applyPrim(PCollection events) { + return NexmarkUtils.castToKnownSize(name, applyTyped(events)); + } +} diff --git a/integration/java/src/main/java/org/apache/beam/integration/nexmark/Query5Model.java b/integration/java/src/main/java/org/apache/beam/integration/nexmark/Query5Model.java new file mode 100644 index 0000000000000..a7dd8f0bc8a72 --- /dev/null +++ b/integration/java/src/main/java/org/apache/beam/integration/nexmark/Query5Model.java @@ -0,0 +1,174 @@ +/* + * 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.integration.nexmark; + +import org.apache.beam.sdk.values.TimestampedValue; + +import org.joda.time.Duration; +import org.joda.time.Instant; + +import java.io.Serializable; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.TreeMap; + +/** + * A direct implementation of {@link Query5}. + */ +public class Query5Model extends NexmarkQueryModel implements Serializable { + /** + * Simulator for query 5. + */ + private class Simulator extends AbstractSimulator { + /** Time of bids still contributing to open windows, indexed by their auction id. */ + private final Map> bids; + + /** When oldest active window starts. */ + private Instant windowStart; + + public Simulator(NexmarkConfiguration configuration) { + super(NexmarkUtils.standardEventIterator(configuration)); + bids = new TreeMap<>(); + windowStart = NexmarkUtils.BEGINNING_OF_TIME; + } + + /** + * Count bids per auction id for bids strictly before {@code end}. Add the auction ids with + * the maximum number of bids to results. + */ + private void countBids(Instant end) { + Map counts = new TreeMap<>(); + long maxCount = 0L; + for (Map.Entry> entry : bids.entrySet()) { + long count = 0L; + long auction = entry.getKey(); + for (Instant bid : entry.getValue()) { + if (bid.isBefore(end)) { + count++; + } + } + if (count > 0) { + counts.put(auction, count); + maxCount = Math.max(maxCount, count); + } + } + for (Map.Entry entry : counts.entrySet()) { + long auction = entry.getKey(); + long count = entry.getValue(); + if (count == maxCount) { + AuctionCount result = new AuctionCount(auction, count); + addResult(TimestampedValue.of(result, end)); + } + } + } + + /** + * Retire bids which are strictly before {@code cutoff}. Return true if there are any bids + * remaining. + */ + private boolean retireBids(Instant cutoff) { + boolean anyRemain = false; + for (Map.Entry> entry : bids.entrySet()) { + long auction = entry.getKey(); + Iterator itr = entry.getValue().iterator(); + while (itr.hasNext()) { + Instant bid = itr.next(); + if (bid.isBefore(cutoff)) { + NexmarkUtils.info("retire: %s for %s", bid, auction); + itr.remove(); + } else { + anyRemain = true; + } + } + } + return anyRemain; + } + + /** + * Retire active windows until we've reached {@code newWindowStart}. + */ + private void retireWindows(Instant newWindowStart) { + while (!newWindowStart.equals(windowStart)) { + NexmarkUtils.info("retiring window %s, aiming for %s", windowStart, newWindowStart); + // Count bids in the window (windowStart, windowStart + size]. + countBids(windowStart.plus(Duration.standardSeconds(configuration.windowSizeSec))); + // Advance the window. + windowStart = windowStart.plus(Duration.standardSeconds(configuration.windowPeriodSec)); + // Retire bids which will never contribute to a future window. + if (!retireBids(windowStart)) { + // Can fast forward to latest window since no more outstanding bids. + windowStart = newWindowStart; + } + } + } + + /** + * Add bid to state. + */ + private void captureBid(Bid bid, Instant timestamp) { + List existing = bids.get(bid.auction); + if (existing == null) { + existing = new ArrayList<>(); + bids.put(bid.auction, existing); + } + existing.add(timestamp); + } + + @Override + public void run() { + TimestampedValue timestampedEvent = nextInput(); + if (timestampedEvent == null) { + // Drain the remaining windows. + retireWindows(NexmarkUtils.END_OF_TIME); + allDone(); + return; + } + + Event event = timestampedEvent.getValue(); + if (event.bid == null) { + // Ignore non-bid events. + return; + } + Instant timestamp = timestampedEvent.getTimestamp(); + Instant newWindowStart = windowStart(Duration.standardSeconds(configuration.windowSizeSec), + Duration.standardSeconds(configuration.windowPeriodSec), timestamp); + // Capture results from any windows we can now retire. + retireWindows(newWindowStart); + // Capture current bid. + captureBid(event.bid, timestamp); + } + } + + public Query5Model(NexmarkConfiguration configuration) { + super(configuration); + } + + @Override + public AbstractSimulator simulator() { + return new Simulator(configuration); + } + + @Override + protected Collection toCollection(Iterator> itr) { + return toValue(itr); + } +} diff --git a/integration/java/src/main/java/org/apache/beam/integration/nexmark/Query6.java b/integration/java/src/main/java/org/apache/beam/integration/nexmark/Query6.java new file mode 100644 index 0000000000000..49c0d68321c92 --- /dev/null +++ b/integration/java/src/main/java/org/apache/beam/integration/nexmark/Query6.java @@ -0,0 +1,154 @@ +/* + * 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.integration.nexmark; + +import org.apache.beam.sdk.transforms.Combine; +import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.transforms.ParDo; +import org.apache.beam.sdk.transforms.windowing.AfterPane; +import org.apache.beam.sdk.transforms.windowing.GlobalWindows; +import org.apache.beam.sdk.transforms.windowing.Repeatedly; +import org.apache.beam.sdk.transforms.windowing.Window; +import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.PCollection; +import com.google.common.collect.Lists; + +import org.joda.time.Duration; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; + +/** + * Query 6, 'Average Selling Price by Seller'. Select the average selling price over the + * last 10 closed auctions by the same seller. In CQL syntax: + * + *

              + * SELECT Istream(AVG(Q.final), Q.seller)
              + * FROM (SELECT Rstream(MAX(B.price) AS final, A.seller)
              + *       FROM Auction A [ROWS UNBOUNDED], Bid B [ROWS UNBOUNDED]
              + *       WHERE A.id=B.auction AND B.datetime < A.expires AND A.expires < CURRENT_TIME
              + *       GROUP BY A.id, A.seller) [PARTITION BY A.seller ROWS 10] Q
              + * GROUP BY Q.seller;
              + * 
              + * + *

              We are a little more exact with selecting winning bids: see {@link WinningBids}. + */ +class Query6 extends NexmarkQuery { + /** + * Combiner to keep track of up to {@code maxNumBids} of the most recent wining bids and calculate + * their average selling price. + */ + private static class MovingMeanSellingPrice extends Combine.CombineFn, Long> { + private final int maxNumBids; + + public MovingMeanSellingPrice(int maxNumBids) { + this.maxNumBids = maxNumBids; + } + + @Override + public List createAccumulator() { + return new ArrayList<>(); + } + + @Override + public List addInput(List accumulator, Bid input) { + accumulator.add(input); + Collections.sort(accumulator, Bid.ASCENDING_TIME_THEN_PRICE); + if (accumulator.size() > maxNumBids) { + accumulator.remove(0); + } + return accumulator; + } + + @Override + public List mergeAccumulators(Iterable> accumulators) { + List result = new ArrayList<>(); + for (List accumulator : accumulators) { + for (Bid bid : accumulator) { + result.add(bid); + } + } + Collections.sort(result, Bid.ASCENDING_TIME_THEN_PRICE); + if (result.size() > maxNumBids) { + result = Lists.newArrayList(result.listIterator(result.size() - maxNumBids)); + } + return result; + } + + @Override + public Long extractOutput(List accumulator) { + if (accumulator.isEmpty()) { + return 0L; + } + long sumOfPrice = 0; + for (Bid bid : accumulator) { + sumOfPrice += bid.price; + } + return Math.round((double) sumOfPrice / accumulator.size()); + } + } + + public Query6(NexmarkConfiguration configuration) { + super(configuration, "Query6"); + } + + private PCollection applyTyped(PCollection events) { + return events + // Find the winning bid for each closed auction. + .apply(new WinningBids(name + ".WinningBids", configuration)) + + // Key the winning bid by the seller id. + .apply( + ParDo.named(name + ".Rekey") + .of(new DoFn>() { + @Override + public void processElement(ProcessContext c) { + Auction auction = c.element().auction; + Bid bid = c.element().bid; + c.output(KV.of(auction.seller, bid)); + } + })) + + // Re-window to update on every wining bid. + .apply( + Window.>into(new GlobalWindows()) + .triggering(Repeatedly.forever(AfterPane.elementCountAtLeast(1))) + .accumulatingFiredPanes() + .withAllowedLateness(Duration.ZERO)) + + // Find the average of last 10 winning bids for each seller. + .apply(Combine.perKey(new MovingMeanSellingPrice(10))) + + // Project into our datatype. + .apply( + ParDo.named(name + ".Select") + .of(new DoFn, SellerPrice>() { + @Override + public void processElement(ProcessContext c) { + c.output(new SellerPrice(c.element().getKey(), c.element().getValue())); + } + })); + } + + @Override + protected PCollection applyPrim(PCollection events) { + return NexmarkUtils.castToKnownSize(name, applyTyped(events)); + } +} diff --git a/integration/java/src/main/java/org/apache/beam/integration/nexmark/Query6Model.java b/integration/java/src/main/java/org/apache/beam/integration/nexmark/Query6Model.java new file mode 100644 index 0000000000000..639ec9f76a1ad --- /dev/null +++ b/integration/java/src/main/java/org/apache/beam/integration/nexmark/Query6Model.java @@ -0,0 +1,128 @@ +/* + * 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.integration.nexmark; + +import org.apache.beam.sdk.transforms.windowing.BoundedWindow; +import org.apache.beam.sdk.values.TimestampedValue; + +import org.joda.time.Instant; +import org.junit.Assert; + +import java.io.Serializable; +import java.util.Collection; +import java.util.Iterator; +import java.util.Map; +import java.util.TreeMap; + +/** + * A direct implementation of {@link Query6}. + */ +public class Query6Model extends NexmarkQueryModel implements Serializable { + /** + * Simulator for query 6. + */ + private static class Simulator extends AbstractSimulator { + /** The cumulative count of winning bids, indexed by seller id. */ + private final Map numWinningBidsPerSeller; + + /** The cumulative total of winning bid prices, indexed by seller id. */ + private final Map totalWinningBidPricesPerSeller; + + private Instant lastTimestamp; + + public Simulator(NexmarkConfiguration configuration) { + super(new WinningBidsSimulator(configuration).results()); + numWinningBidsPerSeller = new TreeMap<>(); + totalWinningBidPricesPerSeller = new TreeMap<>(); + lastTimestamp = BoundedWindow.TIMESTAMP_MIN_VALUE; + } + + /** + * Update the per-seller running counts/sums. + */ + private void captureWinningBid(Auction auction, Bid bid, Instant timestamp) { + NexmarkUtils.info("winning auction, bid: %s, %s", auction, bid); + Long count = numWinningBidsPerSeller.get(auction.seller); + if (count == null) { + count = 1L; + } else { + count += 1; + } + numWinningBidsPerSeller.put(auction.seller, count); + Long total = totalWinningBidPricesPerSeller.get(auction.seller); + if (total == null) { + total = bid.price; + } else { + total += bid.price; + } + totalWinningBidPricesPerSeller.put(auction.seller, total); + TimestampedValue intermediateResult = TimestampedValue.of( + new SellerPrice(auction.seller, Math.round((double) total / count)), timestamp); + addIntermediateResult(intermediateResult); + } + + + @Override + protected void run() { + TimestampedValue timestampedWinningBid = nextInput(); + if (timestampedWinningBid == null) { + for (long seller : numWinningBidsPerSeller.keySet()) { + long count = numWinningBidsPerSeller.get(seller); + long total = totalWinningBidPricesPerSeller.get(seller); + addResult(TimestampedValue.of( + new SellerPrice(seller, Math.round((double) total / count)), lastTimestamp)); + } + allDone(); + return; + } + + lastTimestamp = timestampedWinningBid.getTimestamp(); + captureWinningBid(timestampedWinningBid.getValue().auction, + timestampedWinningBid.getValue().bid, lastTimestamp); + } + } + + public Query6Model(NexmarkConfiguration configuration) { + super(configuration); + } + + @Override + public AbstractSimulator simulator() { + return new Simulator(configuration); + } + + @Override + protected Iterable> relevantResults( + Iterable> results) { + // Find the last (in processing time) reported average price for each seller. + Map> finalAverages = new TreeMap<>(); + for (TimestampedValue obj : results) { + Assert.assertTrue("have SellerPrice", obj.getValue() instanceof SellerPrice); + SellerPrice sellerPrice = (SellerPrice) obj.getValue(); + finalAverages.put( + sellerPrice.seller, TimestampedValue.of((KnownSize) sellerPrice, obj.getTimestamp())); + } + return finalAverages.values(); + } + + @Override + protected Collection toCollection(Iterator> itr) { + return toValue(itr); + } +} diff --git a/integration/java/src/main/java/org/apache/beam/integration/nexmark/Query7.java b/integration/java/src/main/java/org/apache/beam/integration/nexmark/Query7.java new file mode 100644 index 0000000000000..1f63b35bdca87 --- /dev/null +++ b/integration/java/src/main/java/org/apache/beam/integration/nexmark/Query7.java @@ -0,0 +1,87 @@ +/* + * 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.integration.nexmark; + +import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.transforms.Max; +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.PCollection; +import org.apache.beam.sdk.values.PCollectionView; + +import org.joda.time.Duration; + +/** + * Query 7, 'Highest Bid'. Select the bids with the highest bid + * price in the last minute. In CQL syntax: + * + *

              + * SELECT Rstream(B.auction, B.price, B.bidder)
              + * FROM Bid [RANGE 1 MINUTE SLIDE 1 MINUTE] B
              + * WHERE B.price = (SELECT MAX(B1.price)
              + *                  FROM BID [RANGE 1 MINUTE SLIDE 1 MINUTE] B1);
              + * 
              + * + *

              We will use a shorter window to help make testing easier. We'll also implement this using + * a side-input in order to exercise that functionality. (A combiner, as used in Query 5, is + * a more efficient approach.). + */ +class Query7 extends NexmarkQuery { + public Query7(NexmarkConfiguration configuration) { + super(configuration, "Query7"); + } + + private PCollection applyTyped(PCollection events) { + // Window the bids. + PCollection slidingBids = events.apply(JUST_BIDS).apply( + Window.into(FixedWindows.of(Duration.standardSeconds(configuration.windowSizeSec)))); + + // Find the largest price in all bids. + // NOTE: It would be more efficient to write this query much as we did for Query5, using + // a binary combiner to accumulate the bids with maximal price. As written this query + // requires an additional scan per window, with the associated cost of snapshotted state and + // its I/O. We'll keep this implementation since it illustrates the use of side inputs. + final PCollectionView maxPriceView = + slidingBids // + .apply(BID_TO_PRICE) + .apply(Max.longsGlobally().withFanout(configuration.fanout).asSingletonView()); + + return slidingBids + // Select all bids which have that maximum price (there may be more than one). + .apply( + ParDo.named(name + ".Select") + .withSideInputs(maxPriceView) + .of(new DoFn() { + @Override + public void processElement(ProcessContext c) { + long maxPrice = c.sideInput(maxPriceView); + Bid bid = c.element(); + if (bid.price == maxPrice) { + c.output(bid); + } + } + })); + } + + @Override + protected PCollection applyPrim(PCollection events) { + return NexmarkUtils.castToKnownSize(name, applyTyped(events)); + } +} diff --git a/integration/java/src/main/java/org/apache/beam/integration/nexmark/Query7Model.java b/integration/java/src/main/java/org/apache/beam/integration/nexmark/Query7Model.java new file mode 100644 index 0000000000000..e8351336ad5b9 --- /dev/null +++ b/integration/java/src/main/java/org/apache/beam/integration/nexmark/Query7Model.java @@ -0,0 +1,128 @@ +/* + * 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.integration.nexmark; + +import org.apache.beam.sdk.transforms.windowing.BoundedWindow; +import org.apache.beam.sdk.values.TimestampedValue; + +import org.joda.time.Duration; +import org.joda.time.Instant; + +import java.io.Serializable; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Iterator; +import java.util.List; + +/** + * A direct implementation of {@link Query7}. + */ +public class Query7Model extends NexmarkQueryModel implements Serializable { + /** + * Simulator for query 7. + */ + private class Simulator extends AbstractSimulator { + /** Bids with highest bid price seen in the current window. */ + private final List highestBids; + + /** When current window started. */ + private Instant windowStart; + + private Instant lastTimestamp; + + public Simulator(NexmarkConfiguration configuration) { + super(NexmarkUtils.standardEventIterator(configuration)); + highestBids = new ArrayList<>(); + windowStart = NexmarkUtils.BEGINNING_OF_TIME; + lastTimestamp = BoundedWindow.TIMESTAMP_MIN_VALUE; + } + + /** + * Transfer the currently winning bids into results and retire them. + */ + private void retireWindow(Instant timestamp) { + for (Bid bid : highestBids) { + addResult(TimestampedValue.of(bid, timestamp)); + } + highestBids.clear(); + } + + /** + * Keep just the highest price bid. + */ + private void captureBid(Bid bid) { + Iterator itr = highestBids.iterator(); + boolean isWinning = true; + while (itr.hasNext()) { + Bid existingBid = itr.next(); + if (existingBid.price > bid.price) { + isWinning = false; + break; + } + NexmarkUtils.info("smaller price: %s", existingBid); + itr.remove(); + } + if (isWinning) { + NexmarkUtils.info("larger price: %s", bid); + highestBids.add(bid); + } + } + + @Override + protected void run() { + TimestampedValue timestampedEvent = nextInput(); + if (timestampedEvent == null) { + // Capture all remaining bids in results. + retireWindow(lastTimestamp); + allDone(); + return; + } + + Event event = timestampedEvent.getValue(); + if (event.bid == null) { + // Ignore non-bid events. + return; + } + lastTimestamp = timestampedEvent.getTimestamp(); + Instant newWindowStart = windowStart(Duration.standardSeconds(configuration.windowSizeSec), + Duration.standardSeconds(configuration.windowSizeSec), lastTimestamp); + if (!newWindowStart.equals(windowStart)) { + // Capture highest priced bids in current window and retire it. + retireWindow(lastTimestamp); + windowStart = newWindowStart; + } + // Keep only the highest bids. + captureBid(event.bid); + } + } + + public Query7Model(NexmarkConfiguration configuration) { + super(configuration); + } + + @Override + public AbstractSimulator simulator() { + return new Simulator(configuration); + } + + @Override + protected Collection toCollection(Iterator> itr) { + return toValueOrder(itr); + } +} diff --git a/integration/java/src/main/java/org/apache/beam/integration/nexmark/Query8.java b/integration/java/src/main/java/org/apache/beam/integration/nexmark/Query8.java new file mode 100644 index 0000000000000..e58453bca361a --- /dev/null +++ b/integration/java/src/main/java/org/apache/beam/integration/nexmark/Query8.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.integration.nexmark; + +import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.transforms.ParDo; +import org.apache.beam.sdk.transforms.join.CoGbkResult; +import org.apache.beam.sdk.transforms.join.CoGroupByKey; +import org.apache.beam.sdk.transforms.join.KeyedPCollectionTuple; +import org.apache.beam.sdk.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; + +/** + * Query 8, 'Monitor New Users'. Select people who have entered the system and created auctions + * in the last 12 hours, updated every 12 hours. In CQL syntax: + * + *

              + * SELECT Rstream(P.id, P.name, A.reserve)
              + * FROM Person [RANGE 12 HOUR] P, Auction [RANGE 12 HOUR] A
              + * WHERE P.id = A.seller;
              + * 
              + * + *

              To make things a bit more dynamic and easier to test we'll use a much shorter window. + */ +class Query8 extends NexmarkQuery { + public Query8(NexmarkConfiguration configuration) { + super(configuration, "Query8"); + } + + private PCollection applyTyped(PCollection events) { + // Window and key new people by their id. + PCollection> personsById = + events.apply(JUST_NEW_PERSONS) + .apply(Window.into( + FixedWindows.of(Duration.standardSeconds(configuration.windowSizeSec))) + .named("Query8.WindowPersons")) + .apply(PERSON_BY_ID); + + // Window and key new auctions by their id. + PCollection> auctionsBySeller = + events.apply(JUST_NEW_AUCTIONS) + .apply(Window.into( + FixedWindows.of(Duration.standardSeconds(configuration.windowSizeSec))) + .named("Query8.WindowAuctions")) + .apply(AUCTION_BY_SELLER); + + // Join people and auctions and project the person id, name and auction reserve price. + return KeyedPCollectionTuple.of(PERSON_TAG, personsById) + .and(AUCTION_TAG, auctionsBySeller) + .apply(CoGroupByKey.create()) + .apply( + ParDo.named(name + ".Select") + .of(new DoFn, IdNameReserve>() { + @Override + public void processElement(ProcessContext c) { + Person person = c.element().getValue().getOnly(PERSON_TAG, null); + if (person == null) { + // Person was not created in last window period. + return; + } + for (Auction auction : c.element().getValue().getAll(AUCTION_TAG)) { + c.output(new IdNameReserve(person.id, person.name, auction.reserve)); + } + } + })); + } + + @Override + protected PCollection applyPrim(PCollection events) { + return NexmarkUtils.castToKnownSize(name, applyTyped(events)); + } +} diff --git a/integration/java/src/main/java/org/apache/beam/integration/nexmark/Query8Model.java b/integration/java/src/main/java/org/apache/beam/integration/nexmark/Query8Model.java new file mode 100644 index 0000000000000..00f7355fe1d1f --- /dev/null +++ b/integration/java/src/main/java/org/apache/beam/integration/nexmark/Query8Model.java @@ -0,0 +1,145 @@ +/* + * 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.integration.nexmark; + +import org.apache.beam.sdk.values.TimestampedValue; +import com.google.common.collect.ArrayListMultimap; +import com.google.common.collect.Multimap; + +import org.joda.time.Duration; +import org.joda.time.Instant; + +import java.io.Serializable; +import java.util.Collection; +import java.util.HashMap; +import java.util.Iterator; +import java.util.Map; + +/** + * A direct implementation of {@link Query8}. + */ +public class Query8Model extends NexmarkQueryModel implements Serializable { + /** + * Simulator for query 8. + */ + private class Simulator extends AbstractSimulator { + /** New persons seen in the current window, indexed by id. */ + private final Map newPersons; + + /** New auctions seen in the current window, indexed by seller id. */ + private final Multimap newAuctions; + + /** When did the current window start. */ + private Instant windowStart; + + public Simulator(NexmarkConfiguration configuration) { + super(NexmarkUtils.standardEventIterator(configuration)); + newPersons = new HashMap<>(); + newAuctions = ArrayListMultimap.create(); + windowStart = NexmarkUtils.BEGINNING_OF_TIME; + } + + /** + * Retire all persons added in last window. + */ + private void retirePersons() { + for (Map.Entry entry : newPersons.entrySet()) { + NexmarkUtils.info("retire: %s", entry.getValue()); + } + newPersons.clear(); + } + + /** + * Retire all auctions added in last window. + */ + private void retireAuctions() { + for (Map.Entry entry : newAuctions.entries()) { + NexmarkUtils.info("retire: %s", entry.getValue()); + } + newAuctions.clear(); + } + + /** + * Capture new result. + */ + private void addResult(Auction auction, Person person, Instant timestamp) { + addResult(TimestampedValue.of( + new IdNameReserve(person.id, person.name, auction.reserve), timestamp)); + } + + @Override + public void run() { + TimestampedValue timestampedEvent = nextInput(); + if (timestampedEvent == null) { + allDone(); + return; + } + + Event event = timestampedEvent.getValue(); + if (event.bid != null) { + // Ignore bid events. + // Keep looking for next events. + return; + } + Instant timestamp = timestampedEvent.getTimestamp(); + Instant newWindowStart = windowStart(Duration.standardSeconds(configuration.windowSizeSec), + Duration.standardSeconds(configuration.windowSizeSec), timestamp); + if (!newWindowStart.equals(windowStart)) { + // Retire this window. + retirePersons(); + retireAuctions(); + windowStart = newWindowStart; + } + + if (event.newAuction != null) { + // Join new auction with existing person, if any. + Person person = newPersons.get(event.newAuction.seller); + if (person != null) { + addResult(event.newAuction, person, timestamp); + } else { + // Remember auction for future new people. + newAuctions.put(event.newAuction.seller, event.newAuction); + } + } else { + // Join new person with existing auctions. + for (Auction auction : newAuctions.get(event.newPerson.id)) { + addResult(auction, event.newPerson, timestamp); + } + // We'll never need these auctions again. + newAuctions.removeAll(event.newPerson.id); + // Remember person for future auctions. + newPersons.put(event.newPerson.id, event.newPerson); + } + } + } + + public Query8Model(NexmarkConfiguration configuration) { + super(configuration); + } + + @Override + public AbstractSimulator simulator() { + return new Simulator(configuration); + } + + @Override + protected Collection toCollection(Iterator> itr) { + return toValue(itr); + } +} diff --git a/integration/java/src/main/java/org/apache/beam/integration/nexmark/Query9.java b/integration/java/src/main/java/org/apache/beam/integration/nexmark/Query9.java new file mode 100644 index 0000000000000..2c0a5266a3912 --- /dev/null +++ b/integration/java/src/main/java/org/apache/beam/integration/nexmark/Query9.java @@ -0,0 +1,40 @@ +/* + * 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.integration.nexmark; + +import org.apache.beam.sdk.values.PCollection; + +/** + * Query "9", 'Winning bids'. Select just the winning bids. Not in original NEXMark suite, but + * handy for testing. See {@link WinningBids} for the details. + */ +class Query9 extends NexmarkQuery { + public Query9(NexmarkConfiguration configuration) { + super(configuration, "Query9"); + } + + private PCollection applyTyped(PCollection events) { + return events.apply(new WinningBids(name, configuration)); + } + + @Override + protected PCollection applyPrim(PCollection events) { + return NexmarkUtils.castToKnownSize(name, applyTyped(events)); + } +} diff --git a/integration/java/src/main/java/org/apache/beam/integration/nexmark/Query9Model.java b/integration/java/src/main/java/org/apache/beam/integration/nexmark/Query9Model.java new file mode 100644 index 0000000000000..1fad648c954ad --- /dev/null +++ b/integration/java/src/main/java/org/apache/beam/integration/nexmark/Query9Model.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.integration.nexmark; + +import org.apache.beam.sdk.values.TimestampedValue; + +import java.io.Serializable; +import java.util.Collection; +import java.util.Iterator; + +/** + * A direct implementation of {@link Query9}. + */ +public class Query9Model extends NexmarkQueryModel implements Serializable { + public Query9Model(NexmarkConfiguration configuration) { + super(configuration); + } + + @Override + public AbstractSimulator simulator() { + return new WinningBidsSimulator(configuration); + } + + @Override + protected Collection toCollection(Iterator> itr) { + return toValue(itr); + } +} diff --git a/integration/java/src/main/java/org/apache/beam/integration/nexmark/README.md b/integration/java/src/main/java/org/apache/beam/integration/nexmark/README.md new file mode 100644 index 0000000000000..5e3332722496e --- /dev/null +++ b/integration/java/src/main/java/org/apache/beam/integration/nexmark/README.md @@ -0,0 +1,166 @@ + + +# NEXMark integration suite + +This is a suite of pipelines inspired by the 'continuous data stream' +queries in [http://datalab.cs.pdx.edu/niagaraST/NEXMark/] +(http://datalab.cs.pdx.edu/niagaraST/NEXMark/). + +The queries are over a simple online auction system with tables of +**Person**, **Auction** and **Bid** records. + +The queries are: + +* **Query1**: What are the bid values in Euro's? + Illustrates a simple map. +* **Query2**: What are the auctions with particular auction numbers? + Illustrates a simple filter. +* **Query3**: Who is selling in particular US states? + Illustrates an incremental join (using per-key state) and filter. +* **Query4**: What is the average selling price for each auction + category? + Illustrates complex join (using custom window functions) and + aggregation. +* **Query5**: Which auctions have seen the most bids in the last period? + Illustrates sliding windows and combiners. +* **Query6**: What is the average selling price per seller for their + last 10 closed auctions. + Shares the same 'winning bids' core as for **Query4**, and + illustrates a specialized combiner. +* **Query7**: What are the highest bids per period? + Deliberately implemented using a side input to illustrate fanout. +* **Query8**: Who has entered the system and created an auction in + the last period? + Illustrates a simple join. + +We have augmented the original queries with five more: + +* **Query0**: Pass-through. + Allows us to measure the monitoring overhead. +* **Query9**: Winning-bids. + A common sub-query shared by **Query4** and **Query6**. +* **Query10**: Log all events to GCS files. + Illustrates windows with large side effects on firing. +* **Query11**: How many bids did a user make in each session they + were active? + Illustrates session windows. +* **Query12**: How many bids does a user make within a fixed + processing time limit? + Illustrates working in processing time in the Global window, as + compared with event time in non-Global windows for all the other + queries. + +The queries can be executed using a 'Driver' for a given backend. +Currently the supported drivers are: + +* **NexmarkInProcessDriver** for running locally on a single machine. +* **NexmarkGoogleDriver** for running on the Google Cloud Dataflow + service. Requires a Google Cloud account. +* **NexmarkFlinkDriver** for running on a Flink cluster. Requires the + cluster to be established and the Nexmark jar to be distributed to + each worker. + +Other drivers are straightforward. + +Test data is deterministically synthesized on demand. The test +data may be synthesized in the same pipeline as the query itself, +or may be published to Pubsub. + +The query results may be: + +* Published to Pubsub. +* Written to text files as plain text. +* Written to text files using an Avro encoding. +* Send to BigQuery. +* Discarded. + +Options are provided for measuring progress, measuring overall +pipeline performance, and comparing that performance against a known +baseline. However that machinery has only been implemented against +the Google Cloud Dataflow driver. + +## Running on Google Cloud Dataflow + +An example invocation for **Query10** on the Google Cloud Dataflow +service. + +``` +java -cp integration/java/target/java-integration-all-bundled-0.2.0-incubating-SNAPSHOT.jar \ + org.apache.beam.integration.nexmark.NexmarkGoogleDriver \ + --project= \ + --zone= \ + --workerMachineType=n1-highmem-8 \ + --stagingLocation= \ + --streaming=true \ + --sourceType=PUBSUB \ + --pubSubMode=PUBLISH_ONLY \ + --pubsubTopic= \ + --resourceNameMode=VERBATIM \ + --manageResources=false \ + --monitorJobs=false \ + --numEventGenerators=64 \ + --numWorkers=16 \ + --maxNumWorkers=16 \ + --query=10 \ + --firstEventRate=100000 \ + --nextEventRate=100000 \ + --ratePeriodSec=3600 \ + --isRateLimited=true \ + --avgPersonByteSize=500 \ + --avgAuctionByteSize=500 \ + --avgBidByteSize=500 \ + --probDelayedEvent=0.000001 \ + --occasionalDelaySec=3600 \ + --numEvents=0 \ + --useWallclockEventTime=true \ + --usePubsubPublishTime=true \ + --experiments=enable_custom_pubsub_sink +``` + +``` +java -cp integration/java/target/java-integration-all-bundled-0.2.0-incubating-SNAPSHOT.jar \ + org.apache.beam.integration.nexmark.NexmarkGoogleDriver \ + --project= \ + --zone= \ + --workerMachineType=n1-highmem-8 \ + --stagingLocation= \ + --streaming=true \ + --sourceType=PUBSUB \ + --pubSubMode=SUBSCRIBE_ONLY \ + --pubsubSubscription= \ + --resourceNameMode=VERBATIM \ + --manageResources=false \ + --monitorJobs=false \ + --numWorkers=64 \ + --maxNumWorkers=64 \ + --query=10 \ + --usePubsubPublishTime=true \ + --outputPath= \ + --windowSizeSec=600 \ + --occasionalDelaySec=3600 \ + --maxLogEvents=10000 \ + --experiments=enable_custom_pubsub_source +``` + +## Running on Flink + +See [BEAM_ON_FLINK_ON_GCP](./BEAM_ON_FLINK_ON_GCP.md) for instructions +on running a NexMark pipeline using Flink hosted on a Google Compute +Platform cluster. diff --git a/integration/java/src/main/java/org/apache/beam/integration/nexmark/SellerPrice.java b/integration/java/src/main/java/org/apache/beam/integration/nexmark/SellerPrice.java new file mode 100644 index 0000000000000..4324b994f8c34 --- /dev/null +++ b/integration/java/src/main/java/org/apache/beam/integration/nexmark/SellerPrice.java @@ -0,0 +1,91 @@ +/* + * 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.integration.nexmark; + +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.VarLongCoder; + +import com.fasterxml.jackson.annotation.JsonProperty; +import com.fasterxml.jackson.core.JsonProcessingException; + +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.io.Serializable; + +/** + * Result of {@link Query6}. + */ +public class SellerPrice implements KnownSize, Serializable { + private static final Coder LONG_CODER = VarLongCoder.of(); + + public static final Coder CODER = new AtomicCoder() { + @Override + public void encode(SellerPrice value, OutputStream outStream, + Coder.Context context) + throws CoderException, IOException { + LONG_CODER.encode(value.seller, outStream, Context.NESTED); + LONG_CODER.encode(value.price, outStream, Context.NESTED); + } + + @Override + public SellerPrice decode( + InputStream inStream, Coder.Context context) + throws CoderException, IOException { + long seller = LONG_CODER.decode(inStream, Context.NESTED); + long price = LONG_CODER.decode(inStream, Context.NESTED); + return new SellerPrice(seller, price); + } + }; + + @JsonProperty + public final long seller; + + /** Price in cents. */ + @JsonProperty + public final long price; + + // For Avro only. + @SuppressWarnings("unused") + private SellerPrice() { + seller = 0; + price = 0; + } + + public SellerPrice(long seller, long price) { + this.seller = seller; + this.price = price; + } + + @Override + public long sizeInBytes() { + return 8 + 8; + } + + @Override + public String toString() { + try { + return NexmarkUtils.MAPPER.writeValueAsString(this); + } catch (JsonProcessingException e) { + throw new RuntimeException(e); + } + } +} diff --git a/integration/java/src/main/java/org/apache/beam/integration/nexmark/UnboundedEventSource.java b/integration/java/src/main/java/org/apache/beam/integration/nexmark/UnboundedEventSource.java new file mode 100644 index 0000000000000..2898251f62c57 --- /dev/null +++ b/integration/java/src/main/java/org/apache/beam/integration/nexmark/UnboundedEventSource.java @@ -0,0 +1,329 @@ +/* + * 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.integration.nexmark; + +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.io.UnboundedSource; +import org.apache.beam.sdk.options.PipelineOptions; +import org.apache.beam.sdk.transforms.windowing.BoundedWindow; +import org.apache.beam.sdk.values.TimestampedValue; + +import org.joda.time.Duration; +import org.joda.time.Instant; + +import java.util.ArrayList; +import java.util.List; +import java.util.NoSuchElementException; +import java.util.PriorityQueue; +import java.util.Queue; +import java.util.concurrent.ThreadLocalRandom; + +import javax.annotation.Nullable; + +/** + * A custom, unbounded source of event records. + * + *

              If {@code isRateLimited} is true, events become available for return from the reader such + * that the overall rate respect the {@code interEventDelayUs} period if possible. Otherwise, + * events are returned every time the system asks for one. + */ +class UnboundedEventSource extends UnboundedSource { + private static final Duration BACKLOG_PERIOD = Duration.standardSeconds(30); + + /** Configuration for generator to use when reading synthetic events. May be split. */ + private final GeneratorConfig config; + + /** How many unbounded sources to create. */ + private final int numEventGenerators; + + /** How many seconds to hold back the watermark. */ + private final long watermarkHoldbackSec; + + /** Are we rate limiting the events? */ + private final boolean isRateLimited; + + public UnboundedEventSource(GeneratorConfig config, int numEventGenerators, + long watermarkHoldbackSec, boolean isRateLimited) { + this.config = config; + this.numEventGenerators = numEventGenerators; + this.watermarkHoldbackSec = watermarkHoldbackSec; + this.isRateLimited = isRateLimited; + } + + /** A reader to pull events from the generator. */ + private class EventReader extends UnboundedReader { + /** Generator we are reading from. */ + private final Generator generator; + + /** + * Current watermark (ms since epoch). Initially set to beginning of time. + * Then updated to be the time of the next generated event. + * Then, once all events have been generated, set to the end of time. + */ + private long watermark; + + /** + * Current backlog (ms), as delay between timestamp of last returned event and the timestamp + * we should be up to according to wall-clock time. Used only for logging. + */ + private long backlogDurationMs; + + /** + * Current backlog, as estimated number of event bytes we are behind, or null if + * unknown. Reported to callers. + */ + @Nullable + private Long backlogBytes; + + /** + * Wallclock time (ms since epoch) we last reported the backlog, or -1 if never reported. + */ + private long lastReportedBacklogWallclock; + + /** + * Event time (ms since epoch) of pending event at last reported backlog, or -1 if never + * calculated. + */ + private long timestampAtLastReportedBacklogMs; + + /** Next event to make 'current' when wallclock time has advanced sufficiently. */ + @Nullable + private TimestampedValue pendingEvent; + + /** Wallclock time when {@link #pendingEvent} is due, or -1 if no pending event. */ + private long pendingEventWallclockTime; + + /** Current event to return from getCurrent. */ + @Nullable + private TimestampedValue currentEvent; + + /** Events which have been held back so as to force them to be late. */ + private Queue heldBackEvents = new PriorityQueue<>(); + + public EventReader(Generator generator) { + this.generator = generator; + watermark = NexmarkUtils.BEGINNING_OF_TIME.getMillis(); + lastReportedBacklogWallclock = -1; + pendingEventWallclockTime = -1; + timestampAtLastReportedBacklogMs = -1; + } + + public EventReader(GeneratorConfig config) { + this(new Generator(config)); + } + + @Override + public boolean start() { + NexmarkUtils.error("starting unbounded generator %s", generator); + return advance(); + } + + + @Override + public boolean advance() { + long now = System.currentTimeMillis(); + + while (pendingEvent == null) { + if (!generator.hasNext() && heldBackEvents.isEmpty()) { + // No more events, EVER. + if (isRateLimited) { + updateBacklog(System.currentTimeMillis(), 0); + } + if (watermark < BoundedWindow.TIMESTAMP_MAX_VALUE.getMillis()) { + watermark = BoundedWindow.TIMESTAMP_MAX_VALUE.getMillis(); + NexmarkUtils.error("stopped unbounded generator %s", generator); + } + return false; + } + + Generator.NextEvent next = heldBackEvents.peek(); + if (next != null && next.wallclockTimestamp <= now) { + // Time to use the held-back event. + heldBackEvents.poll(); + NexmarkUtils.error("replaying held-back event %dms behind watermark", + watermark - next.eventTimestamp); + } else if (generator.hasNext()) { + next = generator.nextEvent(); + if (isRateLimited && config.configuration.probDelayedEvent > 0.0 + && config.configuration.occasionalDelaySec > 0 + && ThreadLocalRandom.current().nextDouble() < config.configuration.probDelayedEvent) { + // We'll hold back this event and go around again. + long delayMs = + ThreadLocalRandom.current().nextLong(config.configuration.occasionalDelaySec * 1000) + + 1L; + NexmarkUtils.error("delaying event by %dms", delayMs); + heldBackEvents.add(next.withDelay(delayMs)); + continue; + } + } else { + // Waiting for held-back event to fire. + if (isRateLimited) { + updateBacklog(now, 0); + } + return false; + } + + pendingEventWallclockTime = next.wallclockTimestamp; + pendingEvent = TimestampedValue.of(next.event, new Instant(next.eventTimestamp)); + long newWatermark = + next.watermark - Duration.standardSeconds(watermarkHoldbackSec).getMillis(); + if (newWatermark > watermark) { + watermark = newWatermark; + } + } + + if (isRateLimited) { + if (pendingEventWallclockTime > now) { + // We want this event to fire in the future. Try again later. + updateBacklog(now, 0); + return false; + } + updateBacklog(now, now - pendingEventWallclockTime); + } + + // This event is ready to fire. + currentEvent = pendingEvent; + pendingEvent = null; + return true; + } + + private void updateBacklog(long now, long newBacklogDurationMs) { + backlogDurationMs = newBacklogDurationMs; + long interEventDelayUs = generator.currentInterEventDelayUs(); + if (interEventDelayUs != 0) { + long backlogEvents = (backlogDurationMs * 1000 + interEventDelayUs - 1) / interEventDelayUs; + backlogBytes = generator.getCurrentConfig().estimatedBytesForEvents(backlogEvents); + } + if (lastReportedBacklogWallclock < 0 + || now - lastReportedBacklogWallclock > BACKLOG_PERIOD.getMillis()) { + double timeDialation = Double.NaN; + if (pendingEvent != null + && lastReportedBacklogWallclock >= 0 + && timestampAtLastReportedBacklogMs >= 0) { + long wallclockProgressionMs = now - lastReportedBacklogWallclock; + long eventTimeProgressionMs = + pendingEvent.getTimestamp().getMillis() - timestampAtLastReportedBacklogMs; + timeDialation = (double) eventTimeProgressionMs / (double) wallclockProgressionMs; + } + NexmarkUtils.error( + "unbounded generator backlog now %dms (%s bytes) at %dus interEventDelay " + + "with %f time dilation", + backlogDurationMs, backlogBytes, interEventDelayUs, timeDialation); + lastReportedBacklogWallclock = now; + if (pendingEvent != null) { + timestampAtLastReportedBacklogMs = pendingEvent.getTimestamp().getMillis(); + } + } + } + + @Override + public Event getCurrent() { + if (currentEvent == null) { + throw new NoSuchElementException(); + } + return currentEvent.getValue(); + } + + @Override + public Instant getCurrentTimestamp() { + if (currentEvent == null) { + throw new NoSuchElementException(); + } + return currentEvent.getTimestamp(); + } + + @Override + public void close() { + // Nothing to close. + } + + @Override + public UnboundedEventSource getCurrentSource() { + return UnboundedEventSource.this; + } + + @Override + public Instant getWatermark() { + return new Instant(watermark); + } + + @Override + public Generator.Checkpoint getCheckpointMark() { + return generator.toCheckpoint(); + } + + @Override + public long getSplitBacklogBytes() { + return backlogBytes == null ? BACKLOG_UNKNOWN : backlogBytes; + } + + @Override + public String toString() { + return String.format("EventReader(%d, %d, %d)", + generator.getCurrentConfig().getStartEventId(), generator.getNextEventId(), + generator.getCurrentConfig().getStopEventId()); + } + } + + @Override + public Coder getCheckpointMarkCoder() { + return Generator.Checkpoint.CODER_INSTANCE; + } + + @Override + public List generateInitialSplits( + int desiredNumSplits, PipelineOptions options) { + NexmarkUtils.error( + "splitting unbounded source %s into %d sub-sources", config, numEventGenerators); + List results = new ArrayList<>(); + // Ignore desiredNumSplits and use numEventGenerators instead. + for (GeneratorConfig subConfig : config.split(numEventGenerators)) { + results.add(new UnboundedEventSource(subConfig, 1, watermarkHoldbackSec, isRateLimited)); + } + return results; + } + + @Override + public EventReader createReader( + PipelineOptions options, @Nullable Generator.Checkpoint checkpoint) { + if (checkpoint == null) { + NexmarkUtils.error("creating initial unbounded reader for %s", config); + return new EventReader(config); + } else { + NexmarkUtils.error("resuming unbounded reader from %s", checkpoint); + return new EventReader(checkpoint.toGenerator(config)); + } + } + + @Override + public void validate() { + // Nothing to validate. + } + + @Override + public Coder getDefaultOutputCoder() { + return Event.CODER; + } + + @Override + public String toString() { + return String.format( + "UnboundedEventSource(%d, %d)", config.getStartEventId(), config.getStopEventId()); + } +} diff --git a/integration/java/src/main/java/org/apache/beam/integration/nexmark/WinningBids.java b/integration/java/src/main/java/org/apache/beam/integration/nexmark/WinningBids.java new file mode 100644 index 0000000000000..16f901c7a6c44 --- /dev/null +++ b/integration/java/src/main/java/org/apache/beam/integration/nexmark/WinningBids.java @@ -0,0 +1,378 @@ +/* + * 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.integration.nexmark; + +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.VarIntCoder; +import org.apache.beam.sdk.coders.VarLongCoder; +import org.apache.beam.sdk.transforms.Aggregator; +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.SumLongFn; +import org.apache.beam.sdk.transforms.join.CoGbkResult; +import org.apache.beam.sdk.transforms.join.CoGroupByKey; +import org.apache.beam.sdk.transforms.join.KeyedPCollectionTuple; +import org.apache.beam.sdk.transforms.windowing.BoundedWindow; +import org.apache.beam.sdk.transforms.windowing.IntervalWindow; +import org.apache.beam.sdk.transforms.windowing.Window; +import org.apache.beam.sdk.transforms.windowing.WindowFn; +import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.PCollection; +import com.google.common.base.Preconditions; + +import com.fasterxml.jackson.annotation.JsonCreator; + +import org.joda.time.Instant; + +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.io.Serializable; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.List; +import java.util.Map; +import java.util.TreeMap; + +/** + * A transform to find the winning bid for each closed auction. In pseudo CQL syntax: + * + *

              + * SELECT Rstream(A.*, B.auction, B.bidder, MAX(B.price), B.dateTime)
              + * FROM Auction A [ROWS UNBOUNDED], Bid B [ROWS UNBOUNDED]
              + * WHERE A.id=B.auction AND B.datetime < A.expires AND A.expires < CURRENT_TIME
              + * GROUP BY A.id
              + * 
              + * + *

              We will also check that the winning bid is above the auction reserve. Note that + * we ignore the auction opening bid value since it has no impact on which bid eventually wins, + * if any. + * + *

              Our implementation will use a custom windowing function in order to bring bids and + * auctions together without requiring global state. + */ +public class WinningBids extends PTransform, PCollection> { + /** Windows for open auctions and bids. */ + private static class AuctionOrBidWindow extends IntervalWindow implements Serializable { + /** Id of auction this window is for. */ + public final long auction; + + /** + * True if this window represents an actual auction, and thus has a start/end + * time matching that of the auction. False if this window represents a bid, and + * thus has an unbounded start/end time. + */ + public final boolean isAuctionWindow; + + /** For avro only. */ + private AuctionOrBidWindow() { + super(TIMESTAMP_MIN_VALUE, TIMESTAMP_MAX_VALUE); + auction = 0; + isAuctionWindow = false; + } + + private AuctionOrBidWindow( + Instant start, Instant end, long auctionId, boolean isAuctionWindow) { + super(start, end); + this.auction = auctionId; + this.isAuctionWindow = isAuctionWindow; + } + + /** Return an auction window for {@code auction}. */ + public static AuctionOrBidWindow forAuction(Instant timestamp, Auction auction) { + AuctionOrBidWindow result = + new AuctionOrBidWindow(timestamp, new Instant(auction.expires), auction.id, true); + return result; + } + + /** + * Return a bid window for {@code bid}. It should later be merged into + * the corresponding auction window. However, it is possible this bid is for an already + * expired auction, or for an auction which the system has not yet seen. So we + * give the bid a bit of wiggle room in its interval. + */ + public static AuctionOrBidWindow forBid( + long expectedAuctionDurationMs, Instant timestamp, Bid bid) { + // At this point we don't know which auctions are still valid, and the bid may + // be for an auction which won't start until some unknown time in the future + // (due to Generator.AUCTION_ID_LEAD in Generator.nextBid). + // A real system would atomically reconcile bids and auctions by a separate mechanism. + // If we give bids an unbounded window it is possible a bid for an auction which + // has already expired would cause the system watermark to stall, since that window + // would never be retired. + // Instead, we will just give the bid a finite window which expires at + // the upper bound of auctions assuming the auction starts at the same time as the bid, + // and assuming the system is running at its lowest event rate (as per interEventDelayUs). + AuctionOrBidWindow result = new AuctionOrBidWindow( + timestamp, timestamp.plus(expectedAuctionDurationMs * 2), bid.auction, false); + return result; + } + + /** Is this an auction window? */ + public boolean isAuctionWindow() { + return isAuctionWindow; + } + + @Override + public String toString() { + return String.format("AuctionOrBidWindow{start:%s; end:%s; auction:%d; isAuctionWindow:%s}", + start(), end(), auction, isAuctionWindow); + } + } + + /** + * Encodes an {@link AuctionOrBidWindow} as an {@link IntervalWindow} and an auction id long. + */ + private static class AuctionOrBidWindowCoder extends AtomicCoder { + private static final AuctionOrBidWindowCoder INSTANCE = new AuctionOrBidWindowCoder(); + private static final Coder SUPER_CODER = IntervalWindow.getCoder(); + private static final Coder ID_CODER = VarLongCoder.of(); + private static final Coder INT_CODER = VarIntCoder.of(); + + @JsonCreator + public static AuctionOrBidWindowCoder of() { + return INSTANCE; + } + + @Override + public void encode(AuctionOrBidWindow window, OutputStream outStream, Context context) + throws IOException, CoderException { + SUPER_CODER.encode(window, outStream, Context.NESTED); + ID_CODER.encode(window.auction, outStream, Context.NESTED); + INT_CODER.encode(window.isAuctionWindow ? 1 : 0, outStream, Context.NESTED); + } + + @Override + public AuctionOrBidWindow decode(InputStream inStream, Context context) + throws IOException, CoderException { + IntervalWindow superWindow = SUPER_CODER.decode(inStream, Context.NESTED); + long auction = ID_CODER.decode(inStream, Context.NESTED); + boolean isAuctionWindow = INT_CODER.decode(inStream, Context.NESTED) == 0 ? false : true; + return new AuctionOrBidWindow( + superWindow.start(), superWindow.end(), auction, isAuctionWindow); + } + } + + /** Assign events to auction windows and merges them intelligently. */ + private static class AuctionOrBidWindowFn extends WindowFn { + /** Expected duration of auctions in ms. */ + private final long expectedAuctionDurationMs; + + public AuctionOrBidWindowFn(long expectedAuctionDurationMs) { + this.expectedAuctionDurationMs = expectedAuctionDurationMs; + } + + @Override + public Collection assignWindows(AssignContext c) { + Event event = c.element(); + if (event.newAuction != null) { + // Assign auctions to an auction window which expires at the auction's close. + return Arrays.asList(AuctionOrBidWindow.forAuction(c.timestamp(), event.newAuction)); + } else if (event.bid != null) { + // Assign bids to a temporary bid window which will later be merged into the appropriate + // auction window. + return Arrays.asList( + AuctionOrBidWindow.forBid(expectedAuctionDurationMs, c.timestamp(), event.bid)); + } else { + // Don't assign people to any window. They will thus be dropped. + return Arrays.asList(); + } + } + + @Override + public void mergeWindows(MergeContext c) throws Exception { + // Split and index the auction and bid windows by auction id. + Map idToTrueAuctionWindow = new TreeMap<>(); + Map> idToBidAuctionWindows = new TreeMap<>(); + for (AuctionOrBidWindow window : c.windows()) { + if (window.isAuctionWindow()) { + idToTrueAuctionWindow.put(window.auction, window); + } else { + List bidWindows = idToBidAuctionWindows.get(window.auction); + if (bidWindows == null) { + bidWindows = new ArrayList<>(); + idToBidAuctionWindows.put(window.auction, bidWindows); + } + bidWindows.add(window); + } + } + + // Merge all 'bid' windows into their corresponding 'auction' window, provided the + // auction has not expired. + for (long auction : idToTrueAuctionWindow.keySet()) { + AuctionOrBidWindow auctionWindow = idToTrueAuctionWindow.get(auction); + List bidWindows = idToBidAuctionWindows.get(auction); + if (bidWindows != null) { + List toBeMerged = new ArrayList<>(); + for (AuctionOrBidWindow bidWindow : bidWindows) { + if (bidWindow.start().isBefore(auctionWindow.end())) { + toBeMerged.add(bidWindow); + } + // else: This bid window will remain until its expire time, at which point it + // will expire without ever contributing to an output. + } + if (!toBeMerged.isEmpty()) { + toBeMerged.add(auctionWindow); + c.merge(toBeMerged, auctionWindow); + } + } + } + } + + @Override + public boolean isCompatible(WindowFn other) { + return other instanceof AuctionOrBidWindowFn; + } + + @Override + public Coder windowCoder() { + return AuctionOrBidWindowCoder.of(); + } + + @Override + public AuctionOrBidWindow getSideInputWindow(BoundedWindow window) { + throw new UnsupportedOperationException("AuctionWindowFn not supported for side inputs"); + } + + /** + * Below we will GBK auctions and bids on their auction ids. Then we will reduce those + * per id to emit {@code (auction, winning bid)} pairs for auctions which have expired with at + * least one valid bid. We would like those output pairs to have a timestamp of the auction's + * expiry (since that's the earliest we know for sure we have the correct winner). We would + * also like to make that winning results are available to following stages at the auction's + * expiry. + * + *

              + * Each result of the GBK will have a timestamp of the min of the result of this object's + * assignOutputTime over all records which end up in one of its iterables. Thus we get the + * desired behavior if we ignore each record's timestamp and always return the auction window's + * 'maxTimestamp', which will correspond to the auction's expiry. + * + *

              + * In contrast, if this object's assignOutputTime were to return 'inputTimestamp' + * (the usual implementation), then each GBK record will take as its timestamp the minimum of + * the timestamps of all bids and auctions within it, which will always be the auction's + * timestamp. An auction which expires well into the future would thus hold up the watermark + * of the GBK results until that auction expired. That in turn would hold up all winning pairs. + */ + @Override + public Instant getOutputTime( + Instant inputTimestamp, AuctionOrBidWindow window) { + return window.maxTimestamp(); + } + } + + private final AuctionOrBidWindowFn auctionOrBidWindowFn; + + public WinningBids(String name, NexmarkConfiguration configuration) { + super(name); + // What's the expected auction time (when the system is running at the lowest event rate). + long[] interEventDelayUs = configuration.rateShape.interEventDelayUs( + configuration.firstEventRate, configuration.nextEventRate, + configuration.rateUnit, configuration.numEventGenerators); + long longestDelayUs = 0; + for (int i = 0; i < interEventDelayUs.length; i++) { + longestDelayUs = Math.max(longestDelayUs, interEventDelayUs[i]); + } + // Adjust for proportion of auction events amongst all events. + longestDelayUs = + (longestDelayUs * GeneratorConfig.PROPORTION_DENOMINATOR) + / GeneratorConfig.AUCTION_PROPORTION; + // Adjust for number of in-flight auctions. + longestDelayUs = longestDelayUs * configuration.numInFlightAuctions; + long expectedAuctionDurationMs = (longestDelayUs + 999) / 1000; + NexmarkUtils.console("Expected auction duration is %d ms", expectedAuctionDurationMs); + auctionOrBidWindowFn = new AuctionOrBidWindowFn(expectedAuctionDurationMs); + } + + @Override + public PCollection apply(PCollection events) { + // Window auctions and bids into custom auction windows. New people events will be discarded. + // This will allow us to bring bids and auctions together irrespective of how long + // each auction is open for. + events = events.apply(Window.named("Window").into(auctionOrBidWindowFn)); + + // Key auctions by their id. + PCollection> auctionsById = + events.apply(NexmarkQuery.JUST_NEW_AUCTIONS).apply(NexmarkQuery.AUCTION_BY_ID); + + // Key bids by their auction id. + PCollection> bidsByAuctionId = + events.apply(NexmarkQuery.JUST_BIDS).apply(NexmarkQuery.BID_BY_AUCTION); + + // Find the highest price valid bid for each closed auction. + return + // Join auctions and bids. + KeyedPCollectionTuple.of(NexmarkQuery.AUCTION_TAG, auctionsById) + .and(NexmarkQuery.BID_TAG, bidsByAuctionId) + .apply(CoGroupByKey.create()) + + // Filter and select. + .apply( + ParDo.named(name + ".Join") + .of(new DoFn, AuctionBid>() { + final Aggregator noAuctionCounter = + createAggregator("noAuction", new SumLongFn()); + final Aggregator underReserveCounter = + createAggregator("underReserve", new SumLongFn()); + final Aggregator noValidBidsCounter = + createAggregator("noValidBids", new SumLongFn()); + + + @Override + public void processElement(ProcessContext c) { + Auction auction = + c.element().getValue().getOnly(NexmarkQuery.AUCTION_TAG, null); + if (auction == null) { + // We have bids without a matching auction. Give up. + noAuctionCounter.addValue(1L); + return; + } + // Find the current winning bid for auction. + // The earliest bid with the maximum price above the reserve wins. + Bid bestBid = null; + for (Bid bid : c.element().getValue().getAll(NexmarkQuery.BID_TAG)) { + // Bids too late for their auction will have been + // filtered out by the window merge function. + Preconditions.checkState(bid.dateTime < auction.expires); + if (bid.price < auction.reserve) { + // Bid price is below auction reserve. + underReserveCounter.addValue(1L); + continue; + } + + if (bestBid == null + || Bid.PRICE_THEN_DESCENDING_TIME.compare(bid, bestBid) > 0) { + bestBid = bid; + } + } + if (bestBid == null) { + // We don't have any valid bids for auction. + noValidBidsCounter.addValue(1L); + return; + } + c.output(new AuctionBid(auction, bestBid)); + } + })); + } +} diff --git a/integration/java/src/main/java/org/apache/beam/integration/nexmark/WinningBidsSimulator.java b/integration/java/src/main/java/org/apache/beam/integration/nexmark/WinningBidsSimulator.java new file mode 100644 index 0000000000000..b61aed1c220b6 --- /dev/null +++ b/integration/java/src/main/java/org/apache/beam/integration/nexmark/WinningBidsSimulator.java @@ -0,0 +1,203 @@ +/* + * 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.integration.nexmark; + +import org.apache.beam.sdk.transforms.windowing.BoundedWindow; +import org.apache.beam.sdk.values.TimestampedValue; + +import org.joda.time.Instant; + +import java.util.ArrayList; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.TreeMap; +import java.util.TreeSet; + +import javax.annotation.Nullable; + +/** + * A simulator of the {@code WinningBids} query. + */ +public class WinningBidsSimulator extends AbstractSimulator { + /** Auctions currently still open, indexed by auction id. */ + private final Map openAuctions; + + /** The ids of auctions known to be closed. */ + private final Set closedAuctions; + + /** Current best valid bids for open auctions, indexed by auction id. */ + private final Map bestBids; + + /** Bids for auctions we havn't seen yet. */ + private final List bidsWithoutAuctions; + + /** + * Timestamp of last new auction or bid event (ms since epoch). + */ + private long lastTimestamp; + + public WinningBidsSimulator(NexmarkConfiguration configuration) { + super(NexmarkUtils.standardEventIterator(configuration)); + openAuctions = new TreeMap<>(); + closedAuctions = new TreeSet<>(); + bestBids = new TreeMap<>(); + bidsWithoutAuctions = new ArrayList<>(); + lastTimestamp = BoundedWindow.TIMESTAMP_MIN_VALUE.getMillis(); + } + + /** + * Try to account for {@code bid} in state. Return true if bid has now been + * accounted for by {@code bestBids}. + */ + private boolean captureBestBid(Bid bid, boolean shouldLog) { + if (closedAuctions.contains(bid.auction)) { + // Ignore bids for known, closed auctions. + if (shouldLog) { + NexmarkUtils.info("closed auction: %s", bid); + } + return true; + } + Auction auction = openAuctions.get(bid.auction); + if (auction == null) { + // We don't have an auction for this bid yet, so can't determine if it is + // winning or not. + if (shouldLog) { + NexmarkUtils.info("pending auction: %s", bid); + } + return false; + } + if (bid.price < auction.reserve) { + // Bid price is too low. + if (shouldLog) { + NexmarkUtils.info("below reserve: %s", bid); + } + return true; + } + Bid existingBid = bestBids.get(bid.auction); + if (existingBid == null || Bid.PRICE_THEN_DESCENDING_TIME.compare(existingBid, bid) < 0) { + // We've found a (new) best bid for a known auction. + bestBids.put(bid.auction, bid); + if (shouldLog) { + NexmarkUtils.info("new winning bid: %s", bid); + } + } else { + if (shouldLog) { + NexmarkUtils.info("ignoring low bid: %s", bid); + } + } + return true; + } + + /** + * Try to match bids without auctions to auctions. + */ + private void flushBidsWithoutAuctions() { + Iterator itr = bidsWithoutAuctions.iterator(); + while (itr.hasNext()) { + Bid bid = itr.next(); + if (captureBestBid(bid, false)) { + NexmarkUtils.info("bid now accounted for: %s", bid); + itr.remove(); + } + } + } + + /** + * Return the next winning bid for an expired auction relative to {@code timestamp}. + * Return null if no more winning bids, in which case all expired auctions will + * have been removed from our state. Retire auctions in order of expire time. + */ + @Nullable + private TimestampedValue nextWinningBid(long timestamp) { + Map> toBeRetired = new TreeMap<>(); + for (Map.Entry entry : openAuctions.entrySet()) { + if (entry.getValue().expires <= timestamp) { + List idsAtTime = toBeRetired.get(entry.getValue().expires); + if (idsAtTime == null) { + idsAtTime = new ArrayList<>(); + toBeRetired.put(entry.getValue().expires, idsAtTime); + } + idsAtTime.add(entry.getKey()); + } + } + for (Map.Entry> entry : toBeRetired.entrySet()) { + for (long id : entry.getValue()) { + Auction auction = openAuctions.get(id); + NexmarkUtils.info("retiring auction: %s", auction); + openAuctions.remove(id); + Bid bestBid = bestBids.get(id); + if (bestBid != null) { + TimestampedValue result = + TimestampedValue.of(new AuctionBid(auction, bestBid), new Instant(auction.expires)); + NexmarkUtils.info("winning: %s", result); + return result; + } + } + } + return null; + } + + @Override + protected void run() { + if (lastTimestamp > BoundedWindow.TIMESTAMP_MIN_VALUE.getMillis()) { + // We may have finally seen the auction a bid was intended for. + flushBidsWithoutAuctions(); + TimestampedValue result = nextWinningBid(lastTimestamp); + if (result != null) { + addResult(result); + return; + } + } + + TimestampedValue timestampedEvent = nextInput(); + if (timestampedEvent == null) { + // No more events. Flush any still open auctions. + TimestampedValue result = + nextWinningBid(BoundedWindow.TIMESTAMP_MAX_VALUE.getMillis()); + if (result == null) { + // We are done. + allDone(); + return; + } + addResult(result); + return; + } + + Event event = timestampedEvent.getValue(); + if (event.newPerson != null) { + // Ignore new person events. + return; + } + + lastTimestamp = timestampedEvent.getTimestamp().getMillis(); + if (event.newAuction != null) { + // Add this new open auction to our state. + openAuctions.put(event.newAuction.id, event.newAuction); + } else { + if (!captureBestBid(event.bid, true)) { + // We don't know what to do with this bid yet. + NexmarkUtils.info("bid not yet accounted for: %s", event.bid); + bidsWithoutAuctions.add(event.bid); + } + } + // Keep looking for winning bids. + } +} diff --git a/integration/java/src/test/java/org/apache/beam/integration/nexmark/BoundedEventSourceTest.java b/integration/java/src/test/java/org/apache/beam/integration/nexmark/BoundedEventSourceTest.java new file mode 100644 index 0000000000000..f0172679e463a --- /dev/null +++ b/integration/java/src/test/java/org/apache/beam/integration/nexmark/BoundedEventSourceTest.java @@ -0,0 +1,71 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.beam.integration.nexmark; + +import org.apache.beam.runners.dataflow.options.DataflowPipelineOptions; +import org.apache.beam.sdk.options.PipelineOptionsFactory; +import org.apache.beam.sdk.testing.SourceTestUtils; + +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +/** + * Test {@link BoundedEventSource}. + */ +@RunWith(JUnit4.class) +public class BoundedEventSourceTest { + private GeneratorConfig makeConfig(long n) { + return new GeneratorConfig( + NexmarkConfiguration.DEFAULT, System.currentTimeMillis(), 0, n, 0); + } + + @Test + public void sourceAndReadersWork() throws Exception { + DataflowPipelineOptions options = PipelineOptionsFactory.as(DataflowPipelineOptions.class); + long n = 200L; + BoundedEventSource source = new BoundedEventSource(makeConfig(n), 1); + + SourceTestUtils.assertUnstartedReaderReadsSameAsItsSource( + source.createReader(options), options); + } + + @Test + public void splitAtFractionRespectsContract() throws Exception { + DataflowPipelineOptions options = PipelineOptionsFactory.as(DataflowPipelineOptions.class); + long n = 20L; + BoundedEventSource source = new BoundedEventSource(makeConfig(n), 1); + + // Can't split if already consumed. + SourceTestUtils.assertSplitAtFractionFails(source, 10, 0.3, options); + + SourceTestUtils.assertSplitAtFractionSucceedsAndConsistent(source, 5, 0.3, options); + + SourceTestUtils.assertSplitAtFractionExhaustive(source, options); + } + + @Test + public void splitIntoBundlesRespectsContract() throws Exception { + DataflowPipelineOptions options = PipelineOptionsFactory.as(DataflowPipelineOptions.class); + long n = 200L; + BoundedEventSource source = new BoundedEventSource(makeConfig(n), 1); + SourceTestUtils.assertSourcesEqualReferenceSource( + source, source.splitIntoBundles(10, options), options); + } +} diff --git a/integration/java/src/test/java/org/apache/beam/integration/nexmark/GeneratorTest.java b/integration/java/src/test/java/org/apache/beam/integration/nexmark/GeneratorTest.java new file mode 100644 index 0000000000000..bbaee26c25c42 --- /dev/null +++ b/integration/java/src/test/java/org/apache/beam/integration/nexmark/GeneratorTest.java @@ -0,0 +1,111 @@ +/* + * 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.integration.nexmark; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +import java.util.ArrayList; +import java.util.Iterator; +import java.util.List; + +/** + * Test {@link Generator}. + */ +@RunWith(JUnit4.class) +public class GeneratorTest { + private GeneratorConfig makeConfig(long n) { + return new GeneratorConfig( + NexmarkConfiguration.DEFAULT, System.currentTimeMillis(), 0, n, 0); + } + + private long consume(long n, Iterator itr) { + for (long i = 0; i < n; i++) { + assertTrue(itr.hasNext()); + itr.next(); + } + return n; + } + + private long consume(Iterator itr) { + long n = 0; + while (itr.hasNext()) { + itr.next(); + n++; + } + return n; + } + + @Test + public void splitAtFractionPreservesOverallEventCount() { + long n = 55729L; + GeneratorConfig initialConfig = makeConfig(n); + long expected = initialConfig.getStopEventId() - initialConfig.getStartEventId(); + + long actual = 0; + + Generator initialGenerator = new Generator(initialConfig); + + // Consume some events. + actual += consume(5000, initialGenerator); + + + // Split once. + GeneratorConfig remainConfig1 = initialGenerator.splitAtEventId(9000L); + Generator remainGenerator1 = new Generator(remainConfig1); + + // Consume some more events. + actual += consume(2000, initialGenerator); + actual += consume(3000, remainGenerator1); + + // Split again. + GeneratorConfig remainConfig2 = remainGenerator1.splitAtEventId(30000L); + Generator remainGenerator2 = new Generator(remainConfig2); + + // Run to completion. + actual += consume(initialGenerator); + actual += consume(remainGenerator1); + actual += consume(remainGenerator2); + + assertEquals(expected, actual); + } + + @Test + public void splitPreservesOverallEventCount() { + long n = 51237L; + GeneratorConfig initialConfig = makeConfig(n); + long expected = initialConfig.getStopEventId() - initialConfig.getStartEventId(); + + List generators = new ArrayList<>(); + for (GeneratorConfig subConfig : initialConfig.split(20)) { + generators.add(new Generator(subConfig)); + } + + long actual = 0; + for (Generator generator : generators) { + actual += consume(generator); + } + + assertEquals(expected, actual); + } +} diff --git a/integration/java/src/test/java/org/apache/beam/integration/nexmark/QueryTest.java b/integration/java/src/test/java/org/apache/beam/integration/nexmark/QueryTest.java new file mode 100644 index 0000000000000..860fa78b13a68 --- /dev/null +++ b/integration/java/src/test/java/org/apache/beam/integration/nexmark/QueryTest.java @@ -0,0 +1,103 @@ +/* + * 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.integration.nexmark; + +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.PCollection.IsBounded; +import org.apache.beam.sdk.values.TimestampedValue; + +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +/** + * Test the various NEXMark queries yield results coherent with their models. + */ +@RunWith(JUnit4.class) +public class QueryTest { + private static final NexmarkConfiguration CONFIG = NexmarkConfiguration.DEFAULT.clone(); + + static { + CONFIG.numEvents = 2000; + } + + /** Test {@code query} matches {@code model}. */ + private static void queryMatchesModel(String name, NexmarkQuery query, NexmarkQueryModel model) { + Pipeline p = TestPipeline.create(); + NexmarkUtils.setupPipeline(NexmarkUtils.CoderStrategy.HAND, p); + PCollection> results = + p.apply(NexmarkUtils.batchEventsSource(name, CONFIG)).apply(query); + results.setIsBoundedInternal(IsBounded.BOUNDED); + PAssert.that(results).satisfies(model.assertionFor()); + p.run(); + } + + @Test + public void query0MatchesModel() { + queryMatchesModel("Query0Test", new Query0(CONFIG), new Query0Model(CONFIG)); + } + + @Test + public void query1MatchesModel() { + queryMatchesModel("Query1Test", new Query1(CONFIG), new Query1Model(CONFIG)); + } + + @Test + public void query2MatchesModel() { + queryMatchesModel("Query2Test", new Query2(CONFIG), new Query2Model(CONFIG)); + } + + @Test + public void query3MatchesModel() { + queryMatchesModel("Query3Test", new Query3(CONFIG), new Query3Model(CONFIG)); + } + + @Test + public void query4MatchesModel() { + queryMatchesModel("Query4Test", new Query4(CONFIG), new Query4Model(CONFIG)); + } + + @Test + public void query5MatchesModel() { + queryMatchesModel("Query5Test", new Query5(CONFIG), new Query5Model(CONFIG)); + } + + @Test + public void query6MatchesModel() { + queryMatchesModel("Query6Test", new Query6(CONFIG), new Query6Model(CONFIG)); + } + + @Test + public void query7MatchesModel() { + queryMatchesModel("Query7Test", new Query7(CONFIG), new Query7Model(CONFIG)); + } + + @Test + public void query8MatchesModel() { + queryMatchesModel("Query8Test", new Query8(CONFIG), new Query8Model(CONFIG)); + } + + @Test + public void query9MatchesModel() { + queryMatchesModel("Query9Test", new Query9(CONFIG), new Query9Model(CONFIG)); + } +} diff --git a/integration/java/src/test/java/org/apache/beam/integration/nexmark/UnboundedEventSourceTest.java b/integration/java/src/test/java/org/apache/beam/integration/nexmark/UnboundedEventSourceTest.java new file mode 100644 index 0000000000000..5d72f775d6ab5 --- /dev/null +++ b/integration/java/src/test/java/org/apache/beam/integration/nexmark/UnboundedEventSourceTest.java @@ -0,0 +1,109 @@ +/* + * 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.integration.nexmark; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; + +import org.apache.beam.sdk.Pipeline; +import org.apache.beam.sdk.io.UnboundedSource.CheckpointMark; +import org.apache.beam.sdk.io.UnboundedSource.UnboundedReader; +import org.apache.beam.sdk.options.PipelineOptions; +import org.apache.beam.sdk.testing.TestPipeline; + +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +import java.io.IOException; +import java.util.HashSet; +import java.util.Random; +import java.util.Set; + +/** + * Test UnboundedEventSource. + */ +@RunWith(JUnit4.class) +public class UnboundedEventSourceTest { + private GeneratorConfig makeConfig(long n) { + return new GeneratorConfig( + NexmarkConfiguration.DEFAULT, System.currentTimeMillis(), 0, n, 0); + } + + /** + * Helper for tracking which ids we've seen (so we can detect dups) and + * confirming reading events match the model events. + */ + private static class EventIdChecker { + private Set seenPersonIds = new HashSet<>(); + private Set seenAuctionIds = new HashSet<>(); + + public void add(Event event) { + if (event.newAuction != null) { + assertTrue(seenAuctionIds.add(event.newAuction.id)); + } else if (event.newPerson != null) { + assertTrue(seenPersonIds.add(event.newPerson.id)); + } + } + + public void add(int n, UnboundedReader reader, Generator modelGenerator) + throws IOException { + for (int i = 0; i < n; i++) { + assertTrue(modelGenerator.hasNext()); + Event modelEvent = modelGenerator.next().getValue(); + assertTrue(reader.advance()); + Event actualEvent = reader.getCurrent(); + assertEquals(modelEvent.toString(), actualEvent.toString()); + add(actualEvent); + } + } + } + + /** + * Check aggressively checkpointing and resuming a reader gives us exactly the + * same event stream as reading directly. + */ + @Test + public void resumeFromCheckpoint() throws IOException { + Random random = new Random(297); + int n = 47293; + GeneratorConfig config = makeConfig(n); + Generator modelGenerator = new Generator(config); + + EventIdChecker checker = new EventIdChecker(); + Pipeline p = TestPipeline.create(); + PipelineOptions options = p.getOptions(); + UnboundedEventSource source = new UnboundedEventSource(config, 1, 0, false); + UnboundedReader reader = source.createReader(options, null); + + while (n > 0) { + int m = Math.min(459 + random.nextInt(455), n); + System.out.printf("reading %d...\n", m); + checker.add(m, reader, modelGenerator); + n -= m; + System.out.printf("splitting with %d remaining...\n", n); + CheckpointMark checkpointMark = reader.getCheckpointMark(); + assertTrue(checkpointMark instanceof Generator.Checkpoint); + reader = source.createReader(options, (Generator.Checkpoint) checkpointMark); + } + + assertFalse(reader.advance()); + } +} From 1bbfb4894aca7db69406ee7f6a8f51b35d269bd4 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Isma=C3=ABl=20Mej=C3=ADa?= Date: Wed, 30 Nov 2016 18:43:02 +0100 Subject: [PATCH 288/346] Update Nexmark to the current Beam snapshot 0.7.0 Refactor from InProcessRunner to DirectRunner Add Spark driver Add Apex runner Refine error logging per class and add log4j properties Move README to top level and add section on configuration Move project to the specific nexmark directory Fix existing issues to pass verify -Prelease Add running on the DirectRunner documentation --- .../nexmark/BEAM_ON_FLINK_ON_GCP.md | 0 .../beam/integration => }/nexmark/README.md | 108 +++++++++++- integration/java/{ => nexmark}/pom.xml | 127 +++++++------- .../nexmark/AbstractSimulator.java | 12 +- .../beam/integration/nexmark/Auction.java | 13 +- .../beam/integration/nexmark/AuctionBid.java | 9 +- .../integration/nexmark/AuctionCount.java | 11 +- .../integration/nexmark/AuctionPrice.java | 11 +- .../apache/beam/integration/nexmark/Bid.java | 13 +- .../integration/nexmark/BidsPerSession.java | 11 +- .../nexmark/BoundedEventSource.java | 20 +-- .../integration/nexmark/CategoryPrice.java | 13 +- .../apache/beam/integration/nexmark/Done.java | 11 +- .../beam/integration/nexmark/Event.java | 12 +- .../beam/integration/nexmark/Generator.java | 21 ++- .../integration/nexmark/GeneratorConfig.java | 5 +- .../integration/nexmark/IdNameReserve.java | 13 +- .../beam/integration/nexmark/KnownSize.java | 1 - .../beam/integration/nexmark/Monitor.java | 27 ++- .../integration/nexmark/NameCityStateId.java | 13 +- .../nexmark/NexmarkApexDriver.java | 48 ++++++ .../nexmark/NexmarkApexRunner.java | 66 ++++++++ .../nexmark/NexmarkConfiguration.java | 2 - .../nexmark/NexmarkDirectDriver.java} | 21 ++- .../nexmark/NexmarkDirectRunner.java} | 17 +- .../integration/nexmark/NexmarkDriver.java | 17 +- .../nexmark/NexmarkFlinkDriver.java | 7 +- .../nexmark/NexmarkFlinkRunner.java | 3 +- .../nexmark/NexmarkGoogleDriver.java | 16 +- .../nexmark/NexmarkGoogleRunner.java | 69 ++++---- .../beam/integration/nexmark/NexmarkPerf.java | 2 - .../integration/nexmark/NexmarkQuery.java | 80 +++++---- .../nexmark/NexmarkQueryModel.java | 15 +- .../integration/nexmark/NexmarkRunner.java | 141 ++++++++-------- .../nexmark/NexmarkSparkDriver.java | 46 +++++ .../nexmark/NexmarkSparkRunner.java | 66 ++++++++ .../integration/nexmark/NexmarkSuite.java | 6 +- .../integration/nexmark/NexmarkUtils.java | 157 ++++++++---------- .../beam/integration/nexmark/Options.java | 4 +- .../beam/integration/nexmark/Person.java | 13 +- .../integration/nexmark/PubsubHelper.java | 9 +- .../beam/integration/nexmark/Query0.java | 23 +-- .../beam/integration/nexmark/Query0Model.java | 5 +- .../beam/integration/nexmark/Query1.java | 8 +- .../beam/integration/nexmark/Query10.java | 130 +++++++-------- .../beam/integration/nexmark/Query11.java | 16 +- .../beam/integration/nexmark/Query12.java | 17 +- .../beam/integration/nexmark/Query1Model.java | 5 +- .../beam/integration/nexmark/Query2.java | 10 +- .../beam/integration/nexmark/Query2Model.java | 5 +- .../beam/integration/nexmark/Query3.java | 99 ++++++----- .../beam/integration/nexmark/Query3Model.java | 7 +- .../beam/integration/nexmark/Query4.java | 23 ++- .../beam/integration/nexmark/Query4Model.java | 14 +- .../beam/integration/nexmark/Query5.java | 30 ++-- .../beam/integration/nexmark/Query5Model.java | 10 +- .../beam/integration/nexmark/Query6.java | 31 ++-- .../beam/integration/nexmark/Query6Model.java | 12 +- .../beam/integration/nexmark/Query7.java | 10 +- .../beam/integration/nexmark/Query7Model.java | 12 +- .../beam/integration/nexmark/Query8.java | 27 ++- .../beam/integration/nexmark/Query8Model.java | 9 +- .../beam/integration/nexmark/Query9.java | 1 - .../beam/integration/nexmark/Query9Model.java | 5 +- .../beam/integration/nexmark/SellerPrice.java | 11 +- .../nexmark/UnboundedEventSource.java | 43 +++-- .../beam/integration/nexmark/WinningBids.java | 69 ++++---- .../nexmark/WinningBidsSimulator.java | 11 +- .../integration/nexmark/package-info.java | 21 +++ .../src/main/resources/log4j.properties | 42 +++++ .../nexmark/BoundedEventSourceTest.java | 1 - .../integration/nexmark/GeneratorTest.java | 9 +- .../beam/integration/nexmark/QueryTest.java | 13 +- .../nexmark/UnboundedEventSourceTest.java | 11 +- pom.xml | 1 + 75 files changed, 1124 insertions(+), 873 deletions(-) rename integration/java/{src/main/java/org/apache/beam/integration => }/nexmark/BEAM_ON_FLINK_ON_GCP.md (100%) rename integration/java/{src/main/java/org/apache/beam/integration => }/nexmark/README.md (61%) rename integration/java/{ => nexmark}/pom.xml (72%) rename integration/java/{ => nexmark}/src/main/java/org/apache/beam/integration/nexmark/AbstractSimulator.java (99%) rename integration/java/{ => nexmark}/src/main/java/org/apache/beam/integration/nexmark/Auction.java (99%) rename integration/java/{ => nexmark}/src/main/java/org/apache/beam/integration/nexmark/AuctionBid.java (99%) rename integration/java/{ => nexmark}/src/main/java/org/apache/beam/integration/nexmark/AuctionCount.java (99%) rename integration/java/{ => nexmark}/src/main/java/org/apache/beam/integration/nexmark/AuctionPrice.java (99%) rename integration/java/{ => nexmark}/src/main/java/org/apache/beam/integration/nexmark/Bid.java (99%) rename integration/java/{ => nexmark}/src/main/java/org/apache/beam/integration/nexmark/BidsPerSession.java (99%) rename integration/java/{ => nexmark}/src/main/java/org/apache/beam/integration/nexmark/BoundedEventSource.java (98%) rename integration/java/{ => nexmark}/src/main/java/org/apache/beam/integration/nexmark/CategoryPrice.java (99%) rename integration/java/{ => nexmark}/src/main/java/org/apache/beam/integration/nexmark/Done.java (99%) rename integration/java/{ => nexmark}/src/main/java/org/apache/beam/integration/nexmark/Event.java (99%) rename integration/java/{ => nexmark}/src/main/java/org/apache/beam/integration/nexmark/Generator.java (99%) rename integration/java/{ => nexmark}/src/main/java/org/apache/beam/integration/nexmark/GeneratorConfig.java (99%) rename integration/java/{ => nexmark}/src/main/java/org/apache/beam/integration/nexmark/IdNameReserve.java (99%) rename integration/java/{ => nexmark}/src/main/java/org/apache/beam/integration/nexmark/KnownSize.java (99%) rename integration/java/{ => nexmark}/src/main/java/org/apache/beam/integration/nexmark/Monitor.java (82%) rename integration/java/{ => nexmark}/src/main/java/org/apache/beam/integration/nexmark/NameCityStateId.java (99%) create mode 100644 integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkApexDriver.java create mode 100644 integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkApexRunner.java rename integration/java/{ => nexmark}/src/main/java/org/apache/beam/integration/nexmark/NexmarkConfiguration.java (99%) rename integration/java/{src/main/java/org/apache/beam/integration/nexmark/NexmarkInProcessDriver.java => nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkDirectDriver.java} (62%) rename integration/java/{src/main/java/org/apache/beam/integration/nexmark/NexmarkInProcessRunner.java => nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkDirectRunner.java} (77%) rename integration/java/{ => nexmark}/src/main/java/org/apache/beam/integration/nexmark/NexmarkDriver.java (98%) rename integration/java/{ => nexmark}/src/main/java/org/apache/beam/integration/nexmark/NexmarkFlinkDriver.java (91%) rename integration/java/{ => nexmark}/src/main/java/org/apache/beam/integration/nexmark/NexmarkFlinkRunner.java (96%) rename integration/java/{ => nexmark}/src/main/java/org/apache/beam/integration/nexmark/NexmarkGoogleDriver.java (91%) rename integration/java/{ => nexmark}/src/main/java/org/apache/beam/integration/nexmark/NexmarkGoogleRunner.java (93%) rename integration/java/{ => nexmark}/src/main/java/org/apache/beam/integration/nexmark/NexmarkPerf.java (99%) rename integration/java/{ => nexmark}/src/main/java/org/apache/beam/integration/nexmark/NexmarkQuery.java (81%) rename integration/java/{ => nexmark}/src/main/java/org/apache/beam/integration/nexmark/NexmarkQueryModel.java (99%) rename integration/java/{ => nexmark}/src/main/java/org/apache/beam/integration/nexmark/NexmarkRunner.java (88%) create mode 100644 integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkSparkDriver.java create mode 100644 integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkSparkRunner.java rename integration/java/{ => nexmark}/src/main/java/org/apache/beam/integration/nexmark/NexmarkSuite.java (95%) rename integration/java/{ => nexmark}/src/main/java/org/apache/beam/integration/nexmark/NexmarkUtils.java (85%) rename integration/java/{ => nexmark}/src/main/java/org/apache/beam/integration/nexmark/Options.java (100%) rename integration/java/{ => nexmark}/src/main/java/org/apache/beam/integration/nexmark/Person.java (99%) rename integration/java/{ => nexmark}/src/main/java/org/apache/beam/integration/nexmark/PubsubHelper.java (99%) rename integration/java/{ => nexmark}/src/main/java/org/apache/beam/integration/nexmark/Query0.java (90%) rename integration/java/{ => nexmark}/src/main/java/org/apache/beam/integration/nexmark/Query0Model.java (99%) rename integration/java/{ => nexmark}/src/main/java/org/apache/beam/integration/nexmark/Query1.java (94%) rename integration/java/{ => nexmark}/src/main/java/org/apache/beam/integration/nexmark/Query10.java (82%) rename integration/java/{ => nexmark}/src/main/java/org/apache/beam/integration/nexmark/Query11.java (90%) rename integration/java/{ => nexmark}/src/main/java/org/apache/beam/integration/nexmark/Query12.java (90%) rename integration/java/{ => nexmark}/src/main/java/org/apache/beam/integration/nexmark/Query1Model.java (99%) rename integration/java/{ => nexmark}/src/main/java/org/apache/beam/integration/nexmark/Query2.java (92%) rename integration/java/{ => nexmark}/src/main/java/org/apache/beam/integration/nexmark/Query2Model.java (99%) rename integration/java/{ => nexmark}/src/main/java/org/apache/beam/integration/nexmark/Query3.java (76%) rename integration/java/{ => nexmark}/src/main/java/org/apache/beam/integration/nexmark/Query3Model.java (99%) rename integration/java/{ => nexmark}/src/main/java/org/apache/beam/integration/nexmark/Query4.java (91%) rename integration/java/{ => nexmark}/src/main/java/org/apache/beam/integration/nexmark/Query4Model.java (99%) rename integration/java/{ => nexmark}/src/main/java/org/apache/beam/integration/nexmark/Query5.java (92%) rename integration/java/{ => nexmark}/src/main/java/org/apache/beam/integration/nexmark/Query5Model.java (99%) rename integration/java/{ => nexmark}/src/main/java/org/apache/beam/integration/nexmark/Query6.java (94%) rename integration/java/{ => nexmark}/src/main/java/org/apache/beam/integration/nexmark/Query6Model.java (99%) rename integration/java/{ => nexmark}/src/main/java/org/apache/beam/integration/nexmark/Query7.java (95%) rename integration/java/{ => nexmark}/src/main/java/org/apache/beam/integration/nexmark/Query7Model.java (99%) rename integration/java/{ => nexmark}/src/main/java/org/apache/beam/integration/nexmark/Query8.java (82%) rename integration/java/{ => nexmark}/src/main/java/org/apache/beam/integration/nexmark/Query8Model.java (99%) rename integration/java/{ => nexmark}/src/main/java/org/apache/beam/integration/nexmark/Query9.java (99%) rename integration/java/{ => nexmark}/src/main/java/org/apache/beam/integration/nexmark/Query9Model.java (99%) rename integration/java/{ => nexmark}/src/main/java/org/apache/beam/integration/nexmark/SellerPrice.java (99%) rename integration/java/{ => nexmark}/src/main/java/org/apache/beam/integration/nexmark/UnboundedEventSource.java (93%) rename integration/java/{ => nexmark}/src/main/java/org/apache/beam/integration/nexmark/WinningBids.java (92%) rename integration/java/{ => nexmark}/src/main/java/org/apache/beam/integration/nexmark/WinningBidsSimulator.java (99%) create mode 100644 integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/package-info.java create mode 100644 integration/java/nexmark/src/main/resources/log4j.properties rename integration/java/{ => nexmark}/src/test/java/org/apache/beam/integration/nexmark/BoundedEventSourceTest.java (99%) rename integration/java/{ => nexmark}/src/test/java/org/apache/beam/integration/nexmark/GeneratorTest.java (99%) rename integration/java/{ => nexmark}/src/test/java/org/apache/beam/integration/nexmark/QueryTest.java (90%) rename integration/java/{ => nexmark}/src/test/java/org/apache/beam/integration/nexmark/UnboundedEventSourceTest.java (99%) diff --git a/integration/java/src/main/java/org/apache/beam/integration/nexmark/BEAM_ON_FLINK_ON_GCP.md b/integration/java/nexmark/BEAM_ON_FLINK_ON_GCP.md similarity index 100% rename from integration/java/src/main/java/org/apache/beam/integration/nexmark/BEAM_ON_FLINK_ON_GCP.md rename to integration/java/nexmark/BEAM_ON_FLINK_ON_GCP.md diff --git a/integration/java/src/main/java/org/apache/beam/integration/nexmark/README.md b/integration/java/nexmark/README.md similarity index 61% rename from integration/java/src/main/java/org/apache/beam/integration/nexmark/README.md rename to integration/java/nexmark/README.md index 5e3332722496e..4c08c28137008 100644 --- a/integration/java/src/main/java/org/apache/beam/integration/nexmark/README.md +++ b/integration/java/nexmark/README.md @@ -23,10 +23,14 @@ This is a suite of pipelines inspired by the 'continuous data stream' queries in [http://datalab.cs.pdx.edu/niagaraST/NEXMark/] (http://datalab.cs.pdx.edu/niagaraST/NEXMark/). -The queries are over a simple online auction system with tables of -**Person**, **Auction** and **Bid** records. +These are multiple queries over a three entities model representing on online auction system: -The queries are: + - **Person** represents a person submitting an item for auction and/or making a bid + on an auction. + - **Auction** represents an item under auction. + - **Bid** represents a bid for an item under auction. + +The queries exercise many aspects of dataflow model on Beam: * **Query1**: What are the bid values in Euro's? Illustrates a simple map. @@ -70,13 +74,14 @@ We have augmented the original queries with five more: The queries can be executed using a 'Driver' for a given backend. Currently the supported drivers are: -* **NexmarkInProcessDriver** for running locally on a single machine. +* **NexmarkDirectDriver** for running locally on a single machine. * **NexmarkGoogleDriver** for running on the Google Cloud Dataflow service. Requires a Google Cloud account. * **NexmarkFlinkDriver** for running on a Flink cluster. Requires the cluster to be established and the Nexmark jar to be distributed to each worker. - +* **NexmarkSparkDriver** for running on a Spark cluster. + Other drivers are straightforward. Test data is deterministically synthesized on demand. The test @@ -96,6 +101,99 @@ pipeline performance, and comparing that performance against a known baseline. However that machinery has only been implemented against the Google Cloud Dataflow driver. +# Configuration + +Common configuration parameters: + +Available Suites: + +- DEFAULT: Test default configuration with query 0. +- SMOKE: Run the 12 default configurations. +- STRESS: Like smoke but for 1m events. +- FULL_THROTTLE: Like SMOKE but 100m events. + + --suite=SMOKE + +Decide if batch or streaming: + + --streaming=true + +Number of events generators + + --numEventGenerators=4 + +## Flink specific configuration + +--suite=SMOKE --manageResources=false --monitorJobs=false \ +--flinkMaster=local + +## Direct specific configuration + +--suite=SMOKE --manageResources=false --monitorJobs=false \ +--enforceEncodability=false --enforceImmutability=false + +## Spark specific configuration + +--suite=SMOKE +--manageResources=false --monitorJobs=false --sparkMaster=local +-Dspark.ui.enabled=false +-DSPARK_LOCAL_IP=localhost +-Dsun.io.serialization.extendedDebugInfo=true + +# Current Status + +Open issues are currently opened on [github](https://github.com/iemejia/beam/issues): + +## Batch Mode / Synthetic / Local + +| Query | Direct | Spark | Flink | Apex | +| ----: | ------ | ------ | ------ | ------ | +| 0 | Ok | #1 | Ok | | +| 1 | Ok | #1 | Ok | | +| 2 | Ok | NEX-01 | Ok | | +| 3 | NEX-07 | NEX-07 | NEX-07 | | +| 4 | Ok | Ok | NEX-02 | | +| 5 | Ok | NEX-03 | Ok | | +| 6 | Ok | OK | NEX-02 | | +| 7 | Ok | NEX-01 | Ok | | +| 8 | Ok | NEX-01 | Ok | | +| 9 | Ok | OK | NEX-02 | | +| 10 | NEX-05 | NEX-04 | Ok | | +| 11 | Ok | NEX-01 | Ok | | +| 12 | Ok | NEX-01 | Ok | | + +## Streaming Mode / Synthetic / Local + +| Query | Direct | Spark | Flink | Apex | +| ----: | ------ | ------ | ------ | ------ | +| 0 | Ok | | | | +| 1 | Ok | | | | +| 2 | Ok | | | | +| 3 | NEX-07 | | | | +| 4 | Ok | | | | +| 5 | Ok | | | | +| 6 | Ok | | | | +| 7 | Ok | | | | +| 8 | Ok | | | | +| 9 | Ok | | | | +| 10 | NEX-05 | | | | +| 11 | Ok | | | | +| 12 | Ok | | | | + +# Running Nexmark + +## Running on the DirectRunner (local) + +Batch Mode + +-Dexec.classpathScope="test" + + mvn exec:java -Dexec.mainClass=org.apache.beam.integration.nexmark.NexmarkDirectDriver -Dexec.args="--suite=SMOKE --streaming=false --numEventGenerators=4 --manageResources=false --monitorJobs=false --enforceEncodability=false --enforceImmutability=false" + +Streaming Mode + + mvn exec:java -Dexec.mainClass=org.apache.beam.integration.nexmark.NexmarkDirectDriver -Dexec.args="--suite=SMOKE --streaming=true --numEventGenerators=4 --manageResources=false --monitorJobs=false --enforceEncodability=false --enforceImmutability=false" + ## Running on Google Cloud Dataflow An example invocation for **Query10** on the Google Cloud Dataflow diff --git a/integration/java/pom.xml b/integration/java/nexmark/pom.xml similarity index 72% rename from integration/java/pom.xml rename to integration/java/nexmark/pom.xml index b160b56123b06..d64eb1bb86530 100644 --- a/integration/java/pom.xml +++ b/integration/java/nexmark/pom.xml @@ -22,31 +22,25 @@ org.apache.beam - parent - 0.2.0-incubating-SNAPSHOT - ../../pom.xml + beam-parent + 0.7.0-SNAPSHOT + ../../../pom.xml - java-integration-all - + beam-integration-java Apache Beam :: Integration Tests :: Java All jar + UTF-8 + UTF-8 + 1.2.0 + 1.6.3 true - - - - kr.motd.maven - os-maven-plugin - 1.4.0.Final - - - org.apache.maven.plugins @@ -74,14 +68,12 @@ maven-checkstyle-plugin - org.apache.maven.plugins maven-source-plugin - org.apache.maven.plugins maven-shade-plugin @@ -122,7 +114,6 @@ org.apache.avro avro-maven-plugin - ${avro.version} schemas @@ -150,35 +141,60 @@ org.apache.beam - java-sdk-all + beam-sdks-java-core - org.apache.beam.runners - google-cloud-dataflow-java - ${project.version} + org.apache.beam + beam-runners-google-cloud-dataflow-java - org.apache.beam.runners - direct - ${project.version} + org.apache.beam + beam-runners-direct-java + + + + + org.apache.beam + beam-runners-flink_2.10 - + + + + + + + + + + org.apache.beam + beam-runners-spark + + + + + + + + + + + + + - org.apache.beam.runners - flink_2.10 - ${project.version} + org.apache.beam + beam-runners-apex + - org.apache.flink - flink-shaded-hadoop2 - 1.0.3 - provided + org.apache.beam + beam-sdks-java-io-google-cloud-platform @@ -196,93 +212,74 @@ com.fasterxml.jackson.core jackson-core - ${jackson.version} com.google.apis google-api-services-bigquery - ${bigquery.version} com.google.cloud.bigdataoss gcsio - ${google-cloud-bigdataoss.version} com.fasterxml.jackson.core jackson-annotations - ${jackson.version} com.fasterxml.jackson.core jackson-databind - ${jackson.version} org.apache.avro avro - ${avro.version} com.google.guava guava - ${guava.version} org.hamcrest hamcrest-all - ${hamcrest.version} + test org.slf4j slf4j-api - ${slf4j.version} compile - - org.slf4j - slf4j-jdk14 - ${slf4j.version} - - runtime - + + + + + + com.google.code.findbugs jsr305 - ${jsr305.version} - runtime - - - - log4j - log4j - 1.2.17 - runtime junit junit - ${junit.version} compile - - io.netty - netty-tcnative-boringssl-static - 1.1.33.Fork13 - ${os.detected.classifier} - runtime - + + + + + + + diff --git a/integration/java/src/main/java/org/apache/beam/integration/nexmark/AbstractSimulator.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/AbstractSimulator.java similarity index 99% rename from integration/java/src/main/java/org/apache/beam/integration/nexmark/AbstractSimulator.java rename to integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/AbstractSimulator.java index 6473c35c39964..c08cdd349adde 100644 --- a/integration/java/src/main/java/org/apache/beam/integration/nexmark/AbstractSimulator.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/AbstractSimulator.java @@ -18,18 +18,16 @@ package org.apache.beam.integration.nexmark; -import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.values.TimestampedValue; - -import org.joda.time.Duration; -import org.joda.time.Instant; - import java.util.ArrayList; import java.util.Iterator; import java.util.List; - import javax.annotation.Nullable; +import org.apache.beam.sdk.transforms.windowing.BoundedWindow; +import org.apache.beam.sdk.values.TimestampedValue; +import org.joda.time.Duration; +import org.joda.time.Instant; + /** * Abstract base class for simulator of a query. * diff --git a/integration/java/src/main/java/org/apache/beam/integration/nexmark/Auction.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Auction.java similarity index 99% rename from integration/java/src/main/java/org/apache/beam/integration/nexmark/Auction.java rename to integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Auction.java index 94f2647e28d7d..16c28aac07440 100644 --- a/integration/java/src/main/java/org/apache/beam/integration/nexmark/Auction.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Auction.java @@ -15,15 +15,8 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.beam.integration.nexmark; -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.StringUtf8Coder; -import org.apache.beam.sdk.coders.VarLongCoder; - import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.core.JsonProcessingException; @@ -32,6 +25,12 @@ import java.io.OutputStream; import java.io.Serializable; +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.StringUtf8Coder; +import org.apache.beam.sdk.coders.VarLongCoder; + /** * An auction submitted by a person. */ diff --git a/integration/java/src/main/java/org/apache/beam/integration/nexmark/AuctionBid.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/AuctionBid.java similarity index 99% rename from integration/java/src/main/java/org/apache/beam/integration/nexmark/AuctionBid.java rename to integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/AuctionBid.java index 8c3697a3780e3..cd52b028b425c 100644 --- a/integration/java/src/main/java/org/apache/beam/integration/nexmark/AuctionBid.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/AuctionBid.java @@ -15,13 +15,8 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.beam.integration.nexmark; -import org.apache.beam.sdk.coders.AtomicCoder; -import org.apache.beam.sdk.coders.Coder; -import org.apache.beam.sdk.coders.CoderException; - import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.core.JsonProcessingException; @@ -30,6 +25,10 @@ import java.io.OutputStream; import java.io.Serializable; +import org.apache.beam.sdk.coders.AtomicCoder; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.coders.CoderException; + /** * Result of {@link WinningBids} transform. */ diff --git a/integration/java/src/main/java/org/apache/beam/integration/nexmark/AuctionCount.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/AuctionCount.java similarity index 99% rename from integration/java/src/main/java/org/apache/beam/integration/nexmark/AuctionCount.java rename to integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/AuctionCount.java index a0fbebc36e2de..ac1f080d1e818 100644 --- a/integration/java/src/main/java/org/apache/beam/integration/nexmark/AuctionCount.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/AuctionCount.java @@ -15,14 +15,8 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.beam.integration.nexmark; -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.VarLongCoder; - import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.core.JsonProcessingException; @@ -31,6 +25,11 @@ import java.io.OutputStream; import java.io.Serializable; +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.VarLongCoder; + /** * Result of {@link Query5}. */ diff --git a/integration/java/src/main/java/org/apache/beam/integration/nexmark/AuctionPrice.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/AuctionPrice.java similarity index 99% rename from integration/java/src/main/java/org/apache/beam/integration/nexmark/AuctionPrice.java rename to integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/AuctionPrice.java index 4f25a9b64de8b..9bdf11cd7f6a7 100644 --- a/integration/java/src/main/java/org/apache/beam/integration/nexmark/AuctionPrice.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/AuctionPrice.java @@ -15,14 +15,8 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.beam.integration.nexmark; -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.VarLongCoder; - import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.core.JsonProcessingException; @@ -31,6 +25,11 @@ import java.io.OutputStream; import java.io.Serializable; +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.VarLongCoder; + /** * Result of {@link Query2}. */ diff --git a/integration/java/src/main/java/org/apache/beam/integration/nexmark/Bid.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Bid.java similarity index 99% rename from integration/java/src/main/java/org/apache/beam/integration/nexmark/Bid.java rename to integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Bid.java index ce2184b221565..04fcfdd043a12 100644 --- a/integration/java/src/main/java/org/apache/beam/integration/nexmark/Bid.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Bid.java @@ -15,15 +15,8 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.beam.integration.nexmark; -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.StringUtf8Coder; -import org.apache.beam.sdk.coders.VarLongCoder; - import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.core.JsonProcessingException; @@ -33,6 +26,12 @@ import java.io.Serializable; import java.util.Comparator; +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.StringUtf8Coder; +import org.apache.beam.sdk.coders.VarLongCoder; + /** * A bid for an item on auction. */ diff --git a/integration/java/src/main/java/org/apache/beam/integration/nexmark/BidsPerSession.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/BidsPerSession.java similarity index 99% rename from integration/java/src/main/java/org/apache/beam/integration/nexmark/BidsPerSession.java rename to integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/BidsPerSession.java index cfdd170431bb7..c6b0fe3a8b3d2 100644 --- a/integration/java/src/main/java/org/apache/beam/integration/nexmark/BidsPerSession.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/BidsPerSession.java @@ -15,14 +15,8 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.beam.integration.nexmark; -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.VarLongCoder; - import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.core.JsonProcessingException; @@ -31,6 +25,11 @@ import java.io.OutputStream; import java.io.Serializable; +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.VarLongCoder; + /** * Result of query 11. */ diff --git a/integration/java/src/main/java/org/apache/beam/integration/nexmark/BoundedEventSource.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/BoundedEventSource.java similarity index 98% rename from integration/java/src/main/java/org/apache/beam/integration/nexmark/BoundedEventSource.java rename to integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/BoundedEventSource.java index f6cc16aef16f7..7dc1bcc855b52 100644 --- a/integration/java/src/main/java/org/apache/beam/integration/nexmark/BoundedEventSource.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/BoundedEventSource.java @@ -15,23 +15,20 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.beam.integration.nexmark; -import org.apache.beam.sdk.coders.Coder; -import org.apache.beam.sdk.io.BoundedSource; -import org.apache.beam.sdk.options.PipelineOptions; -import org.apache.beam.sdk.values.TimestampedValue; - -import org.joda.time.Instant; - import java.io.IOException; import java.util.ArrayList; import java.util.List; import java.util.NoSuchElementException; - import javax.annotation.Nullable; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.io.BoundedSource; +import org.apache.beam.sdk.options.PipelineOptions; +import org.apache.beam.sdk.values.TimestampedValue; +import org.joda.time.Instant; + /** * A custom, bounded source of event records. */ @@ -174,11 +171,6 @@ public long getEstimatedSizeBytes(PipelineOptions options) { return config.getEstimatedSizeBytes(); } - @Override - public boolean producesSortedKeys(PipelineOptions options) { - return false; - } - @Override public EventReader createReader(PipelineOptions options) { NexmarkUtils.info("creating initial bounded reader for %s", config); diff --git a/integration/java/src/main/java/org/apache/beam/integration/nexmark/CategoryPrice.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/CategoryPrice.java similarity index 99% rename from integration/java/src/main/java/org/apache/beam/integration/nexmark/CategoryPrice.java rename to integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/CategoryPrice.java index ab5d92d264d4c..c83fb178340d2 100644 --- a/integration/java/src/main/java/org/apache/beam/integration/nexmark/CategoryPrice.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/CategoryPrice.java @@ -15,15 +15,8 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.beam.integration.nexmark; -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.VarIntCoder; -import org.apache.beam.sdk.coders.VarLongCoder; - import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.core.JsonProcessingException; @@ -32,6 +25,12 @@ import java.io.OutputStream; import java.io.Serializable; +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.VarIntCoder; +import org.apache.beam.sdk.coders.VarLongCoder; + /** * Result of {@link Query4}. */ diff --git a/integration/java/src/main/java/org/apache/beam/integration/nexmark/Done.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Done.java similarity index 99% rename from integration/java/src/main/java/org/apache/beam/integration/nexmark/Done.java rename to integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Done.java index 659da441ed115..3a045f9c9cb59 100644 --- a/integration/java/src/main/java/org/apache/beam/integration/nexmark/Done.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Done.java @@ -15,14 +15,8 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.beam.integration.nexmark; -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.StringUtf8Coder; - import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.core.JsonProcessingException; @@ -31,6 +25,11 @@ import java.io.OutputStream; import java.io.Serializable; +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.StringUtf8Coder; + /** * Result of query 10. */ diff --git a/integration/java/src/main/java/org/apache/beam/integration/nexmark/Event.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Event.java similarity index 99% rename from integration/java/src/main/java/org/apache/beam/integration/nexmark/Event.java rename to integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Event.java index a382b8ebed436..769cedda9df22 100644 --- a/integration/java/src/main/java/org/apache/beam/integration/nexmark/Event.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Event.java @@ -15,21 +15,19 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.beam.integration.nexmark; -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.VarIntCoder; - import java.io.IOException; import java.io.InputStream; import java.io.OutputStream; import java.io.Serializable; - import javax.annotation.Nullable; +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.VarIntCoder; + /** * An event in the auction system, either a (new) {@link Person}, a (new) {@link Auction}, * or a {@link Bid}. diff --git a/integration/java/src/main/java/org/apache/beam/integration/nexmark/Generator.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Generator.java similarity index 99% rename from integration/java/src/main/java/org/apache/beam/integration/nexmark/Generator.java rename to integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Generator.java index 98f4f0042b99e..7adb1b2af9c8f 100644 --- a/integration/java/src/main/java/org/apache/beam/integration/nexmark/Generator.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Generator.java @@ -15,18 +15,9 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.beam.integration.nexmark; -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.VarLongCoder; -import org.apache.beam.sdk.io.UnboundedSource; -import org.apache.beam.sdk.values.TimestampedValue; -import com.google.common.base.Preconditions; - -import org.joda.time.Instant; +import static com.google.common.base.Preconditions.checkNotNull; import java.io.IOException; import java.io.InputStream; @@ -37,6 +28,14 @@ import java.util.List; import java.util.Random; +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.VarLongCoder; +import org.apache.beam.sdk.io.UnboundedSource; +import org.apache.beam.sdk.values.TimestampedValue; +import org.joda.time.Instant; + /** * A generator for synthetic events. We try to make the data vaguely reasonable. We also ensure * most primary key/foreign key relations are correct. Eg: a {@link Bid} event will usually have @@ -202,7 +201,7 @@ public int compareTo(NextEvent other) { private long wallclockBaseTime; private Generator(GeneratorConfig config, long numEvents, long wallclockBaseTime) { - Preconditions.checkNotNull(config); + checkNotNull(config); this.config = config; this.numEvents = numEvents; this.wallclockBaseTime = wallclockBaseTime; diff --git a/integration/java/src/main/java/org/apache/beam/integration/nexmark/GeneratorConfig.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/GeneratorConfig.java similarity index 99% rename from integration/java/src/main/java/org/apache/beam/integration/nexmark/GeneratorConfig.java rename to integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/GeneratorConfig.java index 59aaf492be699..dceff4f4778e1 100644 --- a/integration/java/src/main/java/org/apache/beam/integration/nexmark/GeneratorConfig.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/GeneratorConfig.java @@ -15,15 +15,14 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.beam.integration.nexmark; -import org.apache.beam.sdk.values.KV; - import java.io.Serializable; import java.util.ArrayList; import java.util.List; +import org.apache.beam.sdk.values.KV; + /** * Parameters controlling how {@link Generator} synthesizes {@link Event} elements. */ diff --git a/integration/java/src/main/java/org/apache/beam/integration/nexmark/IdNameReserve.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/IdNameReserve.java similarity index 99% rename from integration/java/src/main/java/org/apache/beam/integration/nexmark/IdNameReserve.java rename to integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/IdNameReserve.java index c72b76aa63f73..21fa3f402a44a 100644 --- a/integration/java/src/main/java/org/apache/beam/integration/nexmark/IdNameReserve.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/IdNameReserve.java @@ -15,15 +15,8 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.beam.integration.nexmark; -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.StringUtf8Coder; -import org.apache.beam.sdk.coders.VarLongCoder; - import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.core.JsonProcessingException; @@ -32,6 +25,12 @@ import java.io.OutputStream; import java.io.Serializable; +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.StringUtf8Coder; +import org.apache.beam.sdk.coders.VarLongCoder; + /** * Result type of {@link Query8}. */ diff --git a/integration/java/src/main/java/org/apache/beam/integration/nexmark/KnownSize.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/KnownSize.java similarity index 99% rename from integration/java/src/main/java/org/apache/beam/integration/nexmark/KnownSize.java rename to integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/KnownSize.java index 394b6db43eeda..2093c487a7295 100644 --- a/integration/java/src/main/java/org/apache/beam/integration/nexmark/KnownSize.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/KnownSize.java @@ -15,7 +15,6 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.beam.integration.nexmark; /** diff --git a/integration/java/src/main/java/org/apache/beam/integration/nexmark/Monitor.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Monitor.java similarity index 82% rename from integration/java/src/main/java/org/apache/beam/integration/nexmark/Monitor.java rename to integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Monitor.java index 6874578937086..02660bfe33121 100644 --- a/integration/java/src/main/java/org/apache/beam/integration/nexmark/Monitor.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Monitor.java @@ -15,20 +15,19 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.beam.integration.nexmark; +import java.io.Serializable; + import org.apache.beam.sdk.transforms.Aggregator; import org.apache.beam.sdk.transforms.DoFn; -import org.apache.beam.sdk.transforms.Max.MaxLongFn; -import org.apache.beam.sdk.transforms.Min.MinLongFn; +import org.apache.beam.sdk.transforms.Max; +import org.apache.beam.sdk.transforms.Min; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.ParDo; -import org.apache.beam.sdk.transforms.Sum.SumLongFn; +import org.apache.beam.sdk.transforms.Sum; import org.apache.beam.sdk.values.PCollection; -import java.io.Serializable; - /** * A monitor of elements with support for later retrieving their aggregators. * @@ -37,19 +36,19 @@ public class Monitor implements Serializable { private class MonitorDoFn extends DoFn { public final Aggregator elementCounter = - createAggregator(counterNamePrefix + "_elements", new SumLongFn()); + createAggregator(counterNamePrefix + "_elements", Sum.ofLongs()); public final Aggregator bytesCounter = - createAggregator(counterNamePrefix + "_bytes", new SumLongFn()); + createAggregator(counterNamePrefix + "_bytes", Sum.ofLongs()); public final Aggregator startTime = - createAggregator(counterNamePrefix + "_startTime", new MinLongFn()); + createAggregator(counterNamePrefix + "_startTime", Min.ofLongs()); public final Aggregator endTime = - createAggregator(counterNamePrefix + "_endTime", new MaxLongFn()); + createAggregator(counterNamePrefix + "_endTime", Max.ofLongs()); public final Aggregator startTimestamp = - createAggregator("startTimestamp", new MinLongFn()); + createAggregator("startTimestamp", Min.ofLongs()); public final Aggregator endTimestamp = - createAggregator("endTimestamp", new MaxLongFn()); + createAggregator("endTimestamp", Max.ofLongs()); - @Override + @ProcessElement public void processElement(ProcessContext c) { elementCounter.addValue(1L); bytesCounter.addValue(c.element().sizeInBytes()); @@ -69,7 +68,7 @@ public void processElement(ProcessContext c) { public Monitor(String name, String counterNamePrefix) { this.counterNamePrefix = counterNamePrefix; doFn = new MonitorDoFn(); - transform = ParDo.named(name + ".Monitor").of(doFn); + transform = ParDo.of(doFn); } public PTransform, PCollection> getTransform() { diff --git a/integration/java/src/main/java/org/apache/beam/integration/nexmark/NameCityStateId.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NameCityStateId.java similarity index 99% rename from integration/java/src/main/java/org/apache/beam/integration/nexmark/NameCityStateId.java rename to integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NameCityStateId.java index 2753d2ecf9356..fe4687bf3ade5 100644 --- a/integration/java/src/main/java/org/apache/beam/integration/nexmark/NameCityStateId.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NameCityStateId.java @@ -15,15 +15,8 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.beam.integration.nexmark; -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.StringUtf8Coder; -import org.apache.beam.sdk.coders.VarLongCoder; - import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.core.JsonProcessingException; @@ -32,6 +25,12 @@ import java.io.OutputStream; import java.io.Serializable; +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.StringUtf8Coder; +import org.apache.beam.sdk.coders.VarLongCoder; + /** * Result of {@link Query3}. */ diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkApexDriver.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkApexDriver.java new file mode 100644 index 0000000000000..4c2721e220027 --- /dev/null +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkApexDriver.java @@ -0,0 +1,48 @@ +/* + * 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.integration.nexmark; + +import org.apache.beam.runners.apex.ApexPipelineOptions; +import org.apache.beam.runners.apex.ApexRunner; +import org.apache.beam.sdk.options.PipelineOptionsFactory; + +/** + * Run NexMark queries using the Apex runner. + */ +public class NexmarkApexDriver extends NexmarkDriver { + /** + * Command line flags. + */ + public interface NexmarkApexOptions extends Options, ApexPipelineOptions { + } + + /** + * Entry point. + */ + public static void main(String[] args) { + // Gather command line args, baseline, configurations, etc. + NexmarkApexOptions options = PipelineOptionsFactory.fromArgs(args) + .withValidation() + .as(NexmarkApexOptions.class); + options.setRunner(ApexRunner.class); + NexmarkApexRunner runner = new NexmarkApexRunner(options); + new NexmarkApexDriver().runAll(options, runner); + } +} + + diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkApexRunner.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkApexRunner.java new file mode 100644 index 0000000000000..ea46082fb20b9 --- /dev/null +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkApexRunner.java @@ -0,0 +1,66 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.integration.nexmark; + +import javax.annotation.Nullable; + +/** + * Run a query using the Apex runner. + */ +public class NexmarkApexRunner extends NexmarkRunner { + @Override + protected boolean isStreaming() { + return options.isStreaming(); + } + + @Override + protected int coresPerWorker() { + return 4; + } + + @Override + protected int maxNumWorkers() { + return 5; + } + + @Override + protected boolean canMonitor() { + return false; + } + + @Override + protected void invokeBuilderForPublishOnlyPipeline( + PipelineBuilder builder) { + builder.build(options); + } + + @Override + protected void waitForPublisherPreload() { + throw new UnsupportedOperationException(); + } + + @Override + @Nullable + protected NexmarkPerf monitor(NexmarkQuery query) { + return null; + } + + public NexmarkApexRunner(NexmarkApexDriver.NexmarkApexOptions options) { + super(options); + } +} diff --git a/integration/java/src/main/java/org/apache/beam/integration/nexmark/NexmarkConfiguration.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkConfiguration.java similarity index 99% rename from integration/java/src/main/java/org/apache/beam/integration/nexmark/NexmarkConfiguration.java rename to integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkConfiguration.java index 2292ba527ac7f..09436640a0f4f 100644 --- a/integration/java/src/main/java/org/apache/beam/integration/nexmark/NexmarkConfiguration.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkConfiguration.java @@ -15,7 +15,6 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.beam.integration.nexmark; import com.fasterxml.jackson.annotation.JsonProperty; @@ -25,7 +24,6 @@ import java.io.Serializable; import java.util.Objects; - /** * Configuration controlling how a query is run. May be supplied by command line or * programmatically. We only capture properties which may influence the resulting diff --git a/integration/java/src/main/java/org/apache/beam/integration/nexmark/NexmarkInProcessDriver.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkDirectDriver.java similarity index 62% rename from integration/java/src/main/java/org/apache/beam/integration/nexmark/NexmarkInProcessDriver.java rename to integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkDirectDriver.java index fe279c031f996..24fcc0150d151 100644 --- a/integration/java/src/main/java/org/apache/beam/integration/nexmark/NexmarkInProcessDriver.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkDirectDriver.java @@ -15,34 +15,33 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.beam.integration.nexmark; -import org.apache.beam.runners.dataflow.options.DataflowPipelineOptions; -import org.apache.beam.runners.direct.InProcessPipelineRunner; +import org.apache.beam.runners.direct.DirectOptions; +import org.apache.beam.runners.direct.DirectRunner; import org.apache.beam.sdk.options.PipelineOptionsFactory; /** - * An implementation of the 'NEXMark queries' using the in-process runner. + * An implementation of the 'NEXMark queries' using the Direct Runner. */ -class NexmarkInProcessDriver extends NexmarkDriver { +class NexmarkDirectDriver extends NexmarkDriver { /** * Command line flags. */ - public interface NexmarkInProcessOptions extends Options, DataflowPipelineOptions { + public interface NexmarkDirectOptions extends Options, DirectOptions { } /** * Entry point. */ public static void main(String[] args) { - NexmarkInProcessOptions options = + NexmarkDirectOptions options = PipelineOptionsFactory.fromArgs(args) .withValidation() - .as(NexmarkInProcessOptions.class); - options.setRunner(InProcessPipelineRunner.class); - NexmarkInProcessRunner runner = new NexmarkInProcessRunner(options); - new NexmarkInProcessDriver().runAll(options, runner); + .as(NexmarkDirectOptions.class); + options.setRunner(DirectRunner.class); + NexmarkDirectRunner runner = new NexmarkDirectRunner(options); + new NexmarkDirectDriver().runAll(options, runner); } } diff --git a/integration/java/src/main/java/org/apache/beam/integration/nexmark/NexmarkInProcessRunner.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkDirectRunner.java similarity index 77% rename from integration/java/src/main/java/org/apache/beam/integration/nexmark/NexmarkInProcessRunner.java rename to integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkDirectRunner.java index ba141f947042c..ffb5649f26961 100644 --- a/integration/java/src/main/java/org/apache/beam/integration/nexmark/NexmarkInProcessRunner.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkDirectRunner.java @@ -15,16 +15,15 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.beam.integration.nexmark; import javax.annotation.Nullable; /** - * Run a singe Nexmark query using a given configuration on Google Dataflow. + * Run a single query using the Direct Runner. */ -class NexmarkInProcessRunner extends NexmarkRunner { - public NexmarkInProcessRunner(NexmarkInProcessDriver.NexmarkInProcessOptions options) { +class NexmarkDirectRunner extends NexmarkRunner { + public NexmarkDirectRunner(NexmarkDirectDriver.NexmarkDirectOptions options) { super(options); } @@ -51,7 +50,7 @@ protected boolean canMonitor() { @Override protected void invokeBuilderForPublishOnlyPipeline(PipelineBuilder builder) { throw new UnsupportedOperationException( - "Cannot use --pubSubMode=COMBINED with InProcessPipelineRunner"); + "Cannot use --pubSubMode=COMBINED with DirectRunner"); } /** @@ -61,7 +60,7 @@ protected void invokeBuilderForPublishOnlyPipeline(PipelineBuilder builder) { @Override protected void waitForPublisherPreload() { throw new UnsupportedOperationException( - "Cannot use --pubSubMode=COMBINED with InProcessPipelineRunner"); + "Cannot use --pubSubMode=COMBINED with DirectRunner"); } /** @@ -71,7 +70,9 @@ protected void waitForPublisherPreload() { @Override @Nullable protected NexmarkPerf monitor(NexmarkQuery query) { - throw new UnsupportedOperationException( - "Cannot use --monitorJobs=true with InProcessPipelineRunner"); + return null; + //TODO Ismael Check how we can do this a real implementation +// throw new UnsupportedOperationException( +// "Cannot use --monitorJobs=true with DirectRunner"); } } diff --git a/integration/java/src/main/java/org/apache/beam/integration/nexmark/NexmarkDriver.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkDriver.java similarity index 98% rename from integration/java/src/main/java/org/apache/beam/integration/nexmark/NexmarkDriver.java rename to integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkDriver.java index dbc1ce27a5049..e6a7b0b44a35f 100644 --- a/integration/java/src/main/java/org/apache/beam/integration/nexmark/NexmarkDriver.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkDriver.java @@ -15,12 +15,8 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.beam.integration.nexmark; -import org.joda.time.Duration; -import org.joda.time.Instant; - import java.io.IOException; import java.nio.charset.StandardCharsets; import java.nio.file.Files; @@ -30,12 +26,14 @@ import java.util.LinkedHashMap; import java.util.List; import java.util.Map; - import javax.annotation.Nullable; +import org.joda.time.Duration; +import org.joda.time.Instant; + /** * An implementation of the 'NEXMark queries' for Google Dataflow. - * These are 11 queries over a three table schema representing on online auction system: + * These are multiple queries over a three table schema representing an online auction system: *

                *
              • {@link Person} represents a person submitting an item for auction and/or making a bid * on an auction. @@ -43,12 +41,11 @@ *
              • {@link Bid} represents a bid for an item under auction. *
              * The queries exercise many aspects of streaming dataflow. - *

              + * *

              We synthesize the creation of people, auctions and bids in real-time. The data is not * particularly sensible. - *

              - *

              See - * + * + *

              See * http://datalab.cs.pdx.edu/niagaraST/NEXMark/ */ public class NexmarkDriver { diff --git a/integration/java/src/main/java/org/apache/beam/integration/nexmark/NexmarkFlinkDriver.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkFlinkDriver.java similarity index 91% rename from integration/java/src/main/java/org/apache/beam/integration/nexmark/NexmarkFlinkDriver.java rename to integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkFlinkDriver.java index 0029a3653bbe8..61a7d293b69ac 100644 --- a/integration/java/src/main/java/org/apache/beam/integration/nexmark/NexmarkFlinkDriver.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkFlinkDriver.java @@ -15,15 +15,14 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.beam.integration.nexmark; import org.apache.beam.runners.flink.FlinkPipelineOptions; -import org.apache.beam.runners.flink.FlinkPipelineRunner; +import org.apache.beam.runners.flink.FlinkRunner; import org.apache.beam.sdk.options.PipelineOptionsFactory; /** - * Run NexMark queries using Beam-on-Flink runner. + * Run NexMark queries using the Flink runner. */ public class NexmarkFlinkDriver extends NexmarkDriver { /** @@ -40,7 +39,7 @@ public static void main(String[] args) { NexmarkFlinkOptions options = PipelineOptionsFactory.fromArgs(args) .withValidation() .as(NexmarkFlinkOptions.class); - options.setRunner(FlinkPipelineRunner.class); + options.setRunner(FlinkRunner.class); NexmarkFlinkRunner runner = new NexmarkFlinkRunner(options); new NexmarkFlinkDriver().runAll(options, runner); } diff --git a/integration/java/src/main/java/org/apache/beam/integration/nexmark/NexmarkFlinkRunner.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkFlinkRunner.java similarity index 96% rename from integration/java/src/main/java/org/apache/beam/integration/nexmark/NexmarkFlinkRunner.java rename to integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkFlinkRunner.java index 569aef66d6388..72debc455871a 100644 --- a/integration/java/src/main/java/org/apache/beam/integration/nexmark/NexmarkFlinkRunner.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkFlinkRunner.java @@ -15,13 +15,12 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.beam.integration.nexmark; import javax.annotation.Nullable; /** - * Run a specific Nexmark query using the Bean-on-Flink runner. + * Run a query using the Flink runner. */ public class NexmarkFlinkRunner extends NexmarkRunner { @Override diff --git a/integration/java/src/main/java/org/apache/beam/integration/nexmark/NexmarkGoogleDriver.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkGoogleDriver.java similarity index 91% rename from integration/java/src/main/java/org/apache/beam/integration/nexmark/NexmarkGoogleDriver.java rename to integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkGoogleDriver.java index 253415522ebbb..003a79b99f608 100644 --- a/integration/java/src/main/java/org/apache/beam/integration/nexmark/NexmarkGoogleDriver.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkGoogleDriver.java @@ -15,18 +15,17 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.beam.integration.nexmark; -import org.apache.beam.runners.dataflow.DataflowPipelineRunner; +import javax.annotation.Nullable; +import org.apache.beam.runners.dataflow.DataflowRunner; import org.apache.beam.runners.dataflow.options.DataflowPipelineOptions; import org.apache.beam.sdk.options.Description; import org.apache.beam.sdk.options.PipelineOptionsFactory; -import javax.annotation.Nullable; /** * An implementation of the 'NEXMark queries' for Google Dataflow. - * These are 11 queries over a three table schema representing on online auction system: + * These are multiple queries over a three table schema representing an online auction system: *

                *
              • {@link Person} represents a person submitting an item for auction and/or making a bid * on an auction. @@ -34,12 +33,11 @@ *
              • {@link Bid} represents a bid for an item under auction. *
              * The queries exercise many aspects of streaming dataflow. - *

              + * *

              We synthesize the creation of people, auctions and bids in real-time. The data is not * particularly sensible. - *

              - *

              See - * + * + *

              See * http://datalab.cs.pdx.edu/niagaraST/NEXMark/ */ class NexmarkGoogleDriver extends NexmarkDriver { @@ -82,7 +80,7 @@ public static void main(String[] args) { NexmarkGoogleOptions options = PipelineOptionsFactory.fromArgs(args) .withValidation() .as(NexmarkGoogleOptions.class); - options.setRunner(DataflowPipelineRunner.class); + options.setRunner(DataflowRunner.class); NexmarkGoogleRunner runner = new NexmarkGoogleRunner(options); new NexmarkGoogleDriver().runAll(options, runner); } diff --git a/integration/java/src/main/java/org/apache/beam/integration/nexmark/NexmarkGoogleRunner.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkGoogleRunner.java similarity index 93% rename from integration/java/src/main/java/org/apache/beam/integration/nexmark/NexmarkGoogleRunner.java rename to integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkGoogleRunner.java index 4b735922ef76b..444534a47cb06 100644 --- a/integration/java/src/main/java/org/apache/beam/integration/nexmark/NexmarkGoogleRunner.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkGoogleRunner.java @@ -15,30 +15,27 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.beam.integration.nexmark; -import org.apache.beam.runners.dataflow.DataflowPipelineJob; -import org.apache.beam.sdk.PipelineResult; -import org.apache.beam.sdk.runners.AggregatorRetrievalException; -import org.apache.beam.sdk.transforms.Aggregator; +import static com.google.common.base.Preconditions.checkNotNull; -import com.google.api.services.dataflow.model.JobMetrics; import com.google.api.services.dataflow.model.MetricUpdate; -import com.google.common.base.Preconditions; import com.google.common.collect.Iterables; -import org.joda.time.Duration; -import org.joda.time.Instant; - import java.io.IOException; import java.math.BigDecimal; import java.util.ArrayList; import java.util.Collection; import java.util.List; -import java.util.concurrent.TimeUnit; import javax.annotation.Nullable; +import org.apache.beam.runners.dataflow.DataflowPipelineJob; +import org.apache.beam.sdk.AggregatorRetrievalException; +import org.apache.beam.sdk.PipelineResult; +import org.apache.beam.sdk.transforms.Aggregator; +import org.joda.time.Duration; +import org.joda.time.Instant; + /** * Run a singe Nexmark query using a given configuration on Google Dataflow. */ @@ -139,8 +136,8 @@ protected void invokeBuilderForPublishOnlyPipeline(PipelineBuilder builder) { */ @Override protected void waitForPublisherPreload() { - Preconditions.checkNotNull(publisherMonitor); - Preconditions.checkNotNull(publisherResult); + checkNotNull(publisherMonitor); + checkNotNull(publisherResult); if (!options.getMonitorJobs()) { return; } @@ -362,13 +359,14 @@ protected NexmarkPerf monitor(NexmarkQuery query) { if (!publisherCancelled) { publisherJob.cancel(); } - publisherJob.waitToFinish(5, TimeUnit.MINUTES, null); + publisherJob.waitUntilFinish(Duration.standardMinutes(5)); } catch (IOException e) { throw new RuntimeException("Unable to cancel publisher job: ", e); - } catch (InterruptedException e) { - Thread.interrupted(); - throw new RuntimeException("Interrupted: publish job still running.", e); - } + } //TODO Ismael +// catch (InterruptedException e) { +// Thread.interrupted(); +// throw new RuntimeException("Interrupted: publish job still running.", e); +// } } return perf; @@ -393,19 +391,20 @@ private MetricType getMetricType(MetricUpdate metric) { /** * Check that watermarks are not too far behind. - *

              + * *

              Returns a list of errors detected. */ private List checkWatermarks(DataflowPipelineJob job, long startMsSinceEpoch) { long now = System.currentTimeMillis(); List errors = new ArrayList<>(); - try { - JobMetrics metricResponse = job.getDataflowClient() - .projects() - .jobs() - .getMetrics(job.getProjectId(), job.getJobId()) - .execute(); - List metrics = metricResponse.getMetrics(); +// try { + //TODO Ismael Ask google +// JobMetrics metricResponse = job.getDataflowClient() +// .projects() +// .jobs() +// .getMetrics(job.getProjectId(), job.getJobId()) +// .execute(); + List metrics = null; // metricResponse.getMetrics(); if (metrics != null) { boolean foundWatermarks = false; for (MetricUpdate metric : metrics) { @@ -420,24 +419,24 @@ private List checkWatermarks(DataflowPipelineJob job, long startMsSinceE continue; } Instant value = - new Instant(scalar.divideToIntegralValue(new BigDecimal(1000)).longValueExact()); + new Instant(scalar.divideToIntegralValue(new BigDecimal(1000)).longValueExact()); Instant updateTime = Instant.parse(metric.getUpdateTime()); if (options.getWatermarkValidationDelaySeconds() == null - || now > startMsSinceEpoch - + Duration.standardSeconds(options.getWatermarkValidationDelaySeconds()) - .getMillis()) { + || now > startMsSinceEpoch + + Duration.standardSeconds(options.getWatermarkValidationDelaySeconds()) + .getMillis()) { Duration threshold = null; if (type == MetricType.SYSTEM_WATERMARK && options.getMaxSystemLagSeconds() != null) { threshold = Duration.standardSeconds(options.getMaxSystemLagSeconds()); } else if (type == MetricType.DATA_WATERMARK - && options.getMaxDataLagSeconds() != null) { + && options.getMaxDataLagSeconds() != null) { threshold = Duration.standardSeconds(options.getMaxDataLagSeconds()); } if (threshold != null && value.isBefore(updateTime.minus(threshold))) { String msg = String.format("High lag for %s: %s vs %s (allowed lag of %s)", - metric.getName().getName(), value, updateTime, threshold); + metric.getName().getName(), value, updateTime, threshold); errors.add(msg); NexmarkUtils.console(msg); } @@ -450,9 +449,9 @@ private List checkWatermarks(DataflowPipelineJob job, long startMsSinceE } } } - } catch (IOException e) { - NexmarkUtils.console("Warning: failed to get JobMetrics: " + e); - } +// } catch (IOException e) { +// NexmarkUtils.console("Warning: failed to get JobMetrics: " + e); +// } return errors; } diff --git a/integration/java/src/main/java/org/apache/beam/integration/nexmark/NexmarkPerf.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkPerf.java similarity index 99% rename from integration/java/src/main/java/org/apache/beam/integration/nexmark/NexmarkPerf.java rename to integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkPerf.java index 6eb7267b7eaef..37b621311ab92 100644 --- a/integration/java/src/main/java/org/apache/beam/integration/nexmark/NexmarkPerf.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkPerf.java @@ -15,7 +15,6 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.beam.integration.nexmark; import com.fasterxml.jackson.annotation.JsonProperty; @@ -23,7 +22,6 @@ import java.io.IOException; import java.util.List; - import javax.annotation.Nullable; /** diff --git a/integration/java/src/main/java/org/apache/beam/integration/nexmark/NexmarkQuery.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkQuery.java similarity index 81% rename from integration/java/src/main/java/org/apache/beam/integration/nexmark/NexmarkQuery.java rename to integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkQuery.java index 462660948bd00..5ef419126db26 100644 --- a/integration/java/src/main/java/org/apache/beam/integration/nexmark/NexmarkQuery.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkQuery.java @@ -15,9 +15,10 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.beam.integration.nexmark; +import javax.annotation.Nullable; + import org.apache.beam.sdk.transforms.Aggregator; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.Filter; @@ -31,8 +32,6 @@ import org.joda.time.Instant; -import javax.annotation.Nullable; - /** * Base class for the eight 'NEXMark' queries. Supplies some fragments common to * multiple queries. @@ -54,7 +53,7 @@ public Boolean apply(Event event) { /** DoFn to convert a new person event to a person. */ protected static final DoFn AS_PERSON = new DoFn() { - @Override + @ProcessElement public void processElement(ProcessContext c) { c.output(c.element().newPerson); } @@ -71,7 +70,7 @@ public Boolean apply(Event event) { /** DoFn to convert a new auction event to an auction. */ protected static final DoFn AS_AUCTION = new DoFn() { - @Override + @ProcessElement public void processElement(ProcessContext c) { c.output(c.element().newAuction); } @@ -88,7 +87,7 @@ public Boolean apply(Event event) { /** DoFn to convert a bid event to a bid. */ protected static final DoFn AS_BID = new DoFn() { - @Override + @ProcessElement public void processElement(ProcessContext c) { c.output(c.element().bid); } @@ -96,9 +95,8 @@ public void processElement(ProcessContext c) { /** Transform to key each person by their id. */ protected static final ParDo.Bound> PERSON_BY_ID = - ParDo.named("PersonById") - .of(new DoFn>() { - @Override + ParDo.of(new DoFn>() { + @ProcessElement public void processElement(ProcessContext c) { c.output(KV.of(c.element().id, c.element())); } @@ -106,9 +104,8 @@ public void processElement(ProcessContext c) { /** Transform to key each auction by its id. */ protected static final ParDo.Bound> AUCTION_BY_ID = - ParDo.named("AuctionById") - .of(new DoFn>() { - @Override + ParDo.of(new DoFn>() { + @ProcessElement public void processElement(ProcessContext c) { c.output(KV.of(c.element().id, c.element())); } @@ -116,9 +113,8 @@ public void processElement(ProcessContext c) { /** Transform to key each auction by its seller id. */ protected static final ParDo.Bound> AUCTION_BY_SELLER = - ParDo.named("AuctionBySeller") - .of(new DoFn>() { - @Override + ParDo.of(new DoFn>() { + @ProcessElement public void processElement(ProcessContext c) { c.output(KV.of(c.element().seller, c.element())); } @@ -126,9 +122,8 @@ public void processElement(ProcessContext c) { /** Transform to key each bid by it's auction id. */ protected static final ParDo.Bound> BID_BY_AUCTION = - ParDo.named("BidByAuction") - .of(new DoFn>() { - @Override + ParDo.of(new DoFn>() { + @ProcessElement public void processElement(ProcessContext c) { c.output(KV.of(c.element().auction, c.element())); } @@ -136,9 +131,8 @@ public void processElement(ProcessContext c) { /** Transform to project the auction id from each bid. */ protected static final ParDo.Bound BID_TO_AUCTION = - ParDo.named("BidToAuction") - .of(new DoFn() { - @Override + ParDo.of(new DoFn() { + @ProcessElement public void processElement(ProcessContext c) { c.output(c.element().auction); } @@ -146,9 +140,8 @@ public void processElement(ProcessContext c) { /** Transform to project the price from each bid. */ protected static final ParDo.Bound BID_TO_PRICE = - ParDo.named("BidToPrice") - .of(new DoFn() { - @Override + ParDo.of(new DoFn() { + @ProcessElement public void processElement(ProcessContext c) { c.output(c.element().price); } @@ -156,9 +149,8 @@ public void processElement(ProcessContext c) { /** Transform to emit each event with the timestamp embedded within it. */ public static final ParDo.Bound EVENT_TIMESTAMP_FROM_DATA = - ParDo.named("OutputWithTimestamp") - .of(new DoFn() { - @Override + ParDo.of(new DoFn() { + @ProcessElement public void processElement(ProcessContext c) { Event e = c.element(); if (e.bid != null) { @@ -177,9 +169,9 @@ public void processElement(ProcessContext c) { protected static final PTransform, PCollection> JUST_NEW_AUCTIONS = new PTransform, PCollection>("justNewAuctions") { @Override - public PCollection apply(PCollection input) { - return input.apply(Filter.byPredicate(IS_NEW_AUCTION).named("IsAuction")) - .apply(ParDo.named("AsAuction").of(AS_AUCTION)); + public PCollection expand(PCollection input) { + return input.apply("IsNewAuction", Filter.by(IS_NEW_AUCTION)) + .apply("AsAuction", ParDo.of(AS_AUCTION)); } }; @@ -189,9 +181,9 @@ public PCollection apply(PCollection input) { protected static final PTransform, PCollection> JUST_NEW_PERSONS = new PTransform, PCollection>("justNewPersons") { @Override - public PCollection apply(PCollection input) { - return input.apply(Filter.byPredicate(IS_NEW_PERSON).named("IsPerson")) - .apply(ParDo.named("AsPerson").of(AS_PERSON)); + public PCollection expand(PCollection input) { + return input.apply("IsNewPerson", Filter.by(IS_NEW_PERSON)) + .apply("AsPerson", ParDo.of(AS_PERSON)); } }; @@ -201,9 +193,9 @@ public PCollection apply(PCollection input) { protected static final PTransform, PCollection> JUST_BIDS = new PTransform, PCollection>("justBids") { @Override - public PCollection apply(PCollection input) { - return input.apply(Filter.byPredicate(IS_BID).named("IsBid")) - .apply(ParDo.named("AsBid").of(AS_BID)); + public PCollection expand(PCollection input) { + return input.apply("IsBid", Filter.by(IS_BID)) + .apply("AsBid", ParDo.of(AS_BID)); } }; @@ -241,25 +233,27 @@ public Aggregator getFatalCount() { protected abstract PCollection applyPrim(PCollection events); @Override - public PCollection> apply(PCollection events) { + public PCollection> expand(PCollection events) { if (configuration.debug) { events = events // Monitor events as they go by. - .apply(eventMonitor.getTransform()) + .apply(name + ".Monitor", eventMonitor.getTransform()) // Count each type of event. - .apply(NexmarkUtils.snoop(name)); + .apply(name + ".Snoop", NexmarkUtils.snoop(name)); } if (configuration.cpuDelayMs > 0) { // Slow down by pegging one core at 100%. - events = events.apply(NexmarkUtils.cpuDelay(name, configuration.cpuDelayMs)); + events = events.apply(name + ".CpuDelay", + NexmarkUtils.cpuDelay(name, configuration.cpuDelayMs)); } if (configuration.diskBusyBytes > 0) { // Slow down by forcing bytes to durable store. - events = events.apply(NexmarkUtils.diskBusy(name, configuration.diskBusyBytes)); + events = events.apply(name + ".DiskBusy", + NexmarkUtils.diskBusy(name, configuration.diskBusyBytes)); } // Run the query. @@ -267,10 +261,10 @@ public PCollection> apply(PCollection events) if (configuration.debug) { // Monitor results as they go by. - queryResults = queryResults.apply(resultMonitor.getTransform()); + queryResults = queryResults.apply(name + ".Debug", resultMonitor.getTransform()); } // Timestamp the query results. - return queryResults.apply(NexmarkUtils.stamp(name)); + return queryResults.apply(name + ".Stamp", NexmarkUtils.stamp(name)); } } diff --git a/integration/java/src/main/java/org/apache/beam/integration/nexmark/NexmarkQueryModel.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkQueryModel.java similarity index 99% rename from integration/java/src/main/java/org/apache/beam/integration/nexmark/NexmarkQueryModel.java rename to integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkQueryModel.java index b42042f0de8e8..a23f82b0654ea 100644 --- a/integration/java/src/main/java/org/apache/beam/integration/nexmark/NexmarkQueryModel.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkQueryModel.java @@ -15,16 +15,8 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.beam.integration.nexmark; -import org.apache.beam.sdk.transforms.SerializableFunction; -import org.apache.beam.sdk.values.TimestampedValue; - -import org.joda.time.Duration; -import org.joda.time.Instant; -import org.junit.Assert; - import java.io.Serializable; import java.util.ArrayList; import java.util.Collection; @@ -33,6 +25,13 @@ import java.util.List; import java.util.Set; +import org.apache.beam.sdk.transforms.SerializableFunction; +import org.apache.beam.sdk.values.TimestampedValue; + +import org.joda.time.Duration; +import org.joda.time.Instant; +import org.junit.Assert; + /** * Base class for models of the eight NEXMark queries. Provides an assertion * function which can be applied against the actual query results to check their consistency diff --git a/integration/java/src/main/java/org/apache/beam/integration/nexmark/NexmarkRunner.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkRunner.java similarity index 88% rename from integration/java/src/main/java/org/apache/beam/integration/nexmark/NexmarkRunner.java rename to integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkRunner.java index b7151f87a143c..8d0893427f6d8 100644 --- a/integration/java/src/main/java/org/apache/beam/integration/nexmark/NexmarkRunner.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkRunner.java @@ -15,28 +15,13 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.beam.integration.nexmark; -import org.apache.beam.sdk.Pipeline; -import org.apache.beam.sdk.PipelineResult; -import org.apache.beam.sdk.io.AvroIO; -import org.apache.beam.sdk.io.BigQueryIO; -import org.apache.beam.sdk.io.PubsubIO; -import org.apache.beam.sdk.io.TextIO; -import org.apache.beam.sdk.testing.PAssert; -import org.apache.beam.sdk.transforms.DoFn; -import org.apache.beam.sdk.transforms.ParDo; -import org.apache.beam.sdk.values.PCollection; -import org.apache.beam.sdk.values.PCollectionTuple; -import org.apache.beam.sdk.values.TimestampedValue; -import org.apache.beam.sdk.values.TupleTag; -import org.apache.beam.sdk.values.TupleTagList; +import static com.google.common.base.Preconditions.checkState; import com.google.api.services.bigquery.model.TableFieldSchema; import com.google.api.services.bigquery.model.TableRow; import com.google.api.services.bigquery.model.TableSchema; -import com.google.common.base.Preconditions; import com.google.common.base.Strings; import com.google.common.collect.ImmutableList; import com.google.common.collect.Lists; @@ -48,6 +33,21 @@ import java.util.concurrent.ThreadLocalRandom; import javax.annotation.Nullable; +import org.apache.beam.sdk.Pipeline; +import org.apache.beam.sdk.PipelineResult; +import org.apache.beam.sdk.io.AvroIO; +import org.apache.beam.sdk.io.PubsubIO; +import org.apache.beam.sdk.io.TextIO; +import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO; +import org.apache.beam.sdk.testing.PAssert; +import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.transforms.ParDo; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.PCollectionTuple; +import org.apache.beam.sdk.values.TimestampedValue; +import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.TupleTagList; + /** * Run a single Nexmark query using a given configuration. */ @@ -264,10 +264,10 @@ private String logsDir(long now) { private PCollection sourceEventsFromSynthetic(Pipeline p) { if (isStreaming()) { NexmarkUtils.console("Generating %d events in streaming mode", configuration.numEvents); - return p.apply(NexmarkUtils.streamEventsSource(queryName, configuration)); + return p.apply(queryName + ".ReadUnbounded", NexmarkUtils.streamEventsSource(configuration)); } else { NexmarkUtils.console("Generating %d events in batch mode", configuration.numEvents); - return p.apply(NexmarkUtils.batchEventsSource(queryName, configuration)); + return p.apply(queryName + ".ReadBounded", NexmarkUtils.batchEventsSource(configuration)); } } @@ -287,15 +287,14 @@ private PCollection sourceEventsFromPubsub(Pipeline p, long now) { subscription = getPubsub().createSubscription(shortTopic, shortSubscription).getPath(); } NexmarkUtils.console("Reading events from Pubsub %s", subscription); - PubsubIO.Read.Bound io = - PubsubIO.Read.named(queryName + ".ReadPubsubEvents") - .subscription(subscription) - .idLabel(NexmarkUtils.PUBSUB_ID) - .withCoder(Event.CODER); + PubsubIO.Read io = + PubsubIO.read().subscription(subscription) + .idLabel(NexmarkUtils.PUBSUB_ID) + .withCoder(Event.CODER); if (!configuration.usePubsubPublishTime) { io = io.timestampLabel(NexmarkUtils.PUBSUB_TIMESTAMP); } - return p.apply(io); + return p.apply(queryName + ".ReadPubsubEvents", io); } /** @@ -308,10 +307,10 @@ private PCollection sourceEventsFromAvro(Pipeline p) { } NexmarkUtils.console("Reading events from Avro files at %s", filename); return p - .apply(AvroIO.Read.named(queryName + ".ReadAvroEvents") + .apply(queryName + ".ReadAvroEvents", AvroIO.Read .from(filename + "*.avro") .withSchema(Event.class)) - .apply(NexmarkQuery.EVENT_TIMESTAMP_FROM_DATA); + .apply("OutputWithTimestamp", NexmarkQuery.EVENT_TIMESTAMP_FROM_DATA); } /** @@ -333,15 +332,14 @@ private void sinkEventsToPubsub(PCollection events, long now) { topic = getPubsub().createTopic(shortTopic).getPath(); } NexmarkUtils.console("Writing events to Pubsub %s", topic); - PubsubIO.Write.Bound io = - PubsubIO.Write.named(queryName + ".WritePubsubEvents") - .topic(topic) + PubsubIO.Write io = + PubsubIO.write().topic(topic) .idLabel(NexmarkUtils.PUBSUB_ID) .withCoder(Event.CODER); if (!configuration.usePubsubPublishTime) { io = io.timestampLabel(NexmarkUtils.PUBSUB_TIMESTAMP); } - events.apply(io); + events.apply(queryName + ".WritePubsubEvents", io); } /** @@ -356,14 +354,13 @@ private void sinkResultsToPubsub(PCollection formattedResults, long now) topic = getPubsub().createTopic(shortTopic).getPath(); } NexmarkUtils.console("Writing results to Pubsub %s", topic); - PubsubIO.Write.Bound io = - PubsubIO.Write.named(queryName + ".WritePubsubResults") - .topic(topic) - .idLabel(NexmarkUtils.PUBSUB_ID); + PubsubIO.Write io = + PubsubIO.write().topic(topic) + .idLabel(NexmarkUtils.PUBSUB_ID); if (!configuration.usePubsubPublishTime) { io = io.timestampLabel(NexmarkUtils.PUBSUB_TIMESTAMP); } - formattedResults.apply(io); + formattedResults.apply(queryName + ".WritePubsubResults", io); } /** @@ -384,25 +381,18 @@ private void sinkEventsToAvro(PCollection source) { throw new RuntimeException("Missing --outputPath"); } NexmarkUtils.console("Writing events to Avro files at %s", filename); - source.apply(AvroIO.Write.named(queryName + ".WriteAvroEvents") - .to(filename + "/event") - .withSuffix(".avro") - .withSchema(Event.class)); + source.apply(queryName + ".WriteAvroEvents", + AvroIO.Write.to(filename + "/event").withSuffix(".avro").withSchema(Event.class)); source.apply(NexmarkQuery.JUST_BIDS) - .apply(AvroIO.Write.named(queryName + ".WriteAvroBids") - .to(filename + "/bid") - .withSuffix(".avro") - .withSchema(Bid.class)); + .apply(queryName + ".WriteAvroBids", + AvroIO.Write.to(filename + "/bid").withSuffix(".avro").withSchema(Bid.class)); source.apply(NexmarkQuery.JUST_NEW_AUCTIONS) - .apply(AvroIO.Write.named( - queryName + ".WriteAvroAuctions") - .to(filename + "/auction") - .withSuffix(".avro") - .withSchema(Auction.class)); + .apply(queryName + ".WriteAvroAuctions", + AvroIO.Write.to(filename + "/auction").withSuffix(".avro") + .withSchema(Auction.class)); source.apply(NexmarkQuery.JUST_NEW_PERSONS) - .apply(AvroIO.Write.named(queryName + ".WriteAvroPeople") - .to(filename + "/person") - .withSuffix(".avro") + .apply(queryName + ".WriteAvroPeople", + AvroIO.Write.to(filename + "/person").withSuffix(".avro") .withSchema(Person.class)); } @@ -412,13 +402,12 @@ private void sinkEventsToAvro(PCollection source) { private void sinkResultsToText(PCollection formattedResults, long now) { String filename = textFilename(now); NexmarkUtils.console("Writing results to text files at %s", filename); - formattedResults.apply( - TextIO.Write.named(queryName + ".WriteTextResults") - .to(filename)); + formattedResults.apply(queryName + ".WriteTextResults", + TextIO.Write.to(filename)); } private static class StringToTableRow extends DoFn { - @Override + @ProcessElement public void processElement(ProcessContext c) { int n = ThreadLocalRandom.current().nextInt(10); List records = new ArrayList<>(n); @@ -445,15 +434,13 @@ private void sinkResultsToBigQuery( new TableFieldSchema().setName("value").setType("STRING"))))); NexmarkUtils.console("Writing results to BigQuery table %s", tableSpec); BigQueryIO.Write.Bound io = - BigQueryIO.Write.named(queryName + ".WriteBigQueryResults") - .to(tableSpec) + BigQueryIO.Write.to(tableSpec) .withSchema(tableSchema) .withCreateDisposition(BigQueryIO.Write.CreateDisposition.CREATE_IF_NEEDED) .withWriteDisposition(BigQueryIO.Write.WriteDisposition.WRITE_APPEND); formattedResults - .apply(ParDo.named(queryName + ".StringToTableRow") - .of(new StringToTableRow())) - .apply(io); + .apply(queryName + ".StringToTableRow", ParDo.of(new StringToTableRow())) + .apply(queryName + ".WriteBigQueryResults", io); } // ================================================================================ @@ -481,8 +468,8 @@ private PCollection createSource(Pipeline p, final long now) { break; case PUBLISH_ONLY: // Send synthesized events to Pubsub in this job. - sinkEventsToPubsub(sourceEventsFromSynthetic(p).apply(NexmarkUtils.snoop(queryName)), - now); + sinkEventsToPubsub(sourceEventsFromSynthetic(p).apply(queryName + ".Snoop", + NexmarkUtils.snoop(queryName)), now); break; case COMBINED: // Send synthesized events to Pubsub in separate publisher job. @@ -495,7 +482,8 @@ public void build(Options publishOnlyOptions) { NexmarkUtils.setupPipeline(configuration.coderStrategy, sp); publisherMonitor = new Monitor(queryName, "publisher"); sinkEventsToPubsub( - sourceEventsFromSynthetic(sp).apply(publisherMonitor.getTransform()), + sourceEventsFromSynthetic(sp) + .apply(queryName + ".Monitor", publisherMonitor.getTransform()), now); publisherResult = sp.run(); } @@ -523,7 +511,7 @@ public void build(Options publishOnlyOptions) { private static final TupleTag SIDE = new TupleTag(){}; private static class PartitionDoFn extends DoFn { - @Override + @ProcessElement public void processElement(ProcessContext c) { if (c.element().hashCode() % 2 == 0) { c.output(c.element()); @@ -539,19 +527,21 @@ public void processElement(ProcessContext c) { private void sink(PCollection> results, long now) { if (configuration.sinkType == NexmarkUtils.SinkType.COUNT_ONLY) { // Avoid the cost of formatting the results. - results.apply(NexmarkUtils.devNull(queryName)); + results.apply(queryName + ".DevNull", NexmarkUtils.devNull(queryName)); return; } - PCollection formattedResults = results.apply(NexmarkUtils.format(queryName)); + PCollection formattedResults = + results.apply(queryName + ".Format", NexmarkUtils.format(queryName)); if (options.getLogResults()) { - formattedResults = formattedResults.apply(NexmarkUtils.log(queryName + ".Results")); + formattedResults = formattedResults.apply(queryName + ".Results.Log", + NexmarkUtils.log(queryName + ".Results")); } switch (configuration.sinkType) { case DEVNULL: // Discard all results - formattedResults.apply(NexmarkUtils.devNull(queryName)); + formattedResults.apply(queryName + ".DevNull", NexmarkUtils.devNull(queryName)); break; case PUBSUB: sinkResultsToPubsub(formattedResults, now); @@ -565,9 +555,8 @@ private void sink(PCollection> results, long now) { break; case BIGQUERY: // Multiple BigQuery backends to mimic what most customers do. - PCollectionTuple res = formattedResults.apply( - ParDo.named(queryName + ".Partition") - .withOutputTags(MAIN, TupleTagList.of(SIDE)) + PCollectionTuple res = formattedResults.apply(queryName + ".Partition", + ParDo.withOutputTags(MAIN, TupleTagList.of(SIDE)) .of(new PartitionDoFn())); sinkResultsToBigQuery(res.get(MAIN), now, "main"); sinkResultsToBigQuery(res.get(SIDE), now, "side"); @@ -617,9 +606,9 @@ public NexmarkPerf run(NexmarkConfiguration runConfiguration) { // // Setup per-run state. // - Preconditions.checkState(configuration == null); - Preconditions.checkState(pubsub == null); - Preconditions.checkState(queryName == null); + checkState(configuration == null); + checkState(pubsub == null); + checkState(queryName == null); configuration = runConfiguration; // GCS URI patterns to delete on exit. @@ -681,7 +670,8 @@ public NexmarkPerf run(NexmarkConfiguration runConfiguration) { PCollection source = createSource(p, now); if (options.getLogEvents()) { - source = source.apply(NexmarkUtils.log(queryName + ".Events")); + source = source.apply(queryName + ".Events.Log", + NexmarkUtils.log(queryName + ".Events")); } // Source will be null if source type is PUBSUB and mode is PUBLISH_ONLY. @@ -728,6 +718,7 @@ public NexmarkPerf run(NexmarkConfiguration runConfiguration) { waitForPublisherPreload(); } mainResult = p.run(); + mainResult.waitUntilFinish(); return monitor(query); } finally { // diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkSparkDriver.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkSparkDriver.java new file mode 100644 index 0000000000000..1ea963d819363 --- /dev/null +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkSparkDriver.java @@ -0,0 +1,46 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.integration.nexmark; + +import org.apache.beam.runners.spark.SparkPipelineOptions; +import org.apache.beam.runners.spark.SparkRunner; +import org.apache.beam.sdk.options.PipelineOptionsFactory; + +/** + * Run NexMark queries using the Spark runner. + */ +class NexmarkSparkDriver extends NexmarkDriver { + /** + * Command line flags. + */ + public interface NexmarkSparkOptions extends Options, SparkPipelineOptions { + } + + /** + * Entry point. + */ + public static void main(String[] args) { + NexmarkSparkOptions options = + PipelineOptionsFactory.fromArgs(args) + .withValidation() + .as(NexmarkSparkOptions.class); + options.setRunner(SparkRunner.class); + NexmarkSparkRunner runner = new NexmarkSparkRunner(options); + new NexmarkSparkDriver().runAll(options, runner); + } +} diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkSparkRunner.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkSparkRunner.java new file mode 100644 index 0000000000000..81c6b2bfa8b6d --- /dev/null +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkSparkRunner.java @@ -0,0 +1,66 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.integration.nexmark; + +import javax.annotation.Nullable; + +/** + * Run a query using the Spark runner. + */ +public class NexmarkSparkRunner extends NexmarkRunner { + @Override + protected boolean isStreaming() { + return options.isStreaming(); + } + + @Override + protected int coresPerWorker() { + return 4; + } + + @Override + protected int maxNumWorkers() { + return 5; + } + + @Override + protected boolean canMonitor() { + return false; + } + + @Override + protected void invokeBuilderForPublishOnlyPipeline( + PipelineBuilder builder) { + builder.build(options); + } + + @Override + protected void waitForPublisherPreload() { + throw new UnsupportedOperationException(); + } + + @Override + @Nullable + protected NexmarkPerf monitor(NexmarkQuery query) { + return null; + } + + public NexmarkSparkRunner(NexmarkSparkDriver.NexmarkSparkOptions options) { + super(options); + } +} diff --git a/integration/java/src/main/java/org/apache/beam/integration/nexmark/NexmarkSuite.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkSuite.java similarity index 95% rename from integration/java/src/main/java/org/apache/beam/integration/nexmark/NexmarkSuite.java rename to integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkSuite.java index cccaeb146723c..bc47540b9a4b1 100644 --- a/integration/java/src/main/java/org/apache/beam/integration/nexmark/NexmarkSuite.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkSuite.java @@ -15,7 +15,6 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.beam.integration.nexmark; import java.util.ArrayList; @@ -33,7 +32,7 @@ public enum NexmarkSuite { DEFAULT(defaultConf()), /** - * Sweep through all 11 queries using the default configuration. + * Sweep through all queries using the default configuration. * 100k/10k events (depending on query). */ SMOKE(smoke()), @@ -50,7 +49,8 @@ public enum NexmarkSuite { private static List defaultConf() { List configurations = new ArrayList<>(); - configurations.add(new NexmarkConfiguration()); + NexmarkConfiguration configuration = new NexmarkConfiguration(); + configurations.add(configuration); return configurations; } diff --git a/integration/java/src/main/java/org/apache/beam/integration/nexmark/NexmarkUtils.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkUtils.java similarity index 85% rename from integration/java/src/main/java/org/apache/beam/integration/nexmark/NexmarkUtils.java rename to integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkUtils.java index 13ed5808617d5..6588f8577ac60 100644 --- a/integration/java/src/main/java/org/apache/beam/integration/nexmark/NexmarkUtils.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkUtils.java @@ -15,9 +15,19 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.beam.integration.nexmark; +import com.fasterxml.jackson.databind.ObjectMapper; +import com.google.common.collect.ImmutableList; +import com.google.common.hash.Hashing; + +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.nio.charset.StandardCharsets; +import java.util.Iterator; +import java.util.List; + import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.coders.AvroCoder; import org.apache.beam.sdk.coders.ByteArrayCoder; @@ -27,47 +37,33 @@ import org.apache.beam.sdk.coders.CustomCoder; import org.apache.beam.sdk.coders.SerializableCoder; import org.apache.beam.sdk.io.Read; -import org.apache.beam.sdk.runners.DirectPipelineRunner; import org.apache.beam.sdk.runners.PipelineRunner; 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.PTransform; import org.apache.beam.sdk.transforms.ParDo; -import org.apache.beam.sdk.transforms.Sum.SumLongFn; +import org.apache.beam.sdk.transforms.Sum; import org.apache.beam.sdk.transforms.windowing.AfterPane; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.GlobalWindows; import org.apache.beam.sdk.transforms.windowing.Window; -import org.apache.beam.sdk.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.StateSpec; +import org.apache.beam.sdk.util.state.StateSpecs; import org.apache.beam.sdk.util.state.ValueState; +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.TimestampedValue; -import com.google.common.collect.ImmutableList; -import com.google.common.hash.Hashing; - -import com.fasterxml.jackson.databind.ObjectMapper; - import org.joda.time.Duration; import org.joda.time.Instant; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.io.IOException; -import java.io.InputStream; -import java.io.OutputStream; -import java.nio.charset.StandardCharsets; -import java.util.Iterator; -import java.util.List; - /** * Odd's 'n Ends used throughout queries and driver. */ public class NexmarkUtils { - private static final Logger LOG = LoggerFactory.getLogger(NexmarkGoogleDriver.class.getName()); + private static final Logger LOG = LoggerFactory.getLogger(NexmarkUtils.class); /** * Mapper for (de)serializing JSON. @@ -197,7 +193,7 @@ public long rateToPeriodUs(long rate) { /** * Shape of event rate. */ - public static enum RateShape { + public enum RateShape { SQUARE, SINE; @@ -295,18 +291,6 @@ public static void info(String format, Object... args) { } } - /** - * Log error message. - */ - public static void error(String format, Object... args) { - if (LOG_ERROR) { - LOG.error(String.format(format, args)); - if (LOG_TO_CONSOLE) { - System.out.println(String.format(format, args)); - } - } - } - /** * Log message to console. For client side only. */ @@ -342,11 +326,12 @@ public static void console(String format, Object... args) { */ public static void setupPipeline(CoderStrategy coderStrategy, Pipeline p) { PipelineRunner runner = p.getRunner(); - if (runner instanceof DirectPipelineRunner) { - // Disable randomization of output since we want to check batch and streaming match the - // model both locally and on the cloud. - ((DirectPipelineRunner) runner).withUnorderednessTesting(false); - } + //TODO Ismael check +// if (runner instanceof DirectRunner) { +// // Disable randomization of output since we want to check batch and streaming match the +// // model both locally and on the cloud. +// ((DirectRunner) runner).withUnorderednessTesting(false); +// } CoderRegistry registry = p.getCoderRegistry(); switch (coderStrategy) { @@ -396,45 +381,41 @@ public static Iterator> standardEventIterator( * Return a transform which yields a finite number of synthesized events generated * as a batch. */ - public static PTransform> batchEventsSource( - String name, NexmarkConfiguration configuration) { - return Read - .from(new BoundedEventSource( - NexmarkUtils.standardGeneratorConfig(configuration), configuration.numEventGenerators)) - .named(name + ".ReadBounded"); + public static PTransform> batchEventsSource( + NexmarkConfiguration configuration) { + return Read.from(new BoundedEventSource( + NexmarkUtils.standardGeneratorConfig(configuration), configuration.numEventGenerators)); } /** * Return a transform which yields a finite number of synthesized events generated * on-the-fly in real time. */ - public static PTransform> streamEventsSource( - String name, NexmarkConfiguration configuration) { + public static PTransform> streamEventsSource( + NexmarkConfiguration configuration) { return Read.from(new UnboundedEventSource(NexmarkUtils.standardGeneratorConfig(configuration), configuration.numEventGenerators, configuration.watermarkHoldbackSec, - configuration.isRateLimited)) - .named(name + ".ReadUnbounded"); + configuration.isRateLimited)); } /** * Return a transform to pass-through events, but count them as they go by. */ public static ParDo.Bound snoop(final String name) { - return ParDo.named(name + ".Snoop") - .of(new DoFn() { + return ParDo.of(new DoFn() { final Aggregator eventCounter = - createAggregator("events", new SumLongFn()); + createAggregator("events", Sum.ofLongs()); final Aggregator newPersonCounter = - createAggregator("newPersons", new SumLongFn()); + createAggregator("newPersons", Sum.ofLongs()); final Aggregator newAuctionCounter = - createAggregator("newAuctions", new SumLongFn()); + createAggregator("newAuctions", Sum.ofLongs()); final Aggregator bidCounter = - createAggregator("bids", new SumLongFn()); + createAggregator("bids", Sum.ofLongs()); final Aggregator endOfStreamCounter = - createAggregator("endOfStream", new SumLongFn()); + createAggregator("endOfStream", Sum.ofLongs()); - @Override + @ProcessElement public void processElement(ProcessContext c) { eventCounter.addValue(1L); if (c.element().newPerson != null) { @@ -456,12 +437,11 @@ public void processElement(ProcessContext c) { * Return a transform to count and discard each element. */ public static ParDo.Bound devNull(String name) { - return ParDo.named(name + ".DevNull") - .of(new DoFn() { + return ParDo.of(new DoFn() { final Aggregator discardCounter = - createAggregator("discarded", new SumLongFn()); + createAggregator("discarded", Sum.ofLongs()); - @Override + @ProcessElement public void processElement(ProcessContext c) { discardCounter.addValue(1L); } @@ -472,11 +452,10 @@ public void processElement(ProcessContext c) { * Return a transform to log each element, passing it through unchanged. */ public static ParDo.Bound log(final String name) { - return ParDo.named(name + ".Log") - .of(new DoFn() { - @Override + return ParDo.of(new DoFn() { + @ProcessElement public void processElement(ProcessContext c) { - error("%s: %s", name, c.element()); + LOG.info("%s: %s", name, c.element()); c.output(c.element()); } }); @@ -486,12 +465,11 @@ public void processElement(ProcessContext c) { * Return a transform to format each element as a string. */ public static ParDo.Bound format(String name) { - return ParDo.named(name + ".Format") - .of(new DoFn() { + return ParDo.of(new DoFn() { final Aggregator recordCounter = - createAggregator("records", new SumLongFn()); + createAggregator("records", Sum.ofLongs()); - @Override + @ProcessElement public void processElement(ProcessContext c) { recordCounter.addValue(1L); c.output(c.element().toString()); @@ -503,9 +481,8 @@ public void processElement(ProcessContext c) { * Return a transform to make explicit the timestamp of each element. */ public static ParDo.Bound> stamp(String name) { - return ParDo.named(name + ".Stamp") - .of(new DoFn>() { - @Override + return ParDo.of(new DoFn>() { + @ProcessElement public void processElement(ProcessContext c) { c.output(TimestampedValue.of(c.element(), c.timestamp())); } @@ -519,14 +496,14 @@ public static PTransform, PCollection> hash( final long numEvents, String name) { return new PTransform, PCollection>(name) { @Override - public PCollection apply(PCollection input) { + public PCollection expand(PCollection input) { return input.apply(Window.into(new GlobalWindows()) .triggering(AfterPane.elementCountAtLeast((int) numEvents)) .withAllowedLateness(Duration.standardDays(1)) .discardingFiredPanes()) - .apply(ParDo.named(name + ".Hash").of(new DoFn() { - @Override + .apply(name + ".Hash", ParDo.of(new DoFn() { + @ProcessElement public void processElement(ProcessContext c) { long hash = Hashing.murmur3_128() @@ -557,9 +534,8 @@ public Long apply(Long left, Long right) { * Return a transform to keep the CPU busy for given milliseconds on every record. */ public static ParDo.Bound cpuDelay(String name, final long delayMs) { - return ParDo.named(name + ".CpuDelay") - .of(new DoFn() { - @Override + return ParDo.of(new DoFn() { + @ProcessElement public void processElement(ProcessContext c) { long now = System.currentTimeMillis(); long end = now + delayMs; @@ -582,17 +558,16 @@ public void processElement(ProcessContext c) { }); } - private static final StateTag> DUMMY_TAG = - StateTags.value("dummy", ByteArrayCoder.of()); + private static final StateSpec> DUMMY_TAG = + StateSpecs.value(ByteArrayCoder.of()); private static final int MAX_BUFFER_SIZE = 1 << 24; /** * Return a transform to write given number of bytes to durable store on every record. */ public static ParDo.Bound diskBusy(String name, final long bytes) { - return ParDo.named(name + ".DiskBusy") - .of(new DoFn() { - @Override + return ParDo.of(new DoFn() { + @ProcessElement public void processElement(ProcessContext c) { long remain = bytes; long start = System.currentTimeMillis(); @@ -604,9 +579,10 @@ public void processElement(ProcessContext c) { for (int i = 0; i < thisBytes; i++) { arr[i] = (byte) now; } - ValueState state = c.windowingInternals().stateInternals().state( - StateNamespaces.global(), DUMMY_TAG); - state.write(arr); + //TODO Ismael google on state +// ValueState state = c.windowingInternals().stateInternals().state( +// StateNamespaces.global(), DUMMY_TAG); +// state.write(arr); now = System.currentTimeMillis(); } c.output(c.element()); @@ -617,11 +593,9 @@ public void processElement(ProcessContext c) { /** * Return a transform to cast each element to {@link KnownSize}. */ - private static ParDo.Bound castToKnownSize( - final String name) { - return ParDo.named(name + ".Forget") - .of(new DoFn() { - @Override + private static ParDo.Bound castToKnownSize() { + return ParDo.of(new DoFn() { + @ProcessElement public void processElement(ProcessContext c) { c.output(c.element()); } @@ -672,7 +646,8 @@ private static Coder makeCastingCoder(Coder */ public static PCollection castToKnownSize( final String name, PCollection elements) { - return elements.apply(castToKnownSize(name)).setCoder(makeCastingCoder(elements.getCoder())); + return elements.apply(name + ".Forget", castToKnownSize()) + .setCoder(makeCastingCoder(elements.getCoder())); } // Do not instantiate. diff --git a/integration/java/src/main/java/org/apache/beam/integration/nexmark/Options.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Options.java similarity index 100% rename from integration/java/src/main/java/org/apache/beam/integration/nexmark/Options.java rename to integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Options.java index 4f5304d5a1599..efaf6988d3b94 100644 --- a/integration/java/src/main/java/org/apache/beam/integration/nexmark/Options.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Options.java @@ -15,13 +15,13 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.beam.integration.nexmark; +import javax.annotation.Nullable; + import org.apache.beam.sdk.options.Default; import org.apache.beam.sdk.options.Description; import org.apache.beam.sdk.options.PubsubOptions; -import javax.annotation.Nullable; /** * Command line flags. diff --git a/integration/java/src/main/java/org/apache/beam/integration/nexmark/Person.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Person.java similarity index 99% rename from integration/java/src/main/java/org/apache/beam/integration/nexmark/Person.java rename to integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Person.java index 6fcf388b72724..251a6eeb85022 100644 --- a/integration/java/src/main/java/org/apache/beam/integration/nexmark/Person.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Person.java @@ -15,15 +15,8 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.beam.integration.nexmark; -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.StringUtf8Coder; -import org.apache.beam.sdk.coders.VarLongCoder; - import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.core.JsonProcessingException; @@ -32,6 +25,12 @@ import java.io.OutputStream; import java.io.Serializable; +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.StringUtf8Coder; +import org.apache.beam.sdk.coders.VarLongCoder; + /** * A person either creating an auction or making a bid. */ diff --git a/integration/java/src/main/java/org/apache/beam/integration/nexmark/PubsubHelper.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/PubsubHelper.java similarity index 99% rename from integration/java/src/main/java/org/apache/beam/integration/nexmark/PubsubHelper.java rename to integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/PubsubHelper.java index 1255154dc7676..a79a25b9535f5 100644 --- a/integration/java/src/main/java/org/apache/beam/integration/nexmark/PubsubHelper.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/PubsubHelper.java @@ -15,18 +15,17 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.beam.integration.nexmark; -import org.apache.beam.sdk.options.PubsubOptions; -import org.apache.beam.sdk.util.PubsubClient; -import org.apache.beam.sdk.util.PubsubJsonClient; - import java.io.IOException; import java.util.ArrayList; import java.util.Collection; import java.util.List; +import org.apache.beam.sdk.options.PubsubOptions; +import org.apache.beam.sdk.util.PubsubClient; +import org.apache.beam.sdk.util.PubsubJsonClient; + /** * Helper for working with pubsub. */ diff --git a/integration/java/src/main/java/org/apache/beam/integration/nexmark/Query0.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query0.java similarity index 90% rename from integration/java/src/main/java/org/apache/beam/integration/nexmark/Query0.java rename to integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query0.java index ea0d7ca200668..e88fce0f8a1ca 100644 --- a/integration/java/src/main/java/org/apache/beam/integration/nexmark/Query0.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query0.java @@ -15,22 +15,20 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.beam.integration.nexmark; +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.IOException; + import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.CoderException; 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.SumLongFn; +import org.apache.beam.sdk.transforms.Sum; import org.apache.beam.sdk.values.PCollection; - -import java.io.ByteArrayInputStream; -import java.io.ByteArrayOutputStream; -import java.io.IOException; - /** * Query 0: Pass events through unchanged. However, force them to do a round trip through * serialization so that we measure the impact of the choice of coders. @@ -42,17 +40,14 @@ public Query0(NexmarkConfiguration configuration) { private PCollection applyTyped(PCollection events) { final Coder coder = events.getCoder(); - return events - // Force round trip through coder. - .apply( - ParDo.named(name + ".Serialize") - .of(new DoFn() { + .apply(name + ".Serialize", + ParDo.of(new DoFn() { private final Aggregator bytes = - createAggregator("bytes", new SumLongFn()); + createAggregator("bytes", Sum.ofLongs()); - @Override + @ProcessElement public void processElement(ProcessContext c) throws CoderException, IOException { ByteArrayOutputStream outStream = new ByteArrayOutputStream(); coder.encode(c.element(), outStream, Coder.Context.OUTER); diff --git a/integration/java/src/main/java/org/apache/beam/integration/nexmark/Query0Model.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query0Model.java similarity index 99% rename from integration/java/src/main/java/org/apache/beam/integration/nexmark/Query0Model.java rename to integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query0Model.java index f3ceca267a05c..b7cdf1cf861a5 100644 --- a/integration/java/src/main/java/org/apache/beam/integration/nexmark/Query0Model.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query0Model.java @@ -15,14 +15,13 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.beam.integration.nexmark; -import org.apache.beam.sdk.values.TimestampedValue; - import java.util.Collection; import java.util.Iterator; +import org.apache.beam.sdk.values.TimestampedValue; + /** * A direct implementation of {@link Query0}. */ diff --git a/integration/java/src/main/java/org/apache/beam/integration/nexmark/Query1.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query1.java similarity index 94% rename from integration/java/src/main/java/org/apache/beam/integration/nexmark/Query1.java rename to integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query1.java index 7e60b9c783dbb..a1ecdeb7ea1ca 100644 --- a/integration/java/src/main/java/org/apache/beam/integration/nexmark/Query1.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query1.java @@ -15,7 +15,6 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.beam.integration.nexmark; import org.apache.beam.sdk.transforms.DoFn; @@ -45,10 +44,9 @@ private PCollection applyTyped(PCollection events) { .apply(JUST_BIDS) // Map the conversion function over all bids. - .apply( - ParDo.named(name + ".ToEuros") - .of(new DoFn() { - @Override + .apply(name + ".ToEuros", + ParDo.of(new DoFn() { + @ProcessElement public void processElement(ProcessContext c) { Bid bid = c.element(); c.output(new Bid( diff --git a/integration/java/src/main/java/org/apache/beam/integration/nexmark/Query10.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query10.java similarity index 82% rename from integration/java/src/main/java/org/apache/beam/integration/nexmark/Query10.java rename to integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query10.java index 74fb28c3cc775..76f93e5ed9df3 100644 --- a/integration/java/src/main/java/org/apache/beam/integration/nexmark/Query10.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query10.java @@ -15,17 +15,27 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.beam.integration.nexmark; +import static com.google.common.base.Preconditions.checkState; + +import com.google.cloud.hadoop.gcsio.GoogleCloudStorageWriteChannel; + +import java.io.IOException; +import java.io.OutputStream; +import java.io.Serializable; +import java.nio.channels.Channels; +import java.nio.channels.WritableByteChannel; +import java.util.concurrent.ThreadLocalRandom; +import javax.annotation.Nullable; + import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.options.GcsOptions; import org.apache.beam.sdk.transforms.Aggregator; import org.apache.beam.sdk.transforms.DoFn; -import org.apache.beam.sdk.transforms.DoFnWithContext; import org.apache.beam.sdk.transforms.GroupByKey; import org.apache.beam.sdk.transforms.ParDo; -import org.apache.beam.sdk.transforms.Sum.SumLongFn; +import org.apache.beam.sdk.transforms.Sum; import org.apache.beam.sdk.transforms.windowing.AfterEach; import org.apache.beam.sdk.transforms.windowing.AfterFirst; import org.apache.beam.sdk.transforms.windowing.AfterPane; @@ -40,20 +50,11 @@ import org.apache.beam.sdk.util.GcsIOChannelFactory; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; - -import com.google.cloud.hadoop.gcsio.GoogleCloudStorageWriteChannel; -import com.google.common.base.Preconditions; - import org.joda.time.Duration; import org.joda.time.Instant; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; -import java.io.IOException; -import java.io.OutputStream; -import java.io.Serializable; -import java.nio.channels.Channels; -import java.nio.channels.WritableByteChannel; -import java.util.concurrent.ThreadLocalRandom; -import javax.annotation.Nullable; /** * Query "10", 'Log to sharded files' (Not in original suite.) @@ -61,6 +62,7 @@ *

              Every windowSizeSec, save all events from the last period into 2*maxWorkers log files. */ class Query10 extends NexmarkQuery { + private static final Logger LOG = LoggerFactory.getLogger(Query10.class); private static final int CHANNEL_BUFFER = 8 << 20; // 8MB private static final int NUM_SHARDS_PER_WORKER = 5; private static final Duration LATE_BATCHING_PERIOD = Duration.standardSeconds(10); @@ -130,8 +132,9 @@ public void setMaxNumWorkers(int maxNumWorkers) { */ private WritableByteChannel openWritableGcsFile(GcsOptions options, String filename) throws IOException { - WritableByteChannel channel = new GcsIOChannelFactory(options).create(filename, "text/plain"); - Preconditions.checkState(channel instanceof GoogleCloudStorageWriteChannel); + WritableByteChannel channel = + GcsIOChannelFactory.fromOptions(options).create(filename, "text/plain"); + checkState(channel instanceof GoogleCloudStorageWriteChannel); ((GoogleCloudStorageWriteChannel) channel).setUploadBufferSize(CHANNEL_BUFFER); return channel; } @@ -177,19 +180,18 @@ private String indexPathFor(BoundedWindow window) { private PCollection applyTyped(PCollection events) { final int numLogShards = maxNumWorkers * NUM_SHARDS_PER_WORKER; - return events - .apply(ParDo.named(name + ".ShardEvents") - .of(new DoFn>() { + return events.apply(name + ".ShardEvents", + ParDo.of(new DoFn>() { final Aggregator lateCounter = - createAggregator("actuallyLateEvent", new SumLongFn()); + createAggregator("actuallyLateEvent", Sum.ofLongs()); final Aggregator onTimeCounter = - createAggregator("actuallyOnTimeEvent", new SumLongFn()); + createAggregator("actuallyOnTimeEvent", Sum.ofLongs()); - @Override + @ProcessElement public void processElement(ProcessContext c) { if (c.element().hasAnnotation("LATE")) { lateCounter.addValue(1L); - NexmarkUtils.error("Observed late: %s", c.element()); + LOG.error("Observed late: %s", c.element()); } else { onTimeCounter.addValue(1L); } @@ -198,9 +200,9 @@ public void processElement(ProcessContext c) { c.output(KV.of(shard, c.element())); } })) - .apply(Window.>into( + .apply(name + ".WindowEvents", + Window.>into( FixedWindows.of(Duration.standardSeconds(configuration.windowSizeSec))) - .named(name + ".WindowEvents") .triggering(AfterEach.inOrder( Repeatedly .forever(AfterPane.elementCountAtLeast(configuration.maxLogEvents)) @@ -213,21 +215,20 @@ public void processElement(ProcessContext c) { // Use a 1 day allowed lateness so that any forgotten hold will stall the // pipeline for that period and be very noticeable. .withAllowedLateness(Duration.standardDays(1))) - .apply(GroupByKey.create()) - .apply( - ParDo.named(name + ".CheckForLateEvents") - .of(new DoFnWithContext>, + .apply(name + ".GroupByKey", GroupByKey.create()) + .apply(name + ".CheckForLateEvents", + ParDo.of(new DoFn>, KV>>() { final Aggregator earlyCounter = - createAggregator("earlyShard", new SumLongFn()); + createAggregator("earlyShard", Sum.ofLongs()); final Aggregator onTimeCounter = - createAggregator("onTimeShard", new SumLongFn()); + createAggregator("onTimeShard", Sum.ofLongs()); final Aggregator lateCounter = - createAggregator("lateShard", new SumLongFn()); + createAggregator("lateShard", Sum.ofLongs()); final Aggregator unexpectedLatePaneCounter = - createAggregator("ERROR_unexpectedLatePane", new SumLongFn()); + createAggregator("ERROR_unexpectedLatePane", Sum.ofLongs()); final Aggregator unexpectedOnTimeElementCounter = - createAggregator("ERROR_unexpectedOnTimeElement", new SumLongFn()); + createAggregator("ERROR_unexpectedOnTimeElement", Sum.ofLongs()); @ProcessElement public void processElement(ProcessContext c, BoundedWindow window) { @@ -241,19 +242,19 @@ public void processElement(ProcessContext c, BoundedWindow window) { } } String shard = c.element().getKey(); - NexmarkUtils.error( + LOG.error( "%s with timestamp %s has %d actually late and %d on-time " + "elements in pane %s for window %s", shard, c.timestamp(), numLate, numOnTime, c.pane(), window.maxTimestamp()); if (c.pane().getTiming() == PaneInfo.Timing.LATE) { if (numLate == 0) { - NexmarkUtils.error( + LOG.error( "ERROR! No late events in late pane for %s", shard); unexpectedLatePaneCounter.addValue(1L); } if (numOnTime > 0) { - NexmarkUtils.error( + LOG.error( "ERROR! Have %d on-time events in late pane for %s", numOnTime, shard); unexpectedOnTimeElementCounter.addValue(1L); @@ -261,7 +262,7 @@ public void processElement(ProcessContext c, BoundedWindow window) { lateCounter.addValue(1L); } else if (c.pane().getTiming() == PaneInfo.Timing.EARLY) { if (numOnTime + numLate < configuration.maxLogEvents) { - NexmarkUtils.error( + LOG.error( "ERROR! Only have %d events in early pane for %s", numOnTime + numLate, shard); } @@ -272,25 +273,25 @@ public void processElement(ProcessContext c, BoundedWindow window) { c.output(c.element()); } })) - .apply( - ParDo.named(name + ".UploadEvents") - .of(new DoFnWithContext>, + .apply(name + ".UploadEvents", + ParDo.of(new DoFn>, KV>() { final Aggregator savedFileCounter = - createAggregator("savedFile", new SumLongFn()); + createAggregator("savedFile", Sum.ofLongs()); final Aggregator writtenRecordsCounter = - createAggregator("writtenRecords", new SumLongFn()); + createAggregator("writtenRecords", Sum.ofLongs()); @ProcessElement - public void process(ProcessContext c, BoundedWindow window) throws IOException { + public void processElement(ProcessContext c, BoundedWindow window) + throws IOException { String shard = c.element().getKey(); GcsOptions options = c.getPipelineOptions().as(GcsOptions.class); OutputFile outputFile = outputFileFor(window, shard, c.pane()); - NexmarkUtils.error( + LOG.error( "Writing %s with record timestamp %s, window timestamp %s, pane %s", shard, c.timestamp(), window.maxTimestamp(), c.pane()); if (outputFile.filename != null) { - NexmarkUtils.error("Beginning write to '%s'", outputFile.filename); + LOG.error("Beginning write to '%s'", outputFile.filename); int n = 0; try (OutputStream output = Channels.newOutputStream(openWritableGcsFile(options, outputFile @@ -299,59 +300,58 @@ public void process(ProcessContext c, BoundedWindow window) throws IOException { Event.CODER.encode(event, output, Coder.Context.OUTER); writtenRecordsCounter.addValue(1L); if (++n % 10000 == 0) { - NexmarkUtils.error("So far written %d records to '%s'", n, + LOG.error("So far written %d records to '%s'", n, outputFile.filename); } } } - NexmarkUtils.error("Written all %d records to '%s'", n, outputFile.filename); + LOG.error("Written all %d records to '%s'", n, outputFile.filename); } savedFileCounter.addValue(1L); c.output(KV.of(null, outputFile)); } })) // Clear fancy triggering from above. - .apply(Window.>into( + .apply(name + ".WindowLogFiles", Window.>into( FixedWindows.of(Duration.standardSeconds(configuration.windowSizeSec))) - .named(name + ".WindowLogFiles") .triggering(AfterWatermark.pastEndOfWindow()) // We expect no late data here, but we'll assume the worst so we can detect any. .withAllowedLateness(Duration.standardDays(1)) .discardingFiredPanes()) - .apply(GroupByKey.create()) - .apply( - ParDo.named(name + ".Index") - .of(new DoFnWithContext>, Done>() { + .apply(name + ".GroupByKey2", GroupByKey.create()) + .apply(name + ".Index", + ParDo.of(new DoFn>, Done>() { final Aggregator unexpectedLateCounter = - createAggregator("ERROR_unexpectedLate", new SumLongFn()); + createAggregator("ERROR_unexpectedLate", Sum.ofLongs()); final Aggregator unexpectedEarlyCounter = - createAggregator("ERROR_unexpectedEarly", new SumLongFn()); + createAggregator("ERROR_unexpectedEarly", Sum.ofLongs()); final Aggregator unexpectedIndexCounter = - createAggregator("ERROR_unexpectedIndex", new SumLongFn()); + createAggregator("ERROR_unexpectedIndex", Sum.ofLongs()); final Aggregator finalizedCounter = - createAggregator("indexed", new SumLongFn()); + createAggregator("indexed", Sum.ofLongs()); @ProcessElement - public void process(ProcessContext c, BoundedWindow window) throws IOException { + public void processElement(ProcessContext c, BoundedWindow window) + throws IOException { if (c.pane().getTiming() == Timing.LATE) { unexpectedLateCounter.addValue(1L); - NexmarkUtils.error("ERROR! Unexpected LATE pane: %s", c.pane()); + LOG.error("ERROR! Unexpected LATE pane: %s", c.pane()); } else if (c.pane().getTiming() == Timing.EARLY) { unexpectedEarlyCounter.addValue(1L); - NexmarkUtils.error("ERROR! Unexpected EARLY pane: %s", c.pane()); + LOG.error("ERROR! Unexpected EARLY pane: %s", c.pane()); } else if (c.pane().getTiming() == Timing.ON_TIME && c.pane().getIndex() != 0) { unexpectedIndexCounter.addValue(1L); - NexmarkUtils.error("ERROR! Unexpected ON_TIME pane index: %s", c.pane()); + LOG.error("ERROR! Unexpected ON_TIME pane index: %s", c.pane()); } else { GcsOptions options = c.getPipelineOptions().as(GcsOptions.class); - NexmarkUtils.error( + LOG.error( "Index with record timestamp %s, window timestamp %s, pane %s", c.timestamp(), window.maxTimestamp(), c.pane()); @Nullable String filename = indexPathFor(window); if (filename != null) { - NexmarkUtils.error("Beginning write to '%s'", filename); + LOG.error("Beginning write to '%s'", filename); int n = 0; try (OutputStream output = Channels.newOutputStream( @@ -361,7 +361,7 @@ public void process(ProcessContext c, BoundedWindow window) throws IOException { n++; } } - NexmarkUtils.error("Written all %d lines to '%s'", n, filename); + LOG.error("Written all %d lines to '%s'", n, filename); } c.output( new Done("written for timestamp " + window.maxTimestamp())); diff --git a/integration/java/src/main/java/org/apache/beam/integration/nexmark/Query11.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query11.java similarity index 90% rename from integration/java/src/main/java/org/apache/beam/integration/nexmark/Query11.java rename to integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query11.java index 9841421ee15e7..c37c1cde1216c 100644 --- a/integration/java/src/main/java/org/apache/beam/integration/nexmark/Query11.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query11.java @@ -15,7 +15,6 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.beam.integration.nexmark; import org.apache.beam.sdk.transforms.Count; @@ -27,7 +26,6 @@ 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; /** @@ -44,10 +42,9 @@ public Query11(NexmarkConfiguration configuration) { private PCollection applyTyped(PCollection events) { return events.apply(JUST_BIDS) - .apply( - ParDo.named(name + ".Rekey") - .of(new DoFn>() { - @Override + .apply(name + ".Rekey", + ParDo.of(new DoFn>() { + @ProcessElement public void processElement(ProcessContext c) { Bid bid = c.element(); c.output(KV.of(bid.bidder, (Void) null)); @@ -59,10 +56,9 @@ public void processElement(ProcessContext c) { .discardingFiredPanes() .withAllowedLateness(Duration.standardSeconds(configuration.occasionalDelaySec / 2))) .apply(Count.perKey()) - .apply( - ParDo.named(name + ".ToResult") - .of(new DoFn, BidsPerSession>() { - @Override + .apply(name + ".ToResult", + ParDo.of(new DoFn, BidsPerSession>() { + @ProcessElement public void processElement(ProcessContext c) { c.output(new BidsPerSession(c.element().getKey(), c.element().getValue())); } diff --git a/integration/java/src/main/java/org/apache/beam/integration/nexmark/Query12.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query12.java similarity index 90% rename from integration/java/src/main/java/org/apache/beam/integration/nexmark/Query12.java rename to integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query12.java index dd39971654fb9..ee754c1769a70 100644 --- a/integration/java/src/main/java/org/apache/beam/integration/nexmark/Query12.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query12.java @@ -15,7 +15,6 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.beam.integration.nexmark; import org.apache.beam.sdk.transforms.Count; @@ -31,7 +30,7 @@ /** * Query "12", 'Processing time windows' (Not in original suite.) - *

              + * *

              Group bids by the same user into processing time windows of windowSize. Emit the count * of bids per window. */ @@ -43,10 +42,9 @@ public Query12(NexmarkConfiguration configuration) { private PCollection applyTyped(PCollection events) { return events .apply(JUST_BIDS) - .apply( - ParDo.named(name + ".Rekey") - .of(new DoFn>() { - @Override + .apply(name + ".Rekey", + ParDo.of(new DoFn>() { + @ProcessElement public void processElement(ProcessContext c) { Bid bid = c.element(); c.output(KV.of(bid.bidder, (Void) null)); @@ -61,10 +59,9 @@ public void processElement(ProcessContext c) { .discardingFiredPanes() .withAllowedLateness(Duration.ZERO)) .apply(Count.perKey()) - .apply( - ParDo.named(name + ".ToResult") - .of(new DoFn, BidsPerSession>() { - @Override + .apply(name + ".ToResult", + ParDo.of(new DoFn, BidsPerSession>() { + @ProcessElement public void processElement(ProcessContext c) { c.output( new BidsPerSession(c.element().getKey(), c.element().getValue())); diff --git a/integration/java/src/main/java/org/apache/beam/integration/nexmark/Query1Model.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query1Model.java similarity index 99% rename from integration/java/src/main/java/org/apache/beam/integration/nexmark/Query1Model.java rename to integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query1Model.java index 462d4262237a4..ace6f7ead6866 100644 --- a/integration/java/src/main/java/org/apache/beam/integration/nexmark/Query1Model.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query1Model.java @@ -15,15 +15,14 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.beam.integration.nexmark; -import org.apache.beam.sdk.values.TimestampedValue; - import java.io.Serializable; import java.util.Collection; import java.util.Iterator; +import org.apache.beam.sdk.values.TimestampedValue; + /** * A direct implementation of {@link Query1}. */ diff --git a/integration/java/src/main/java/org/apache/beam/integration/nexmark/Query2.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query2.java similarity index 92% rename from integration/java/src/main/java/org/apache/beam/integration/nexmark/Query2.java rename to integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query2.java index cede2f36eb7c0..828cdf5f91f41 100644 --- a/integration/java/src/main/java/org/apache/beam/integration/nexmark/Query2.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query2.java @@ -15,7 +15,6 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.beam.integration.nexmark; import org.apache.beam.sdk.transforms.DoFn; @@ -49,7 +48,7 @@ private PCollection applyTyped(PCollection events) { .apply(JUST_BIDS) // Select just the bids for the auctions we care about. - .apply(Filter.byPredicate(new SerializableFunction() { + .apply(Filter.by(new SerializableFunction() { @Override public Boolean apply(Bid bid) { return bid.auction % configuration.auctionSkip == 0; @@ -57,10 +56,9 @@ public Boolean apply(Bid bid) { })) // Project just auction id and price. - .apply( - ParDo.named(name + ".Project") - .of(new DoFn() { - @Override + .apply(name + ".Project", + ParDo.of(new DoFn() { + @ProcessElement public void processElement(ProcessContext c) { Bid bid = c.element(); c.output(new AuctionPrice(bid.auction, bid.price)); diff --git a/integration/java/src/main/java/org/apache/beam/integration/nexmark/Query2Model.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query2Model.java similarity index 99% rename from integration/java/src/main/java/org/apache/beam/integration/nexmark/Query2Model.java rename to integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query2Model.java index 6ccfeeb9d7ba3..7769e521e651e 100644 --- a/integration/java/src/main/java/org/apache/beam/integration/nexmark/Query2Model.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query2Model.java @@ -15,15 +15,14 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.beam.integration.nexmark; -import org.apache.beam.sdk.values.TimestampedValue; - import java.io.Serializable; import java.util.Collection; import java.util.Iterator; +import org.apache.beam.sdk.values.TimestampedValue; + /** * A direct implementation of {@link Query2}. */ diff --git a/integration/java/src/main/java/org/apache/beam/integration/nexmark/Query3.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query3.java similarity index 76% rename from integration/java/src/main/java/org/apache/beam/integration/nexmark/Query3.java rename to integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query3.java index 5b9b17bc6a794..24e112eaea875 100644 --- a/integration/java/src/main/java/org/apache/beam/integration/nexmark/Query3.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query3.java @@ -15,37 +15,33 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.beam.integration.nexmark; +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import javax.annotation.Nullable; + import org.apache.beam.sdk.coders.ListCoder; import org.apache.beam.sdk.transforms.Aggregator; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.Filter; import org.apache.beam.sdk.transforms.ParDo; import org.apache.beam.sdk.transforms.SerializableFunction; -import org.apache.beam.sdk.transforms.Sum.SumLongFn; +import org.apache.beam.sdk.transforms.Sum; import org.apache.beam.sdk.transforms.join.CoGbkResult; import org.apache.beam.sdk.transforms.join.CoGroupByKey; import org.apache.beam.sdk.transforms.join.KeyedPCollectionTuple; import org.apache.beam.sdk.transforms.windowing.FixedWindows; import org.apache.beam.sdk.transforms.windowing.Window; -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.StateTag; -import org.apache.beam.sdk.util.state.StateTags; +import org.apache.beam.sdk.util.state.StateSpec; +import org.apache.beam.sdk.util.state.StateSpecs; import org.apache.beam.sdk.util.state.ValueState; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; - import org.joda.time.Duration; - -import java.io.IOException; -import java.util.ArrayList; -import java.util.List; - -import javax.annotation.Nullable; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * Query 3, 'Local Item Suggestion'. Who is selling in OR, ID or CA in category 10, and for what @@ -65,11 +61,12 @@ *

              A real system would use an external system to maintain the id-to-person association. */ class Query3 extends NexmarkQuery { - private static final StateNamespace GLOBAL_NAMESPACE = StateNamespaces.global(); - private static final StateTag>> AUCTION_LIST_CODED_TAG = - StateTags.value("left", ListCoder.of(Auction.CODER)); - private static final StateTag> PERSON_CODED_TAG = - StateTags.value("right", Person.CODER); + private static final Logger LOG = LoggerFactory.getLogger(Query3.class); +// private static final StateContext GLOBAL_NAMESPACE = StateContexts.global(); + private static final StateSpec>> AUCTION_LIST_CODED_TAG = + StateSpecs.value(ListCoder.of(Auction.CODER)); + private static final StateSpec> PERSON_CODED_TAG = + StateSpecs.value(Person.CODER); /** * Join {@code auctions} and {@code people} by person id and emit their cross-product one pair @@ -84,27 +81,29 @@ class Query3 extends NexmarkQuery { */ private static class JoinDoFn extends DoFn, KV> { private final Aggregator newAuctionCounter = - createAggregator("newAuction", new SumLongFn()); + createAggregator("newAuction", Sum.ofLongs()); private final Aggregator newPersonCounter = - createAggregator("newPerson", new SumLongFn()); + createAggregator("newPerson", Sum.ofLongs()); private final Aggregator newNewOutputCounter = - createAggregator("newNewOutput", new SumLongFn()); + createAggregator("newNewOutput", Sum.ofLongs()); private final Aggregator newOldOutputCounter = - createAggregator("newOldOutput", new SumLongFn()); + createAggregator("newOldOutput", Sum.ofLongs()); private final Aggregator oldNewOutputCounter = - createAggregator("oldNewOutput", new SumLongFn()); - public final Aggregator fatalCounter = createAggregator("fatal", new SumLongFn()); + createAggregator("oldNewOutput", Sum.ofLongs()); + public final Aggregator fatalCounter = createAggregator("fatal", Sum.ofLongs()); - @Override + @ProcessElement public void processElement(ProcessContext c) throws IOException { - // TODO: This is using the internal state API. Rework to use the + //TODO: This is using the internal state API. Rework to use the + //TODO Ismael this is broken for not access to state // We would *almost* implement this by rewindowing into the global window and // running a combiner over the result. The combiner's accumulator would be the // state we use below. However, combiners cannot emit intermediate results, thus // we need to wait for the pending ReduceFn API. - StateInternals stateInternals = c.windowingInternals().stateInternals(); - ValueState personState = stateInternals.state(GLOBAL_NAMESPACE, PERSON_CODED_TAG); - Person existingPerson = personState.read(); +// StateInternals stateInternals = c.windowingInternals().stateInternals(); +// ValueState personState = stateInternals.state(GLOBAL_NAMESPACE, PERSON_CODED_TAG); +// Person existingPerson = personState.read(); + Person existingPerson = null; if (existingPerson != null) { // We've already seen the new person event for this person id. // We can join with any new auctions on-the-fly without needing any @@ -117,17 +116,17 @@ public void processElement(ProcessContext c) throws IOException { return; } - ValueState> auctionsState = - stateInternals.state(GLOBAL_NAMESPACE, AUCTION_LIST_CODED_TAG); +// ValueState> auctionsState = +// stateInternals.state(GLOBAL_NAMESPACE, AUCTION_LIST_CODED_TAG); Person theNewPerson = null; for (Person newPerson : c.element().getValue().getAll(PERSON_TAG)) { if (theNewPerson == null) { theNewPerson = newPerson; } else { if (theNewPerson.equals(newPerson)) { - NexmarkUtils.error("**** duplicate person %s ****", theNewPerson); + LOG.error("**** duplicate person {} ****", theNewPerson); } else { - NexmarkUtils.error("**** conflicting persons %s and %s ****", theNewPerson, newPerson); + LOG.error("**** conflicting persons {} and {} ****", theNewPerson, newPerson); } fatalCounter.addValue(1L); continue; @@ -135,13 +134,13 @@ public void processElement(ProcessContext c) throws IOException { newPersonCounter.addValue(1L); // We've now seen the person for this person id so can flush any // pending auctions for the same seller id. - List pendingAuctions = auctionsState.read(); + List pendingAuctions = null; //auctionsState.read(); if (pendingAuctions != null) { for (Auction pendingAuction : pendingAuctions) { oldNewOutputCounter.addValue(1L); c.output(KV.of(pendingAuction, newPerson)); } - auctionsState.clear(); +// auctionsState.clear(); } // Also deal with any new auctions. for (Auction newAuction : c.element().getValue().getAll(AUCTION_TAG)) { @@ -150,7 +149,8 @@ public void processElement(ProcessContext c) throws IOException { c.output(KV.of(newAuction, newPerson)); } // Remember this person for any future auctions. - personState.write(newPerson); + +// personState.write(newPerson); } if (theNewPerson != null) { return; @@ -158,7 +158,7 @@ public void processElement(ProcessContext c) throws IOException { // We'll need to remember the auctions until we see the corresponding // new person event. - List pendingAuctions = auctionsState.read(); + List pendingAuctions = null; //auctionsState.read(); if (pendingAuctions == null) { pendingAuctions = new ArrayList<>(); } @@ -166,7 +166,7 @@ public void processElement(ProcessContext c) throws IOException { newAuctionCounter.addValue(1L); pendingAuctions.add(newAuction); } - auctionsState.write(pendingAuctions); +// auctionsState.write(pendingAuctions); } } @@ -193,15 +193,15 @@ private PCollection applyTyped(PCollection events) { .apply(JUST_NEW_AUCTIONS) // We only want auctions in category 10. - .apply(Filter.byPredicate(new SerializableFunction() { + .apply(name + ".InCategory", Filter.by(new SerializableFunction() { @Override public Boolean apply(Auction auction) { return auction.category == 10; } - }).named(name + ".InCategory")) + })) // Key auctions by their seller id. - .apply(AUCTION_BY_SELLER); + .apply("AuctionBySeller", AUCTION_BY_SELLER); PCollection> personsById = events @@ -209,29 +209,28 @@ public Boolean apply(Auction auction) { .apply(JUST_NEW_PERSONS) // We only want people in OR, ID, CA. - .apply(Filter.byPredicate(new SerializableFunction() { + .apply(name + ".InState", Filter.by(new SerializableFunction() { @Override public Boolean apply(Person person) { return person.state.equals("OR") || person.state.equals("ID") || person.state.equals("CA"); } - }).named(name + ".InState")) + })) // Key people by their id. - .apply(PERSON_BY_ID); + .apply("PersonById", PERSON_BY_ID); return // Join auctions and people. KeyedPCollectionTuple.of(AUCTION_TAG, auctionsBySellerId) .and(PERSON_TAG, personsById) .apply(CoGroupByKey.create()) - .apply(ParDo.named(name + ".Join").of(joinDoFn)) + .apply(name + ".Join", ParDo.of(joinDoFn)) // Project what we want. - .apply( - ParDo.named(name + ".Project") - .of(new DoFn, NameCityStateId>() { - @Override + .apply(name + ".Project", + ParDo.of(new DoFn, NameCityStateId>() { + @ProcessElement public void processElement(ProcessContext c) { Auction auction = c.element().getKey(); Person person = c.element().getValue(); diff --git a/integration/java/src/main/java/org/apache/beam/integration/nexmark/Query3Model.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query3Model.java similarity index 99% rename from integration/java/src/main/java/org/apache/beam/integration/nexmark/Query3Model.java rename to integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query3Model.java index b865eda613374..85796ee95f668 100644 --- a/integration/java/src/main/java/org/apache/beam/integration/nexmark/Query3Model.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query3Model.java @@ -15,21 +15,20 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.beam.integration.nexmark; -import org.apache.beam.sdk.values.TimestampedValue; import com.google.common.collect.ArrayListMultimap; import com.google.common.collect.Multimap; -import org.joda.time.Instant; - import java.io.Serializable; import java.util.Collection; import java.util.HashMap; import java.util.Iterator; import java.util.Map; +import org.apache.beam.sdk.values.TimestampedValue; +import org.joda.time.Instant; + /** * A direct implementation of {@link Query3}. */ diff --git a/integration/java/src/main/java/org/apache/beam/integration/nexmark/Query4.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query4.java similarity index 91% rename from integration/java/src/main/java/org/apache/beam/integration/nexmark/Query4.java rename to integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query4.java index bc695b7fafb3e..b24410d0a7e46 100644 --- a/integration/java/src/main/java/org/apache/beam/integration/nexmark/Query4.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query4.java @@ -15,7 +15,6 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.beam.integration.nexmark; import org.apache.beam.sdk.transforms.DoFn; @@ -25,14 +24,13 @@ 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; /** * Query 4, 'Average Price for a Category'. Select the average of the wining bid prices for all * closed auctions in each category. In CQL syntax: * - *

              + * 
              {@code
                * SELECT Istream(AVG(Q.final))
                * FROM Category C, (SELECT Rstream(MAX(B.price) AS final, A.category)
                *                   FROM Auction A [ROWS UNBOUNDED], Bid B [ROWS UNBOUNDED]
              @@ -40,7 +38,7 @@
                *                   GROUP BY A.id, A.category) Q
                * WHERE Q.category = C.id
                * GROUP BY C.id;
              - * 
              + * }
              * *

              For extra spiciness our implementation differs slightly from the above: *

                @@ -67,14 +65,14 @@ private PCollection applyTyped(PCollection events) { .apply(new WinningBids(name + ".WinningBids", configuration)); // Monitor winning bids - winningBids = winningBids.apply(winningBidsMonitor.getTransform()); + winningBids = winningBids.apply(name + ".WinningBidsMonitor", + winningBidsMonitor.getTransform()); return winningBids // Key the winning bid price by the auction category. - .apply( - ParDo.named(name + ".Rekey") - .of(new DoFn>() { - @Override + .apply(name + ".Rekey", + ParDo.of(new DoFn>() { + @ProcessElement public void processElement(ProcessContext c) { Auction auction = c.element().auction; Bid bid = c.element().bid; @@ -92,10 +90,9 @@ public void processElement(ProcessContext c) { .apply(Mean.perKey().withHotKeyFanout(configuration.fanout)) // For testing against Query4Model, capture which results are 'final'. - .apply( - ParDo.named(name + ".Project") - .of(new DoFn, CategoryPrice>() { - @Override + .apply(name + ".Project", + ParDo.of(new DoFn, CategoryPrice>() { + @ProcessElement public void processElement(ProcessContext c) { c.output(new CategoryPrice(c.element().getKey(), Math.round(c.element().getValue()), c.pane().isLast())); diff --git a/integration/java/src/main/java/org/apache/beam/integration/nexmark/Query4Model.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query4Model.java similarity index 99% rename from integration/java/src/main/java/org/apache/beam/integration/nexmark/Query4Model.java rename to integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query4Model.java index 24103067020de..afab7e86307cd 100644 --- a/integration/java/src/main/java/org/apache/beam/integration/nexmark/Query4Model.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query4Model.java @@ -15,16 +15,8 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.beam.integration.nexmark; -import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.values.TimestampedValue; - -import org.joda.time.Duration; -import org.joda.time.Instant; -import org.junit.Assert; - import java.io.Serializable; import java.util.ArrayList; import java.util.Collection; @@ -33,6 +25,12 @@ import java.util.Map; import java.util.TreeMap; +import org.apache.beam.sdk.transforms.windowing.BoundedWindow; +import org.apache.beam.sdk.values.TimestampedValue; +import org.joda.time.Duration; +import org.joda.time.Instant; +import org.junit.Assert; + /** * A direct implementation of {@link Query4}. */ diff --git a/integration/java/src/main/java/org/apache/beam/integration/nexmark/Query5.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query5.java similarity index 92% rename from integration/java/src/main/java/org/apache/beam/integration/nexmark/Query5.java rename to integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query5.java index 91a4a28c67c42..70019867ba082 100644 --- a/integration/java/src/main/java/org/apache/beam/integration/nexmark/Query5.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query5.java @@ -15,9 +15,12 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.beam.integration.nexmark; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; + import org.apache.beam.sdk.transforms.Combine; import org.apache.beam.sdk.transforms.Count; import org.apache.beam.sdk.transforms.DoFn; @@ -26,18 +29,13 @@ 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 java.util.ArrayList; -import java.util.Arrays; -import java.util.List; - /** * Query 5, 'Hot Items'. Which auctions have seen the most bids in the last hour (updated every * minute). In CQL syntax: * - *
                + * 
                {@code
                  * SELECT Rstream(auction)
                  * FROM (SELECT B1.auction, count(*) AS num
                  *       FROM Bid [RANGE 60 MINUTE SLIDE 1 MINUTE] B1
                @@ -45,7 +43,7 @@
                  * WHERE num >= ALL (SELECT count(*)
                  *                   FROM Bid [RANGE 60 MINUTE SLIDE 1 MINUTE] B2
                  *                   GROUP BY B2.auction);
                - * 
                + * }
                * *

                To make things a bit more dynamic and easier to test we use much shorter windows, and * we'll also preserve the bid counts. @@ -64,17 +62,16 @@ private PCollection applyTyped(PCollection events) { SlidingWindows.of(Duration.standardSeconds(configuration.windowSizeSec)) .every(Duration.standardSeconds(configuration.windowPeriodSec)))) // Project just the auction id. - .apply(BID_TO_AUCTION) + .apply("BidToAuction", BID_TO_AUCTION) // Count the number of bids per auction id. .apply(Count.perElement()) // We'll want to keep all auctions with the maximal number of bids. // Start by lifting each into a singleton list. - .apply( - ParDo.named(name + ".ToSingletons") - .of(new DoFn, KV, Long>>() { - @Override + .apply(name + ".ToSingletons", + ParDo.of(new DoFn, KV, Long>>() { + @ProcessElement public void processElement(ProcessContext c) { c.output(KV.of(Arrays.asList(c.element().getKey()), c.element().getValue())); } @@ -107,10 +104,9 @@ public KV, Long> apply( .withFanout(configuration.fanout)) // Project into result. - .apply( - ParDo.named(name + ".Select") - .of(new DoFn, Long>, AuctionCount>() { - @Override + .apply(name + ".Select", + ParDo.of(new DoFn, Long>, AuctionCount>() { + @ProcessElement public void processElement(ProcessContext c) { long count = c.element().getValue(); for (long auction : c.element().getKey()) { diff --git a/integration/java/src/main/java/org/apache/beam/integration/nexmark/Query5Model.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query5Model.java similarity index 99% rename from integration/java/src/main/java/org/apache/beam/integration/nexmark/Query5Model.java rename to integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query5Model.java index a7dd8f0bc8a72..f8e466e2b6597 100644 --- a/integration/java/src/main/java/org/apache/beam/integration/nexmark/Query5Model.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query5Model.java @@ -15,14 +15,8 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.beam.integration.nexmark; -import org.apache.beam.sdk.values.TimestampedValue; - -import org.joda.time.Duration; -import org.joda.time.Instant; - import java.io.Serializable; import java.util.ArrayList; import java.util.Collection; @@ -31,6 +25,10 @@ import java.util.Map; import java.util.TreeMap; +import org.apache.beam.sdk.values.TimestampedValue; +import org.joda.time.Duration; +import org.joda.time.Instant; + /** * A direct implementation of {@link Query5}. */ diff --git a/integration/java/src/main/java/org/apache/beam/integration/nexmark/Query6.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query6.java similarity index 94% rename from integration/java/src/main/java/org/apache/beam/integration/nexmark/Query6.java rename to integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query6.java index 49c0d68321c92..d5bcc301063fd 100644 --- a/integration/java/src/main/java/org/apache/beam/integration/nexmark/Query6.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query6.java @@ -15,9 +15,14 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.beam.integration.nexmark; +import com.google.common.collect.Lists; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; + import org.apache.beam.sdk.transforms.Combine; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.ParDo; @@ -27,26 +32,20 @@ import org.apache.beam.sdk.transforms.windowing.Window; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; -import com.google.common.collect.Lists; - import org.joda.time.Duration; -import java.util.ArrayList; -import java.util.Collections; -import java.util.List; - /** * Query 6, 'Average Selling Price by Seller'. Select the average selling price over the * last 10 closed auctions by the same seller. In CQL syntax: * - *

                + * 
                {@code
                  * SELECT Istream(AVG(Q.final), Q.seller)
                  * FROM (SELECT Rstream(MAX(B.price) AS final, A.seller)
                  *       FROM Auction A [ROWS UNBOUNDED], Bid B [ROWS UNBOUNDED]
                  *       WHERE A.id=B.auction AND B.datetime < A.expires AND A.expires < CURRENT_TIME
                  *       GROUP BY A.id, A.seller) [PARTITION BY A.seller ROWS 10] Q
                  * GROUP BY Q.seller;
                - * 
                + * }
                * *

                We are a little more exact with selecting winning bids: see {@link WinningBids}. */ @@ -115,10 +114,9 @@ private PCollection applyTyped(PCollection events) { .apply(new WinningBids(name + ".WinningBids", configuration)) // Key the winning bid by the seller id. - .apply( - ParDo.named(name + ".Rekey") - .of(new DoFn>() { - @Override + .apply(name + ".Rekey", + ParDo.of(new DoFn>() { + @ProcessElement public void processElement(ProcessContext c) { Auction auction = c.element().auction; Bid bid = c.element().bid; @@ -137,10 +135,9 @@ public void processElement(ProcessContext c) { .apply(Combine.perKey(new MovingMeanSellingPrice(10))) // Project into our datatype. - .apply( - ParDo.named(name + ".Select") - .of(new DoFn, SellerPrice>() { - @Override + .apply(name + ".Select", + ParDo.of(new DoFn, SellerPrice>() { + @ProcessElement public void processElement(ProcessContext c) { c.output(new SellerPrice(c.element().getKey(), c.element().getValue())); } diff --git a/integration/java/src/main/java/org/apache/beam/integration/nexmark/Query6Model.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query6Model.java similarity index 99% rename from integration/java/src/main/java/org/apache/beam/integration/nexmark/Query6Model.java rename to integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query6Model.java index 639ec9f76a1ad..d03f0fec18cce 100644 --- a/integration/java/src/main/java/org/apache/beam/integration/nexmark/Query6Model.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query6Model.java @@ -15,21 +15,19 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.beam.integration.nexmark; -import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.values.TimestampedValue; - -import org.joda.time.Instant; -import org.junit.Assert; - import java.io.Serializable; import java.util.Collection; import java.util.Iterator; import java.util.Map; import java.util.TreeMap; +import org.apache.beam.sdk.transforms.windowing.BoundedWindow; +import org.apache.beam.sdk.values.TimestampedValue; +import org.joda.time.Instant; +import org.junit.Assert; + /** * A direct implementation of {@link Query6}. */ diff --git a/integration/java/src/main/java/org/apache/beam/integration/nexmark/Query7.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query7.java similarity index 95% rename from integration/java/src/main/java/org/apache/beam/integration/nexmark/Query7.java rename to integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query7.java index 1f63b35bdca87..7c51c189250d4 100644 --- a/integration/java/src/main/java/org/apache/beam/integration/nexmark/Query7.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query7.java @@ -15,7 +15,6 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.beam.integration.nexmark; import org.apache.beam.sdk.transforms.DoFn; @@ -60,16 +59,15 @@ private PCollection applyTyped(PCollection events) { // its I/O. We'll keep this implementation since it illustrates the use of side inputs. final PCollectionView maxPriceView = slidingBids // - .apply(BID_TO_PRICE) + .apply("BidToPrice", BID_TO_PRICE) .apply(Max.longsGlobally().withFanout(configuration.fanout).asSingletonView()); return slidingBids // Select all bids which have that maximum price (there may be more than one). - .apply( - ParDo.named(name + ".Select") - .withSideInputs(maxPriceView) + .apply(name + ".Select", + ParDo.withSideInputs(maxPriceView) .of(new DoFn() { - @Override + @ProcessElement public void processElement(ProcessContext c) { long maxPrice = c.sideInput(maxPriceView); Bid bid = c.element(); diff --git a/integration/java/src/main/java/org/apache/beam/integration/nexmark/Query7Model.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query7Model.java similarity index 99% rename from integration/java/src/main/java/org/apache/beam/integration/nexmark/Query7Model.java rename to integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query7Model.java index e8351336ad5b9..73e96e24672dc 100644 --- a/integration/java/src/main/java/org/apache/beam/integration/nexmark/Query7Model.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query7Model.java @@ -15,21 +15,19 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.beam.integration.nexmark; -import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.values.TimestampedValue; - -import org.joda.time.Duration; -import org.joda.time.Instant; - import java.io.Serializable; import java.util.ArrayList; import java.util.Collection; import java.util.Iterator; import java.util.List; +import org.apache.beam.sdk.transforms.windowing.BoundedWindow; +import org.apache.beam.sdk.values.TimestampedValue; +import org.joda.time.Duration; +import org.joda.time.Instant; + /** * A direct implementation of {@link Query7}. */ diff --git a/integration/java/src/main/java/org/apache/beam/integration/nexmark/Query8.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query8.java similarity index 82% rename from integration/java/src/main/java/org/apache/beam/integration/nexmark/Query8.java rename to integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query8.java index e58453bca361a..ee5c26c8ad40c 100644 --- a/integration/java/src/main/java/org/apache/beam/integration/nexmark/Query8.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query8.java @@ -15,7 +15,6 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.beam.integration.nexmark; import org.apache.beam.sdk.transforms.DoFn; @@ -50,28 +49,28 @@ public Query8(NexmarkConfiguration configuration) { private PCollection applyTyped(PCollection events) { // Window and key new people by their id. PCollection> personsById = - events.apply(JUST_NEW_PERSONS) - .apply(Window.into( - FixedWindows.of(Duration.standardSeconds(configuration.windowSizeSec))) - .named("Query8.WindowPersons")) - .apply(PERSON_BY_ID); + events + .apply(JUST_NEW_PERSONS) + .apply("Query8.WindowPersons", + Window.into( + FixedWindows.of(Duration.standardSeconds(configuration.windowSizeSec)))) + .apply("PersonById", PERSON_BY_ID); // Window and key new auctions by their id. PCollection> auctionsBySeller = events.apply(JUST_NEW_AUCTIONS) - .apply(Window.into( - FixedWindows.of(Duration.standardSeconds(configuration.windowSizeSec))) - .named("Query8.WindowAuctions")) - .apply(AUCTION_BY_SELLER); + .apply("Query8.WindowAuctions", + Window.into( + FixedWindows.of(Duration.standardSeconds(configuration.windowSizeSec)))) + .apply("AuctionBySeller", AUCTION_BY_SELLER); // Join people and auctions and project the person id, name and auction reserve price. return KeyedPCollectionTuple.of(PERSON_TAG, personsById) .and(AUCTION_TAG, auctionsBySeller) .apply(CoGroupByKey.create()) - .apply( - ParDo.named(name + ".Select") - .of(new DoFn, IdNameReserve>() { - @Override + .apply(name + ".Select", + ParDo.of(new DoFn, IdNameReserve>() { + @ProcessElement public void processElement(ProcessContext c) { Person person = c.element().getValue().getOnly(PERSON_TAG, null); if (person == null) { diff --git a/integration/java/src/main/java/org/apache/beam/integration/nexmark/Query8Model.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query8Model.java similarity index 99% rename from integration/java/src/main/java/org/apache/beam/integration/nexmark/Query8Model.java rename to integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query8Model.java index 00f7355fe1d1f..fdd2a3522f541 100644 --- a/integration/java/src/main/java/org/apache/beam/integration/nexmark/Query8Model.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query8Model.java @@ -15,22 +15,21 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.beam.integration.nexmark; -import org.apache.beam.sdk.values.TimestampedValue; import com.google.common.collect.ArrayListMultimap; import com.google.common.collect.Multimap; -import org.joda.time.Duration; -import org.joda.time.Instant; - import java.io.Serializable; import java.util.Collection; import java.util.HashMap; import java.util.Iterator; import java.util.Map; +import org.apache.beam.sdk.values.TimestampedValue; +import org.joda.time.Duration; +import org.joda.time.Instant; + /** * A direct implementation of {@link Query8}. */ diff --git a/integration/java/src/main/java/org/apache/beam/integration/nexmark/Query9.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query9.java similarity index 99% rename from integration/java/src/main/java/org/apache/beam/integration/nexmark/Query9.java rename to integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query9.java index 2c0a5266a3912..64bf653a765c7 100644 --- a/integration/java/src/main/java/org/apache/beam/integration/nexmark/Query9.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query9.java @@ -15,7 +15,6 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.beam.integration.nexmark; import org.apache.beam.sdk.values.PCollection; diff --git a/integration/java/src/main/java/org/apache/beam/integration/nexmark/Query9Model.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query9Model.java similarity index 99% rename from integration/java/src/main/java/org/apache/beam/integration/nexmark/Query9Model.java rename to integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query9Model.java index 1fad648c954ad..338f02a5bd9c1 100644 --- a/integration/java/src/main/java/org/apache/beam/integration/nexmark/Query9Model.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query9Model.java @@ -15,15 +15,14 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.beam.integration.nexmark; -import org.apache.beam.sdk.values.TimestampedValue; - import java.io.Serializable; import java.util.Collection; import java.util.Iterator; +import org.apache.beam.sdk.values.TimestampedValue; + /** * A direct implementation of {@link Query9}. */ diff --git a/integration/java/src/main/java/org/apache/beam/integration/nexmark/SellerPrice.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/SellerPrice.java similarity index 99% rename from integration/java/src/main/java/org/apache/beam/integration/nexmark/SellerPrice.java rename to integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/SellerPrice.java index 4324b994f8c34..4081287c88b8d 100644 --- a/integration/java/src/main/java/org/apache/beam/integration/nexmark/SellerPrice.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/SellerPrice.java @@ -15,14 +15,8 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.beam.integration.nexmark; -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.VarLongCoder; - import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.core.JsonProcessingException; @@ -31,6 +25,11 @@ import java.io.OutputStream; import java.io.Serializable; +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.VarLongCoder; + /** * Result of {@link Query6}. */ diff --git a/integration/java/src/main/java/org/apache/beam/integration/nexmark/UnboundedEventSource.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/UnboundedEventSource.java similarity index 93% rename from integration/java/src/main/java/org/apache/beam/integration/nexmark/UnboundedEventSource.java rename to integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/UnboundedEventSource.java index 2898251f62c57..9573ef79fcc5a 100644 --- a/integration/java/src/main/java/org/apache/beam/integration/nexmark/UnboundedEventSource.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/UnboundedEventSource.java @@ -15,27 +15,26 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.beam.integration.nexmark; -import org.apache.beam.sdk.coders.Coder; -import org.apache.beam.sdk.io.UnboundedSource; -import org.apache.beam.sdk.options.PipelineOptions; -import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.values.TimestampedValue; - -import org.joda.time.Duration; -import org.joda.time.Instant; - import java.util.ArrayList; import java.util.List; import java.util.NoSuchElementException; import java.util.PriorityQueue; import java.util.Queue; import java.util.concurrent.ThreadLocalRandom; - import javax.annotation.Nullable; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.io.UnboundedSource; +import org.apache.beam.sdk.options.PipelineOptions; +import org.apache.beam.sdk.transforms.windowing.BoundedWindow; +import org.apache.beam.sdk.values.TimestampedValue; +import org.joda.time.Duration; +import org.joda.time.Instant; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + /** * A custom, unbounded source of event records. * @@ -45,6 +44,7 @@ */ class UnboundedEventSource extends UnboundedSource { private static final Duration BACKLOG_PERIOD = Duration.standardSeconds(30); + private static final Logger LOG = LoggerFactory.getLogger(UnboundedEventSource.class); /** Configuration for generator to use when reading synthetic events. May be split. */ private final GeneratorConfig config; @@ -130,7 +130,7 @@ public EventReader(GeneratorConfig config) { @Override public boolean start() { - NexmarkUtils.error("starting unbounded generator %s", generator); + LOG.trace("starting unbounded generator {}", generator); return advance(); } @@ -147,7 +147,7 @@ public boolean advance() { } if (watermark < BoundedWindow.TIMESTAMP_MAX_VALUE.getMillis()) { watermark = BoundedWindow.TIMESTAMP_MAX_VALUE.getMillis(); - NexmarkUtils.error("stopped unbounded generator %s", generator); + LOG.trace("stopped unbounded generator {}", generator); } return false; } @@ -156,7 +156,7 @@ public boolean advance() { if (next != null && next.wallclockTimestamp <= now) { // Time to use the held-back event. heldBackEvents.poll(); - NexmarkUtils.error("replaying held-back event %dms behind watermark", + LOG.debug("replaying held-back event {}ms behind watermark", watermark - next.eventTimestamp); } else if (generator.hasNext()) { next = generator.nextEvent(); @@ -167,7 +167,7 @@ public boolean advance() { long delayMs = ThreadLocalRandom.current().nextLong(config.configuration.occasionalDelaySec * 1000) + 1L; - NexmarkUtils.error("delaying event by %dms", delayMs); + LOG.debug("delaying event by {}ms", delayMs); heldBackEvents.add(next.withDelay(delayMs)); continue; } @@ -221,9 +221,9 @@ private void updateBacklog(long now, long newBacklogDurationMs) { pendingEvent.getTimestamp().getMillis() - timestampAtLastReportedBacklogMs; timeDialation = (double) eventTimeProgressionMs / (double) wallclockProgressionMs; } - NexmarkUtils.error( - "unbounded generator backlog now %dms (%s bytes) at %dus interEventDelay " - + "with %f time dilation", + LOG.debug( + "unbounded generator backlog now {}ms ({} bytes) at {}us interEventDelay " + + "with {} time dilation", backlogDurationMs, backlogBytes, interEventDelayUs, timeDialation); lastReportedBacklogWallclock = now; if (pendingEvent != null) { @@ -289,8 +289,7 @@ public Coder getCheckpointMarkCoder() { @Override public List generateInitialSplits( int desiredNumSplits, PipelineOptions options) { - NexmarkUtils.error( - "splitting unbounded source %s into %d sub-sources", config, numEventGenerators); + LOG.trace("splitting unbounded source into {} sub-sources", numEventGenerators); List results = new ArrayList<>(); // Ignore desiredNumSplits and use numEventGenerators instead. for (GeneratorConfig subConfig : config.split(numEventGenerators)) { @@ -303,10 +302,10 @@ public List generateInitialSplits( public EventReader createReader( PipelineOptions options, @Nullable Generator.Checkpoint checkpoint) { if (checkpoint == null) { - NexmarkUtils.error("creating initial unbounded reader for %s", config); + LOG.trace("creating initial unbounded reader for {}", config); return new EventReader(config); } else { - NexmarkUtils.error("resuming unbounded reader from %s", checkpoint); + LOG.trace("resuming unbounded reader from {}", checkpoint); return new EventReader(checkpoint.toGenerator(config)); } } diff --git a/integration/java/src/main/java/org/apache/beam/integration/nexmark/WinningBids.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/WinningBids.java similarity index 92% rename from integration/java/src/main/java/org/apache/beam/integration/nexmark/WinningBids.java rename to integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/WinningBids.java index 16f901c7a6c44..594195ac5b0e3 100644 --- a/integration/java/src/main/java/org/apache/beam/integration/nexmark/WinningBids.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/WinningBids.java @@ -15,9 +15,23 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.beam.integration.nexmark; +import static com.google.common.base.Preconditions.checkState; + +import com.fasterxml.jackson.annotation.JsonCreator; + +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.io.Serializable; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.List; +import java.util.Map; +import java.util.TreeMap; + import org.apache.beam.sdk.coders.AtomicCoder; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.CoderException; @@ -27,7 +41,7 @@ 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.SumLongFn; +import org.apache.beam.sdk.transforms.Sum; import org.apache.beam.sdk.transforms.join.CoGbkResult; import org.apache.beam.sdk.transforms.join.CoGroupByKey; import org.apache.beam.sdk.transforms.join.KeyedPCollectionTuple; @@ -37,32 +51,17 @@ import org.apache.beam.sdk.transforms.windowing.WindowFn; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; -import com.google.common.base.Preconditions; - -import com.fasterxml.jackson.annotation.JsonCreator; - import org.joda.time.Instant; -import java.io.IOException; -import java.io.InputStream; -import java.io.OutputStream; -import java.io.Serializable; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collection; -import java.util.List; -import java.util.Map; -import java.util.TreeMap; - /** * A transform to find the winning bid for each closed auction. In pseudo CQL syntax: * - *

                + * 
                {@code
                  * SELECT Rstream(A.*, B.auction, B.bidder, MAX(B.price), B.dateTime)
                  * FROM Auction A [ROWS UNBOUNDED], Bid B [ROWS UNBOUNDED]
                - * WHERE A.id=B.auction AND B.datetime < A.expires AND A.expires < CURRENT_TIME
                + * WHERE A.id = B.auction AND B.datetime < A.expires AND A.expires < CURRENT_TIME
                  * GROUP BY A.id
                - * 
                + * }
                * *

                We will also check that the winning bid is above the auction reserve. Note that * we ignore the auction opening bid value since it has no impact on which bid eventually wins, @@ -262,14 +261,12 @@ public AuctionOrBidWindow getSideInputWindow(BoundedWindow window) { * also like to make that winning results are available to following stages at the auction's * expiry. * - *

                - * Each result of the GBK will have a timestamp of the min of the result of this object's + *

                Each result of the GBK will have a timestamp of the min of the result of this object's * assignOutputTime over all records which end up in one of its iterables. Thus we get the * desired behavior if we ignore each record's timestamp and always return the auction window's * 'maxTimestamp', which will correspond to the auction's expiry. * - *

                - * In contrast, if this object's assignOutputTime were to return 'inputTimestamp' + *

                In contrast, if this object's assignOutputTime were to return 'inputTimestamp' * (the usual implementation), then each GBK record will take as its timestamp the minimum of * the timestamps of all bids and auctions within it, which will always be the auction's * timestamp. An auction which expires well into the future would thus hold up the watermark @@ -306,19 +303,20 @@ public WinningBids(String name, NexmarkConfiguration configuration) { } @Override - public PCollection apply(PCollection events) { + public PCollection expand(PCollection events) { // Window auctions and bids into custom auction windows. New people events will be discarded. // This will allow us to bring bids and auctions together irrespective of how long // each auction is open for. - events = events.apply(Window.named("Window").into(auctionOrBidWindowFn)); + events = events.apply("Window", Window.into(auctionOrBidWindowFn)); // Key auctions by their id. PCollection> auctionsById = - events.apply(NexmarkQuery.JUST_NEW_AUCTIONS).apply(NexmarkQuery.AUCTION_BY_ID); + events.apply(NexmarkQuery.JUST_NEW_AUCTIONS) + .apply("AuctionById:", NexmarkQuery.AUCTION_BY_ID); // Key bids by their auction id. PCollection> bidsByAuctionId = - events.apply(NexmarkQuery.JUST_BIDS).apply(NexmarkQuery.BID_BY_AUCTION); + events.apply(NexmarkQuery.JUST_BIDS).apply("BidByAuction", NexmarkQuery.BID_BY_AUCTION); // Find the highest price valid bid for each closed auction. return @@ -328,18 +326,17 @@ public PCollection apply(PCollection events) { .apply(CoGroupByKey.create()) // Filter and select. - .apply( - ParDo.named(name + ".Join") - .of(new DoFn, AuctionBid>() { + .apply(name + ".Join", + ParDo.of(new DoFn, AuctionBid>() { final Aggregator noAuctionCounter = - createAggregator("noAuction", new SumLongFn()); + createAggregator("noAuction", Sum.ofLongs()); final Aggregator underReserveCounter = - createAggregator("underReserve", new SumLongFn()); + createAggregator("underReserve", Sum.ofLongs()); final Aggregator noValidBidsCounter = - createAggregator("noValidBids", new SumLongFn()); + createAggregator("noValidBids", Sum.ofLongs()); - @Override + @ProcessElement public void processElement(ProcessContext c) { Auction auction = c.element().getValue().getOnly(NexmarkQuery.AUCTION_TAG, null); @@ -354,7 +351,7 @@ public void processElement(ProcessContext c) { for (Bid bid : c.element().getValue().getAll(NexmarkQuery.BID_TAG)) { // Bids too late for their auction will have been // filtered out by the window merge function. - Preconditions.checkState(bid.dateTime < auction.expires); + checkState(bid.dateTime < auction.expires); if (bid.price < auction.reserve) { // Bid price is below auction reserve. underReserveCounter.addValue(1L); diff --git a/integration/java/src/main/java/org/apache/beam/integration/nexmark/WinningBidsSimulator.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/WinningBidsSimulator.java similarity index 99% rename from integration/java/src/main/java/org/apache/beam/integration/nexmark/WinningBidsSimulator.java rename to integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/WinningBidsSimulator.java index b61aed1c220b6..59705562e55bb 100644 --- a/integration/java/src/main/java/org/apache/beam/integration/nexmark/WinningBidsSimulator.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/WinningBidsSimulator.java @@ -15,14 +15,8 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.beam.integration.nexmark; -import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.values.TimestampedValue; - -import org.joda.time.Instant; - import java.util.ArrayList; import java.util.Iterator; import java.util.List; @@ -30,9 +24,12 @@ import java.util.Set; import java.util.TreeMap; import java.util.TreeSet; - import javax.annotation.Nullable; +import org.apache.beam.sdk.transforms.windowing.BoundedWindow; +import org.apache.beam.sdk.values.TimestampedValue; +import org.joda.time.Instant; + /** * A simulator of the {@code WinningBids} query. */ diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/package-info.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/package-info.java new file mode 100644 index 0000000000000..65bf7d4878034 --- /dev/null +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/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. + */ +/** + * Nexmark Benchmark Integration Queries. + */ +package org.apache.beam.integration.nexmark; diff --git a/integration/java/nexmark/src/main/resources/log4j.properties b/integration/java/nexmark/src/main/resources/log4j.properties new file mode 100644 index 0000000000000..9d20aeadf6a62 --- /dev/null +++ b/integration/java/nexmark/src/main/resources/log4j.properties @@ -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. +# + +# Set everything to be logged to the console +log4j.rootCategory=DEBUG, console +log4j.appender.console=org.apache.log4j.ConsoleAppender +log4j.appender.console.target=System.err +log4j.appender.console.layout=org.apache.log4j.PatternLayout +log4j.appender.console.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss} %p %c: %m%n + +log4j.logger.org.apache.beam.runners.direct=WARN +log4j.logger.org.apache.beam.sdk=WARN + +# Settings to quiet third party logs that are too verbose +log4j.logger.org.spark_project.jetty=WARN +log4j.logger.org.spark_project.jetty.util.component.AbstractLifeCycle=ERROR + +# Setting to quiet spark logs, Beam logs should standout +log4j.logger.org.apache.beam.runners.spark=INFO +log4j.logger.org.apache.spark=WARN +log4j.logger.org.spark-project=WARN +log4j.logger.io.netty=INFO + +log4j.logger.org.apache.flink=WARN + +# SPARK-9183: Settings to avoid annoying messages when looking up nonexistent UDFs in SparkSQL with Hive support +log4j.logger.org.apache.hadoop.hive.metastore.RetryingHMSHandler=FATAL +log4j.logger.org.apache.hadoop.hive.ql.exec.FunctionRegistry=ERROR diff --git a/integration/java/src/test/java/org/apache/beam/integration/nexmark/BoundedEventSourceTest.java b/integration/java/nexmark/src/test/java/org/apache/beam/integration/nexmark/BoundedEventSourceTest.java similarity index 99% rename from integration/java/src/test/java/org/apache/beam/integration/nexmark/BoundedEventSourceTest.java rename to integration/java/nexmark/src/test/java/org/apache/beam/integration/nexmark/BoundedEventSourceTest.java index f0172679e463a..77957e5d50b61 100644 --- a/integration/java/src/test/java/org/apache/beam/integration/nexmark/BoundedEventSourceTest.java +++ b/integration/java/nexmark/src/test/java/org/apache/beam/integration/nexmark/BoundedEventSourceTest.java @@ -15,7 +15,6 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.beam.integration.nexmark; import org.apache.beam.runners.dataflow.options.DataflowPipelineOptions; diff --git a/integration/java/src/test/java/org/apache/beam/integration/nexmark/GeneratorTest.java b/integration/java/nexmark/src/test/java/org/apache/beam/integration/nexmark/GeneratorTest.java similarity index 99% rename from integration/java/src/test/java/org/apache/beam/integration/nexmark/GeneratorTest.java rename to integration/java/nexmark/src/test/java/org/apache/beam/integration/nexmark/GeneratorTest.java index bbaee26c25c42..4b821ea2fb1fc 100644 --- a/integration/java/src/test/java/org/apache/beam/integration/nexmark/GeneratorTest.java +++ b/integration/java/nexmark/src/test/java/org/apache/beam/integration/nexmark/GeneratorTest.java @@ -15,20 +15,19 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.beam.integration.nexmark; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.JUnit4; - import java.util.ArrayList; import java.util.Iterator; import java.util.List; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + /** * Test {@link Generator}. */ diff --git a/integration/java/src/test/java/org/apache/beam/integration/nexmark/QueryTest.java b/integration/java/nexmark/src/test/java/org/apache/beam/integration/nexmark/QueryTest.java similarity index 90% rename from integration/java/src/test/java/org/apache/beam/integration/nexmark/QueryTest.java rename to integration/java/nexmark/src/test/java/org/apache/beam/integration/nexmark/QueryTest.java index 860fa78b13a68..d4d51f17c71f5 100644 --- a/integration/java/src/test/java/org/apache/beam/integration/nexmark/QueryTest.java +++ b/integration/java/nexmark/src/test/java/org/apache/beam/integration/nexmark/QueryTest.java @@ -15,16 +15,16 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.beam.integration.nexmark; 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.PCollection.IsBounded; +//import org.apache.beam.sdk.values.PCollection.IsBounded; import org.apache.beam.sdk.values.TimestampedValue; +import org.junit.Ignore; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; @@ -33,6 +33,8 @@ * Test the various NEXMark queries yield results coherent with their models. */ @RunWith(JUnit4.class) +@Ignore +//TODO Ismael public class QueryTest { private static final NexmarkConfiguration CONFIG = NexmarkConfiguration.DEFAULT.clone(); @@ -45,10 +47,11 @@ private static void queryMatchesModel(String name, NexmarkQuery query, NexmarkQu Pipeline p = TestPipeline.create(); NexmarkUtils.setupPipeline(NexmarkUtils.CoderStrategy.HAND, p); PCollection> results = - p.apply(NexmarkUtils.batchEventsSource(name, CONFIG)).apply(query); - results.setIsBoundedInternal(IsBounded.BOUNDED); + p.apply(name + ".ReadBounded", NexmarkUtils.batchEventsSource(CONFIG)).apply(query); + //TODO Ismael this should not be called explicitly +// results.setIsBoundedInternal(IsBounded.BOUNDED); PAssert.that(results).satisfies(model.assertionFor()); - p.run(); + p.run().waitUntilFinish(); } @Test diff --git a/integration/java/src/test/java/org/apache/beam/integration/nexmark/UnboundedEventSourceTest.java b/integration/java/nexmark/src/test/java/org/apache/beam/integration/nexmark/UnboundedEventSourceTest.java similarity index 99% rename from integration/java/src/test/java/org/apache/beam/integration/nexmark/UnboundedEventSourceTest.java rename to integration/java/nexmark/src/test/java/org/apache/beam/integration/nexmark/UnboundedEventSourceTest.java index 5d72f775d6ab5..02761d6448967 100644 --- a/integration/java/src/test/java/org/apache/beam/integration/nexmark/UnboundedEventSourceTest.java +++ b/integration/java/nexmark/src/test/java/org/apache/beam/integration/nexmark/UnboundedEventSourceTest.java @@ -15,13 +15,17 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - package org.apache.beam.integration.nexmark; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; +import java.io.IOException; +import java.util.HashSet; +import java.util.Random; +import java.util.Set; + import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.io.UnboundedSource.CheckpointMark; import org.apache.beam.sdk.io.UnboundedSource.UnboundedReader; @@ -32,11 +36,6 @@ import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.io.IOException; -import java.util.HashSet; -import java.util.Random; -import java.util.Set; - /** * Test UnboundedEventSource. */ diff --git a/pom.xml b/pom.xml index 5fd12977972f3..c92d391ccefba 100644 --- a/pom.xml +++ b/pom.xml @@ -187,6 +187,7 @@ sdks runners examples + integration/java/nexmark sdks/java/javadoc From 37ed77cdcf8a3325304ad0d8712c4a57368b6047 Mon Sep 17 00:00:00 2001 From: Etienne Chauchot Date: Thu, 9 Mar 2017 16:45:25 +0100 Subject: [PATCH 289/346] Add comments on queries improvements and fix compilation config --- integration/java/nexmark/pom.xml | 17 +++++++++++++++++ .../beam/integration/nexmark/Query10.java | 4 +++- .../beam/integration/nexmark/Query11.java | 3 ++- .../beam/integration/nexmark/Query12.java | 1 + .../apache/beam/integration/nexmark/Query3.java | 8 +++++--- 5 files changed, 28 insertions(+), 5 deletions(-) diff --git a/integration/java/nexmark/pom.xml b/integration/java/nexmark/pom.xml index d64eb1bb86530..07d14c28bf9de 100644 --- a/integration/java/nexmark/pom.xml +++ b/integration/java/nexmark/pom.xml @@ -134,6 +134,23 @@ org.jacoco jacoco-maven-plugin + + + org.apache.maven.plugins + maven-dependency-plugin + 2.10 + + + analyze-only + + + true + false + + + + + diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query10.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query10.java index 76f93e5ed9df3..7bdcb367937ae 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query10.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query10.java @@ -318,7 +318,9 @@ public void processElement(ProcessContext c, BoundedWindow window) // We expect no late data here, but we'll assume the worst so we can detect any. .withAllowedLateness(Duration.standardDays(1)) .discardingFiredPanes()) - .apply(name + ".GroupByKey2", GroupByKey.create()) + // TODO etienne: unnecessary groupByKey? because aggregators are shared in parallel + // and Pardo is also in parallel, why group all elements in memory of the same executor? + .apply(name + ".GroupByKey2", GroupByKey.create()) .apply(name + ".Index", ParDo.of(new DoFn>, Done>() { final Aggregator unexpectedLateCounter = diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query11.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query11.java index c37c1cde1216c..d610b7cf53b4b 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query11.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query11.java @@ -43,7 +43,8 @@ public Query11(NexmarkConfiguration configuration) { private PCollection applyTyped(PCollection events) { return events.apply(JUST_BIDS) .apply(name + ".Rekey", - ParDo.of(new DoFn>() { + // TODO etienne: why not avoid this ParDo and do a Cont.perElement? + ParDo.of(new DoFn>() { @ProcessElement public void processElement(ProcessContext c) { Bid bid = c.element(); diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query12.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query12.java index ee754c1769a70..72fbb57da6006 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query12.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query12.java @@ -43,6 +43,7 @@ private PCollection applyTyped(PCollection events) { return events .apply(JUST_BIDS) .apply(name + ".Rekey", + // TODO etienne: why not avoid this ParDo and do a Cont.perElement? ParDo.of(new DoFn>() { @ProcessElement public void processElement(ProcessContext c) { diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query3.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query3.java index 24e112eaea875..71969c46eb016 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query3.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query3.java @@ -221,10 +221,12 @@ public Boolean apply(Person person) { .apply("PersonById", PERSON_BY_ID); return - // Join auctions and people. - KeyedPCollectionTuple.of(AUCTION_TAG, auctionsBySellerId) + // Join auctions and people. + // concatenate KeyedPCollections + KeyedPCollectionTuple.of(AUCTION_TAG, auctionsBySellerId) .and(PERSON_TAG, personsById) - .apply(CoGroupByKey.create()) + // group auctions and persons by personId + .apply(CoGroupByKey.create()) .apply(name + ".Join", ParDo.of(joinDoFn)) // Project what we want. From a095e40adf0dc8693fe32395fd86bbefb690af69 Mon Sep 17 00:00:00 2001 From: Etienne Chauchot Date: Wed, 15 Mar 2017 09:52:36 +0100 Subject: [PATCH 290/346] Make NexmarkRunner generic and remove coupling with Google Dataflow issue #28 --- .../nexmark/NexmarkApexRunner.java | 2 + .../nexmark/NexmarkDirectRunner.java | 18 +- .../nexmark/NexmarkFlinkRunner.java | 2 + .../nexmark/NexmarkGoogleDriver.java | 24 - .../nexmark/NexmarkGoogleRunner.java | 505 +----------------- .../integration/nexmark/NexmarkRunner.java | 503 ++++++++++++++++- .../nexmark/NexmarkSparkRunner.java | 2 + .../beam/integration/nexmark/Options.java | 26 + 8 files changed, 540 insertions(+), 542 deletions(-) diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkApexRunner.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkApexRunner.java index ea46082fb20b9..f2da1c7dce106 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkApexRunner.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkApexRunner.java @@ -18,6 +18,8 @@ package org.apache.beam.integration.nexmark; import javax.annotation.Nullable; +import org.apache.beam.runners.apex.ApexRunnerResult; +import org.apache.beam.sdk.PipelineResult; /** * Run a query using the Apex runner. diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkDirectRunner.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkDirectRunner.java index ffb5649f26961..ee234b1382c76 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkDirectRunner.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkDirectRunner.java @@ -18,6 +18,9 @@ package org.apache.beam.integration.nexmark; import javax.annotation.Nullable; +import org.apache.beam.runners.dataflow.DataflowPipelineJob; +import org.apache.beam.runners.direct.DirectRunner; +import org.apache.beam.sdk.PipelineResult; /** * Run a single query using the Direct Runner. @@ -44,7 +47,7 @@ protected int maxNumWorkers() { @Override protected boolean canMonitor() { - return false; + return true; } @Override @@ -62,17 +65,4 @@ protected void waitForPublisherPreload() { throw new UnsupportedOperationException( "Cannot use --pubSubMode=COMBINED with DirectRunner"); } - - /** - * Monitor the performance and progress of a running job. Return final performance if - * it was measured. - */ - @Override - @Nullable - protected NexmarkPerf monitor(NexmarkQuery query) { - return null; - //TODO Ismael Check how we can do this a real implementation -// throw new UnsupportedOperationException( -// "Cannot use --monitorJobs=true with DirectRunner"); - } } diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkFlinkRunner.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkFlinkRunner.java index 72debc455871a..a8b44019e6a90 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkFlinkRunner.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkFlinkRunner.java @@ -18,6 +18,8 @@ package org.apache.beam.integration.nexmark; import javax.annotation.Nullable; +import org.apache.beam.runners.flink.FlinkRunnerResult; +import org.apache.beam.sdk.PipelineResult; /** * Run a query using the Flink runner. diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkGoogleDriver.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkGoogleDriver.java index 003a79b99f608..67c4aeb5fa91a 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkGoogleDriver.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkGoogleDriver.java @@ -45,31 +45,7 @@ class NexmarkGoogleDriver extends NexmarkDriver { - /** - * How long to let streaming pipeline run after all events have been generated and we've - * seen no activity. - */ - private static final Duration DONE_DELAY = Duration.standardMinutes(1); - - /** - * How long to allow no activity without warning. - */ - private static final Duration STUCK_WARNING_DELAY = Duration.standardMinutes(10); - - /** - * How long to let streaming pipeline run after we've - * seen no activity, even if all events have not been generated. - */ - private static final Duration STUCK_TERMINATE_DELAY = Duration.standardDays(3); - - /** - * Delay between perf samples. - */ - private static final Duration PERF_DELAY = Duration.standardSeconds(15); - - /** - * Minimum number of samples needed for 'stead-state' rate calculation. - */ - private static final int MIN_SAMPLES = 9; - - /** - * Minimum length of time over which to consider samples for 'steady-state' rate calculation. - */ - private static final Duration MIN_WINDOW = Duration.standardMinutes(2); public NexmarkGoogleRunner(NexmarkGoogleDriver.NexmarkGoogleOptions options) { super(options); @@ -108,6 +64,11 @@ protected boolean canMonitor() { return true; } + @Override + protected String getJobId(PipelineResult job) { + return ((DataflowPipelineJob)job).getJobId(); + } + @Override protected void invokeBuilderForPublishOnlyPipeline(PipelineBuilder builder) { String jobName = options.getJobName(); @@ -200,460 +161,4 @@ protected void waitForPublisherPreload() { } } - /** - * Monitor the performance and progress of a running job. Return final performance if - * it was measured. - */ - @Override - @Nullable - protected NexmarkPerf monitor(NexmarkQuery query) { - if (!options.getMonitorJobs()) { - return null; - } - if (!(mainResult instanceof DataflowPipelineJob)) { - return null; - } - - if (configuration.debug) { - NexmarkUtils.console("Waiting for main pipeline to 'finish'"); - } else { - NexmarkUtils.console("--debug=false, so job will not self-cancel"); - } - - DataflowPipelineJob job = (DataflowPipelineJob) mainResult; - DataflowPipelineJob publisherJob = (DataflowPipelineJob) publisherResult; - List snapshots = new ArrayList<>(); - long startMsSinceEpoch = System.currentTimeMillis(); - long endMsSinceEpoch = -1; - if (options.getRunningTimeMinutes() != null) { - endMsSinceEpoch = startMsSinceEpoch - + Duration.standardMinutes(options.getRunningTimeMinutes()).getMillis() - - Duration.standardSeconds(configuration.preloadSeconds).getMillis(); - } - long lastActivityMsSinceEpoch = -1; - NexmarkPerf perf = null; - boolean waitingForShutdown = false; - boolean publisherCancelled = false; - List errors = new ArrayList<>(); - - while (true) { - long now = System.currentTimeMillis(); - if (endMsSinceEpoch >= 0 && now > endMsSinceEpoch && !waitingForShutdown) { - NexmarkUtils.console("Reached end of test, cancelling job"); - try { - job.cancel(); - } catch (IOException e) { - throw new RuntimeException("Unable to cancel main job: ", e); - } - if (publisherResult != null) { - try { - publisherJob.cancel(); - } catch (IOException e) { - throw new RuntimeException("Unable to cancel publisher job: ", e); - } - publisherCancelled = true; - } - waitingForShutdown = true; - } - - PipelineResult.State state = job.getState(); - NexmarkUtils.console("%s %s%s", state, queryName, - waitingForShutdown ? " (waiting for shutdown)" : ""); - - NexmarkPerf currPerf; - if (configuration.debug) { - currPerf = currentPerf(startMsSinceEpoch, now, job, snapshots, - query.eventMonitor, query.resultMonitor); - } else { - currPerf = null; - } - - if (perf == null || perf.anyActivity(currPerf)) { - lastActivityMsSinceEpoch = now; - } - - if (options.isStreaming() && !waitingForShutdown) { - Duration quietFor = new Duration(lastActivityMsSinceEpoch, now); - if (query.getFatalCount() != null && getLong(job, query.getFatalCount()) > 0) { - NexmarkUtils.console("job has fatal errors, cancelling."); - errors.add(String.format("Pipeline reported %s fatal errors", query.getFatalCount())); - waitingForShutdown = true; - } else if (configuration.debug && configuration.numEvents > 0 - && currPerf.numEvents == configuration.numEvents - && currPerf.numResults >= 0 && quietFor.isLongerThan(DONE_DELAY)) { - NexmarkUtils.console("streaming query appears to have finished, cancelling job."); - waitingForShutdown = true; - } else if (quietFor.isLongerThan(STUCK_TERMINATE_DELAY)) { - NexmarkUtils.console("streaming query appears to have gotten stuck, cancelling job."); - errors.add("Streaming job was cancelled since appeared stuck"); - waitingForShutdown = true; - } else if (quietFor.isLongerThan(STUCK_WARNING_DELAY)) { - NexmarkUtils.console("WARNING: streaming query appears to have been stuck for %d min.", - quietFor.getStandardMinutes()); - errors.add( - String.format("Streaming query was stuck for %d min", quietFor.getStandardMinutes())); - } - - errors.addAll(checkWatermarks(job, startMsSinceEpoch)); - - if (waitingForShutdown) { - try { - job.cancel(); - } catch (IOException e) { - throw new RuntimeException("Unable to cancel main job: ", e); - } - } - } - - perf = currPerf; - - boolean running = true; - switch (state) { - case UNKNOWN: - case STOPPED: - case RUNNING: - // Keep going. - break; - case DONE: - // All done. - running = false; - break; - case CANCELLED: - running = false; - if (!waitingForShutdown) { - errors.add("Job was unexpectedly cancelled"); - } - break; - case FAILED: - case UPDATED: - // Abnormal termination. - running = false; - errors.add("Job was unexpectedly updated"); - break; - } - - if (!running) { - break; - } - - if (lastActivityMsSinceEpoch == now) { - NexmarkUtils.console("new perf %s", perf); - } else { - NexmarkUtils.console("no activity"); - } - - try { - Thread.sleep(PERF_DELAY.getMillis()); - } catch (InterruptedException e) { - Thread.interrupted(); - NexmarkUtils.console("Interrupted: pipeline is still running"); - } - } - - perf.errors = errors; - perf.snapshots = snapshots; - - if (publisherResult != null) { - NexmarkUtils.console("Shutting down publisher pipeline."); - try { - if (!publisherCancelled) { - publisherJob.cancel(); - } - publisherJob.waitUntilFinish(Duration.standardMinutes(5)); - } catch (IOException e) { - throw new RuntimeException("Unable to cancel publisher job: ", e); - } //TODO Ismael -// catch (InterruptedException e) { -// Thread.interrupted(); -// throw new RuntimeException("Interrupted: publish job still running.", e); -// } - } - - return perf; - } - - enum MetricType { - SYSTEM_WATERMARK, - DATA_WATERMARK, - OTHER - } - - private MetricType getMetricType(MetricUpdate metric) { - String metricName = metric.getName().getName(); - if (metricName.endsWith("windmill-system-watermark")) { - return MetricType.SYSTEM_WATERMARK; - } else if (metricName.endsWith("windmill-data-watermark")) { - return MetricType.DATA_WATERMARK; - } else { - return MetricType.OTHER; - } - } - - /** - * Check that watermarks are not too far behind. - * - *

                Returns a list of errors detected. - */ - private List checkWatermarks(DataflowPipelineJob job, long startMsSinceEpoch) { - long now = System.currentTimeMillis(); - List errors = new ArrayList<>(); -// try { - //TODO Ismael Ask google -// JobMetrics metricResponse = job.getDataflowClient() -// .projects() -// .jobs() -// .getMetrics(job.getProjectId(), job.getJobId()) -// .execute(); - List metrics = null; // metricResponse.getMetrics(); - if (metrics != null) { - boolean foundWatermarks = false; - for (MetricUpdate metric : metrics) { - MetricType type = getMetricType(metric); - if (type == MetricType.OTHER) { - continue; - } - foundWatermarks = true; - @SuppressWarnings("unchecked") - BigDecimal scalar = (BigDecimal) metric.getScalar(); - if (scalar.signum() < 0) { - continue; - } - Instant value = - new Instant(scalar.divideToIntegralValue(new BigDecimal(1000)).longValueExact()); - Instant updateTime = Instant.parse(metric.getUpdateTime()); - - if (options.getWatermarkValidationDelaySeconds() == null - || now > startMsSinceEpoch - + Duration.standardSeconds(options.getWatermarkValidationDelaySeconds()) - .getMillis()) { - Duration threshold = null; - if (type == MetricType.SYSTEM_WATERMARK && options.getMaxSystemLagSeconds() != null) { - threshold = Duration.standardSeconds(options.getMaxSystemLagSeconds()); - } else if (type == MetricType.DATA_WATERMARK - && options.getMaxDataLagSeconds() != null) { - threshold = Duration.standardSeconds(options.getMaxDataLagSeconds()); - } - - if (threshold != null && value.isBefore(updateTime.minus(threshold))) { - String msg = String.format("High lag for %s: %s vs %s (allowed lag of %s)", - metric.getName().getName(), value, updateTime, threshold); - errors.add(msg); - NexmarkUtils.console(msg); - } - } - } - if (!foundWatermarks) { - NexmarkUtils.console("No known watermarks in update: " + metrics); - if (now > startMsSinceEpoch + Duration.standardMinutes(5).getMillis()) { - errors.add("No known watermarks found. Metrics were " + metrics); - } - } - } -// } catch (IOException e) { -// NexmarkUtils.console("Warning: failed to get JobMetrics: " + e); -// } - - return errors; - } - - /** - * Return the current performance given {@code eventMonitor} and {@code resultMonitor}. - */ - private NexmarkPerf currentPerf( - long startMsSinceEpoch, long now, DataflowPipelineJob job, - List snapshots, Monitor eventMonitor, - Monitor resultMonitor) { - NexmarkPerf perf = new NexmarkPerf(); - - long numEvents = getLong(job, eventMonitor.getElementCounter()); - long numEventBytes = getLong(job, eventMonitor.getBytesCounter()); - long eventStart = getTimestamp(now, job, eventMonitor.getStartTime()); - long eventEnd = getTimestamp(now, job, eventMonitor.getEndTime()); - long numResults = getLong(job, resultMonitor.getElementCounter()); - long numResultBytes = getLong(job, resultMonitor.getBytesCounter()); - long resultStart = getTimestamp(now, job, resultMonitor.getStartTime()); - long resultEnd = getTimestamp(now, job, resultMonitor.getEndTime()); - long timestampStart = getTimestamp(now, job, resultMonitor.getStartTimestamp()); - long timestampEnd = getTimestamp(now, job, resultMonitor.getEndTimestamp()); - - long effectiveEnd = -1; - if (eventEnd >= 0 && resultEnd >= 0) { - // It is possible for events to be generated after the last result was emitted. - // (Eg Query 2, which only yields results for a small prefix of the event stream.) - // So use the max of last event and last result times. - effectiveEnd = Math.max(eventEnd, resultEnd); - } else if (resultEnd >= 0) { - effectiveEnd = resultEnd; - } else if (eventEnd >= 0) { - // During startup we may have no result yet, but we would still like to track how - // long the pipeline has been running. - effectiveEnd = eventEnd; - } - - if (effectiveEnd >= 0 && eventStart >= 0 && effectiveEnd >= eventStart) { - perf.runtimeSec = (effectiveEnd - eventStart) / 1000.0; - } - - if (numEvents >= 0) { - perf.numEvents = numEvents; - } - - if (numEvents >= 0 && perf.runtimeSec > 0.0) { - // For streaming we may later replace this with a 'steady-state' value calculated - // from the progress snapshots. - perf.eventsPerSec = numEvents / perf.runtimeSec; - } - - if (numEventBytes >= 0 && perf.runtimeSec > 0.0) { - perf.eventBytesPerSec = numEventBytes / perf.runtimeSec; - } - - if (numResults >= 0) { - perf.numResults = numResults; - } - - if (numResults >= 0 && perf.runtimeSec > 0.0) { - perf.resultsPerSec = numResults / perf.runtimeSec; - } - - if (numResultBytes >= 0 && perf.runtimeSec > 0.0) { - perf.resultBytesPerSec = numResultBytes / perf.runtimeSec; - } - - if (eventStart >= 0) { - perf.startupDelaySec = (eventStart - startMsSinceEpoch) / 1000.0; - } - - if (resultStart >= 0 && eventStart >= 0 && resultStart >= eventStart) { - perf.processingDelaySec = (resultStart - eventStart) / 1000.0; - } - - if (timestampStart >= 0 && timestampEnd >= 0 && perf.runtimeSec > 0.0) { - double eventRuntimeSec = (timestampEnd - timestampStart) / 1000.0; - perf.timeDilation = eventRuntimeSec / perf.runtimeSec; - } - - if (resultEnd >= 0) { - // Fill in the shutdown delay assuming the job has now finished. - perf.shutdownDelaySec = (now - resultEnd) / 1000.0; - } - - perf.jobId = job.getJobId(); - // As soon as available, try to capture cumulative cost at this point too. - - NexmarkPerf.ProgressSnapshot snapshot = new NexmarkPerf.ProgressSnapshot(); - snapshot.secSinceStart = (now - startMsSinceEpoch) / 1000.0; - snapshot.runtimeSec = perf.runtimeSec; - snapshot.numEvents = numEvents; - snapshot.numResults = numResults; - snapshots.add(snapshot); - - captureSteadyState(perf, snapshots); - - return perf; - } - - /** - * Find a 'steady state' events/sec from {@code snapshots} and - * store it in {@code perf} if found. - */ - private void captureSteadyState(NexmarkPerf perf, List snapshots) { - if (!options.isStreaming()) { - return; - } - - // Find the first sample with actual event and result counts. - int dataStart = 0; - for (; dataStart < snapshots.size(); dataStart++) { - if (snapshots.get(dataStart).numEvents >= 0 && snapshots.get(dataStart).numResults >= 0) { - break; - } - } - - // Find the last sample which demonstrated progress. - int dataEnd = snapshots.size() - 1; - for (; dataEnd > dataStart; dataEnd--) { - if (snapshots.get(dataEnd).anyActivity(snapshots.get(dataEnd - 1))) { - break; - } - } - - int numSamples = dataEnd - dataStart + 1; - if (numSamples < MIN_SAMPLES) { - // Not enough samples. - NexmarkUtils.console("%d samples not enough to calculate steady-state event rate", - numSamples); - return; - } - - // We'll look at only the middle third samples. - int sampleStart = dataStart + numSamples / 3; - int sampleEnd = dataEnd - numSamples / 3; - - double sampleSec = - snapshots.get(sampleEnd).secSinceStart - snapshots.get(sampleStart).secSinceStart; - if (sampleSec < MIN_WINDOW.getStandardSeconds()) { - // Not sampled over enough time. - NexmarkUtils.console( - "sample of %.1f sec not long enough to calculate steady-state event rate", - sampleSec); - return; - } - - // Find rate with least squares error. - double sumxx = 0.0; - double sumxy = 0.0; - long prevNumEvents = -1; - for (int i = sampleStart; i <= sampleEnd; i++) { - if (prevNumEvents == snapshots.get(i).numEvents) { - // Skip samples with no change in number of events since they contribute no data. - continue; - } - // Use the effective runtime instead of wallclock time so we can - // insulate ourselves from delays and stutters in the query manager. - double x = snapshots.get(i).runtimeSec; - prevNumEvents = snapshots.get(i).numEvents; - double y = prevNumEvents; - sumxx += x * x; - sumxy += x * y; - } - double eventsPerSec = sumxy / sumxx; - NexmarkUtils.console("revising events/sec from %.1f to %.1f", perf.eventsPerSec, eventsPerSec); - perf.eventsPerSec = eventsPerSec; - } - - /** - * Return the current value for a long counter, or -1 if can't be retrieved. - */ - private long getLong(DataflowPipelineJob job, Aggregator aggregator) { - try { - Collection values = job.getAggregatorValues(aggregator).getValues(); - if (values.size() != 1) { - return -1; - } - return Iterables.getOnlyElement(values); - } catch (AggregatorRetrievalException e) { - return -1; - } - } - - /** - * Return the current value for a time counter, or -1 if can't be retrieved. - */ - private long getTimestamp( - long now, DataflowPipelineJob job, Aggregator aggregator) { - try { - Collection values = job.getAggregatorValues(aggregator).getValues(); - if (values.size() != 1) { - return -1; - } - long value = Iterables.getOnlyElement(values); - if (Math.abs(value - now) > Duration.standardDays(10000).getMillis()) { - return -1; - } - return value; - } catch (AggregatorRetrievalException e) { - return -1; - } - } } diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkRunner.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkRunner.java index 8d0893427f6d8..5365dbe94933e 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkRunner.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkRunner.java @@ -24,15 +24,18 @@ import com.google.api.services.bigquery.model.TableSchema; import com.google.common.base.Strings; 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; import java.util.Collections; import java.util.List; import java.util.concurrent.ThreadLocalRandom; import javax.annotation.Nullable; - +import org.apache.beam.sdk.AggregatorRetrievalException; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.PipelineResult; import org.apache.beam.sdk.io.AvroIO; @@ -40,6 +43,7 @@ import org.apache.beam.sdk.io.TextIO; import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO; 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.values.PCollection; @@ -47,11 +51,40 @@ 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 static org.apache.beam.sdk.metrics.MetricUpdates.MetricUpdate; /** * Run a single Nexmark query using a given configuration. */ public abstract class NexmarkRunner { + /** + * Minimum number of samples needed for 'stead-state' rate calculation. + */ + protected static final int MIN_SAMPLES = 9; + /** + * Minimum length of time over which to consider samples for 'steady-state' rate calculation. + */ + protected static final Duration MIN_WINDOW = Duration.standardMinutes(2); + /** + * Delay between perf samples. + */ + protected static final Duration PERF_DELAY = Duration.standardSeconds(15); + /** + * How long to let streaming pipeline run after all events have been generated and we've + * seen no activity. + */ + private static final Duration DONE_DELAY = Duration.standardMinutes(1); + /** + * How long to allow no activity without warning. + */ + private static final Duration STUCK_WARNING_DELAY = Duration.standardMinutes(10); + /** + * How long to let streaming pipeline run after we've + * seen no activity, even if all events have not been generated. + */ + private static final Duration STUCK_TERMINATE_DELAY = Duration.standardDays(3); /** * Options shared by all runs. */ @@ -131,6 +164,304 @@ private PubsubHelper getPubsub() { */ protected abstract boolean canMonitor(); + /** + * Return the current value for a long counter, or -1 if can't be retrieved. + */ + protected long getLong(PipelineResult job, Aggregator aggregator) { + try { + Collection values = job.getAggregatorValues(aggregator).getValues(); + if (values.size() != 1) { + return -1; + } + return Iterables.getOnlyElement(values); + } catch (AggregatorRetrievalException e) { + return -1; + } + } + + /** + * Return the current value for a time counter, or -1 if can't be retrieved. + */ + protected long getTimestamp( + long now, PipelineResult job, Aggregator aggregator) { + try { + Collection values = job.getAggregatorValues(aggregator).getValues(); + if (values.size() != 1) { + return -1; + } + long value = Iterables.getOnlyElement(values); + if (Math.abs(value - now) > Duration.standardDays(10000).getMillis()) { + return -1; + } + return value; + } catch (AggregatorRetrievalException e) { + return -1; + } + } + + /** + * Find a 'steady state' events/sec from {@code snapshots} and + * store it in {@code perf} if found. + */ + protected void captureSteadyState(NexmarkPerf perf, List snapshots) { + if (!options.isStreaming()) { + return; + } + + // Find the first sample with actual event and result counts. + int dataStart = 0; + for (; dataStart < snapshots.size(); dataStart++) { + if (snapshots.get(dataStart).numEvents >= 0 && snapshots.get(dataStart).numResults >= 0) { + break; + } + } + + // Find the last sample which demonstrated progress. + int dataEnd = snapshots.size() - 1; + for (; dataEnd > dataStart; dataEnd--) { + if (snapshots.get(dataEnd).anyActivity(snapshots.get(dataEnd - 1))) { + break; + } + } + + int numSamples = dataEnd - dataStart + 1; + if (numSamples < MIN_SAMPLES) { + // Not enough samples. + NexmarkUtils.console("%d samples not enough to calculate steady-state event rate", + numSamples); + return; + } + + // We'll look at only the middle third samples. + int sampleStart = dataStart + numSamples / 3; + int sampleEnd = dataEnd - numSamples / 3; + + double sampleSec = + snapshots.get(sampleEnd).secSinceStart - snapshots.get(sampleStart).secSinceStart; + if (sampleSec < MIN_WINDOW.getStandardSeconds()) { + // Not sampled over enough time. + NexmarkUtils.console( + "sample of %.1f sec not long enough to calculate steady-state event rate", + sampleSec); + return; + } + + // Find rate with least squares error. + double sumxx = 0.0; + double sumxy = 0.0; + long prevNumEvents = -1; + for (int i = sampleStart; i <= sampleEnd; i++) { + if (prevNumEvents == snapshots.get(i).numEvents) { + // Skip samples with no change in number of events since they contribute no data. + continue; + } + // Use the effective runtime instead of wallclock time so we can + // insulate ourselves from delays and stutters in the query manager. + double x = snapshots.get(i).runtimeSec; + prevNumEvents = snapshots.get(i).numEvents; + double y = prevNumEvents; + sumxx += x * x; + sumxy += x * y; + } + double eventsPerSec = sumxy / sumxx; + NexmarkUtils.console("revising events/sec from %.1f to %.1f", perf.eventsPerSec, eventsPerSec); + perf.eventsPerSec = eventsPerSec; + } + + /** + * Return the current performance given {@code eventMonitor} and {@code resultMonitor}. + */ + private NexmarkPerf currentPerf( + long startMsSinceEpoch, long now, PipelineResult job, + List snapshots, Monitor eventMonitor, + Monitor resultMonitor) { + NexmarkPerf perf = new NexmarkPerf(); + + long numEvents = getLong(job, eventMonitor.getElementCounter()); + long numEventBytes = getLong(job, eventMonitor.getBytesCounter()); + long eventStart = getTimestamp(now, job, eventMonitor.getStartTime()); + long eventEnd = getTimestamp(now, job, eventMonitor.getEndTime()); + long numResults = getLong(job, resultMonitor.getElementCounter()); + long numResultBytes = getLong(job, resultMonitor.getBytesCounter()); + long resultStart = getTimestamp(now, job, resultMonitor.getStartTime()); + long resultEnd = getTimestamp(now, job, resultMonitor.getEndTime()); + long timestampStart = getTimestamp(now, job, resultMonitor.getStartTimestamp()); + long timestampEnd = getTimestamp(now, job, resultMonitor.getEndTimestamp()); + + long effectiveEnd = -1; + if (eventEnd >= 0 && resultEnd >= 0) { + // It is possible for events to be generated after the last result was emitted. + // (Eg Query 2, which only yields results for a small prefix of the event stream.) + // So use the max of last event and last result times. + effectiveEnd = Math.max(eventEnd, resultEnd); + } else if (resultEnd >= 0) { + effectiveEnd = resultEnd; + } else if (eventEnd >= 0) { + // During startup we may have no result yet, but we would still like to track how + // long the pipeline has been running. + effectiveEnd = eventEnd; + } + + if (effectiveEnd >= 0 && eventStart >= 0 && effectiveEnd >= eventStart) { + perf.runtimeSec = (effectiveEnd - eventStart) / 1000.0; + } + + if (numEvents >= 0) { + perf.numEvents = numEvents; + } + + if (numEvents >= 0 && perf.runtimeSec > 0.0) { + // For streaming we may later replace this with a 'steady-state' value calculated + // from the progress snapshots. + perf.eventsPerSec = numEvents / perf.runtimeSec; + } + + if (numEventBytes >= 0 && perf.runtimeSec > 0.0) { + perf.eventBytesPerSec = numEventBytes / perf.runtimeSec; + } + + if (numResults >= 0) { + perf.numResults = numResults; + } + + if (numResults >= 0 && perf.runtimeSec > 0.0) { + perf.resultsPerSec = numResults / perf.runtimeSec; + } + + if (numResultBytes >= 0 && perf.runtimeSec > 0.0) { + perf.resultBytesPerSec = numResultBytes / perf.runtimeSec; + } + + if (eventStart >= 0) { + perf.startupDelaySec = (eventStart - startMsSinceEpoch) / 1000.0; + } + + if (resultStart >= 0 && eventStart >= 0 && resultStart >= eventStart) { + perf.processingDelaySec = (resultStart - eventStart) / 1000.0; + } + + if (timestampStart >= 0 && timestampEnd >= 0 && perf.runtimeSec > 0.0) { + double eventRuntimeSec = (timestampEnd - timestampStart) / 1000.0; + perf.timeDilation = eventRuntimeSec / perf.runtimeSec; + } + + if (resultEnd >= 0) { + // Fill in the shutdown delay assuming the job has now finished. + perf.shutdownDelaySec = (now - resultEnd) / 1000.0; + } + + perf.jobId = getJobId(job); + // As soon as available, try to capture cumulative cost at this point too. + + NexmarkPerf.ProgressSnapshot snapshot = new NexmarkPerf.ProgressSnapshot(); + snapshot.secSinceStart = (now - startMsSinceEpoch) / 1000.0; + snapshot.runtimeSec = perf.runtimeSec; + snapshot.numEvents = numEvents; + snapshot.numResults = numResults; + snapshots.add(snapshot); + + captureSteadyState(perf, snapshots); + + return perf; + } + + String getJobId(PipelineResult job){return "";} + + // TODO specific to dataflow, see if we can find an equivalent +/* + protected MetricType getMetricType(MetricUpdate metric) { + String metricName = metric.getKey().metricName().name(); + if (metricName.endsWith("windmill-system-watermark")) { + return MetricType.SYSTEM_WATERMARK; + } else if (metricName.endsWith("windmill-data-watermark")) { + return MetricType.DATA_WATERMARK; + } else { + return MetricType.OTHER; + } + } +*/ + + /** + * Check that watermarks are not too far behind. + * + *

                Returns a list of errors detected. + */ + // TODO specific to dataflow, see if we can find an equivalent + /* + private List checkWatermarks(DataflowPipelineJob job, long startMsSinceEpoch) { + long now = System.currentTimeMillis(); + List errors = new ArrayList<>(); + try { + JobMetrics metricResponse = job.getDataflowClient() + .projects() + .jobs() + .getMetrics(job.getProjectId(), job.getJobId()) + .execute(); + List metrics = metricResponse.getMetrics(); + + + + if (metrics != null) { + boolean foundWatermarks = false; + for (MetricUpdate metric : metrics) { + MetricType type = getMetricType(metric); + if (type == MetricType.OTHER) { + continue; + } + foundWatermarks = true; + @SuppressWarnings("unchecked") + BigDecimal scalar = (BigDecimal) metric.getScalar(); + if (scalar.signum() < 0) { + continue; + } + Instant value = + new Instant(scalar.divideToIntegralValue(new BigDecimal(1000)).longValueExact()); + Instant updateTime = Instant.parse(metric.getUpdateTime()); + + if (options.getWatermarkValidationDelaySeconds() == null + || now > startMsSinceEpoch + + Duration.standardSeconds(options.getWatermarkValidationDelaySeconds()) + .getMillis()) { + Duration threshold = null; + if (type == MetricType.SYSTEM_WATERMARK && options.getMaxSystemLagSeconds() != null) { + threshold = Duration.standardSeconds(options.getMaxSystemLagSeconds()); + } else if (type == MetricType.DATA_WATERMARK + && options.getMaxDataLagSeconds() != null) { + threshold = Duration.standardSeconds(options.getMaxDataLagSeconds()); + } + + if (threshold != null && value.isBefore(updateTime.minus(threshold))) { + String msg = String.format("High lag for %s: %s vs %s (allowed lag of %s)", + metric.getKey().metricName().name(), value, updateTime, threshold); + errors.add(msg); + NexmarkUtils.console(msg); + } + } + } + if (!foundWatermarks) { + NexmarkUtils.console("No known watermarks in update: " + metrics); + if (now > startMsSinceEpoch + Duration.standardMinutes(5).getMillis()) { + errors.add("No known watermarks found. Metrics were " + metrics); + } + } + } + } catch (IOException e) { + NexmarkUtils.console("Warning: failed to get JobMetrics: " + e); + } + + return errors; + } +*/ + + // TODO specific to dataflow, see if we can find an equivalent +/* + enum MetricType { + SYSTEM_WATERMARK, + DATA_WATERMARK, + OTHER + } +*/ + /** * Build and run a pipeline using specified options. */ @@ -150,11 +481,173 @@ protected interface PipelineBuilder { protected abstract void waitForPublisherPreload(); /** - * If monitoring, print stats on the main pipeline and return the final perf - * when it has run long enough. Otherwise, return {@literal null} immediately. + * Monitor the performance and progress of a running job. Return final performance if + * it was measured. */ @Nullable - protected abstract NexmarkPerf monitor(NexmarkQuery query); + protected NexmarkPerf monitor(NexmarkQuery query) { + if (!options.getMonitorJobs()) { + return null; + } + + if (configuration.debug) { + NexmarkUtils.console("Waiting for main pipeline to 'finish'"); + } else { + NexmarkUtils.console("--debug=false, so job will not self-cancel"); + } + + PipelineResult job = mainResult; + PipelineResult publisherJob = publisherResult; + List snapshots = new ArrayList<>(); + long startMsSinceEpoch = System.currentTimeMillis(); + long endMsSinceEpoch = -1; + if (options.getRunningTimeMinutes() != null) { + endMsSinceEpoch = startMsSinceEpoch + + Duration.standardMinutes(options.getRunningTimeMinutes()).getMillis() + - Duration.standardSeconds(configuration.preloadSeconds).getMillis(); + } + long lastActivityMsSinceEpoch = -1; + NexmarkPerf perf = null; + boolean waitingForShutdown = false; + boolean publisherCancelled = false; + List errors = new ArrayList<>(); + + while (true) { + long now = System.currentTimeMillis(); + if (endMsSinceEpoch >= 0 && now > endMsSinceEpoch && !waitingForShutdown) { + NexmarkUtils.console("Reached end of test, cancelling job"); + try { + job.cancel(); + } catch (IOException e) { + throw new RuntimeException("Unable to cancel main job: ", e); + } + if (publisherResult != null) { + try { + publisherJob.cancel(); + } catch (IOException e) { + throw new RuntimeException("Unable to cancel publisher job: ", e); + } + publisherCancelled = true; + } + waitingForShutdown = true; + } + + PipelineResult.State state = job.getState(); + NexmarkUtils.console("%s %s%s", state, queryName, + waitingForShutdown ? " (waiting for shutdown)" : ""); + + NexmarkPerf currPerf; + if (configuration.debug) { + currPerf = currentPerf(startMsSinceEpoch, now, job, snapshots, + query.eventMonitor, query.resultMonitor); + } else { + currPerf = null; + } + + if (perf == null || perf.anyActivity(currPerf)) { + lastActivityMsSinceEpoch = now; + } + + if (options.isStreaming() && !waitingForShutdown) { + Duration quietFor = new Duration(lastActivityMsSinceEpoch, now); + if (query.getFatalCount() != null && getLong(job, query.getFatalCount()) > 0) { + NexmarkUtils.console("job has fatal errors, cancelling."); + errors.add(String.format("Pipeline reported %s fatal errors", query.getFatalCount())); + waitingForShutdown = true; + } else if (configuration.debug && configuration.numEvents > 0 + && currPerf.numEvents == configuration.numEvents + && currPerf.numResults >= 0 && quietFor.isLongerThan(DONE_DELAY)) { + NexmarkUtils.console("streaming query appears to have finished, cancelling job."); + waitingForShutdown = true; + } else if (quietFor.isLongerThan(STUCK_TERMINATE_DELAY)) { + NexmarkUtils.console("streaming query appears to have gotten stuck, cancelling job."); + errors.add("Streaming job was cancelled since appeared stuck"); + waitingForShutdown = true; + } else if (quietFor.isLongerThan(STUCK_WARNING_DELAY)) { + NexmarkUtils.console("WARNING: streaming query appears to have been stuck for %d min.", + quietFor.getStandardMinutes()); + errors.add( + String.format("Streaming query was stuck for %d min", quietFor.getStandardMinutes())); + } + + // TODO specific to dataflow, see if we can find an equivalent +// errors.addAll(checkWatermarks(job, startMsSinceEpoch)); + + if (waitingForShutdown) { + try { + job.cancel(); + } catch (IOException e) { + throw new RuntimeException("Unable to cancel main job: ", e); + } + } + } + + perf = currPerf; + + boolean running = true; + switch (state) { + case UNKNOWN: + case STOPPED: + case RUNNING: + // Keep going. + break; + case DONE: + // All done. + running = false; + break; + case CANCELLED: + running = false; + if (!waitingForShutdown) { + errors.add("Job was unexpectedly cancelled"); + } + break; + case FAILED: + case UPDATED: + // Abnormal termination. + running = false; + errors.add("Job was unexpectedly updated"); + break; + } + + if (!running) { + break; + } + + if (lastActivityMsSinceEpoch == now) { + NexmarkUtils.console("new perf %s", perf); + } else { + NexmarkUtils.console("no activity"); + } + + try { + Thread.sleep(PERF_DELAY.getMillis()); + } catch (InterruptedException e) { + Thread.interrupted(); + NexmarkUtils.console("Interrupted: pipeline is still running"); + } + } + + perf.errors = errors; + perf.snapshots = snapshots; + + if (publisherResult != null) { + NexmarkUtils.console("Shutting down publisher pipeline."); + try { + if (!publisherCancelled) { + publisherJob.cancel(); + } + publisherJob.waitUntilFinish(Duration.standardMinutes(5)); + } catch (IOException e) { + throw new RuntimeException("Unable to cancel publisher job: ", e); + } //TODO Ismael +// catch (InterruptedException e) { +// Thread.interrupted(); +// throw new RuntimeException("Interrupted: publish job still running.", e); +// } + } + + return perf; + } // ================================================================================ // Basic sources and sinks @@ -733,5 +1226,7 @@ public NexmarkPerf run(NexmarkConfiguration runConfiguration) { queryName = null; // TODO: Cleanup pathsToDelete } + } + } diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkSparkRunner.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkSparkRunner.java index 81c6b2bfa8b6d..109e8a0080c40 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkSparkRunner.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkSparkRunner.java @@ -18,6 +18,8 @@ package org.apache.beam.integration.nexmark; import javax.annotation.Nullable; +import org.apache.beam.runners.spark.SparkPipelineResult; +import org.apache.beam.sdk.PipelineResult; /** * Run a query using the Spark runner. diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Options.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Options.java index efaf6988d3b94..388473d1d669e 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Options.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Options.java @@ -357,4 +357,30 @@ public interface Options extends PubsubOptions { Boolean getDebug(); void setDebug(Boolean value); + + @Description("If set, cancel running pipelines after this long") + @Nullable + Long getRunningTimeMinutes(); + + void setRunningTimeMinutes(Long value); + + @Description("If set and --monitorJobs is true, check that the system watermark is never more " + + "than this far behind real time") + @Nullable + Long getMaxSystemLagSeconds(); + + void setMaxSystemLagSeconds(Long value); + + @Description("If set and --monitorJobs is true, check that the data watermark is never more " + + "than this far behind real time") + @Nullable + Long getMaxDataLagSeconds(); + + void setMaxDataLagSeconds(Long value); + + @Description("Only start validating watermarks after this many seconds") + @Nullable + Long getWatermarkValidationDelaySeconds(); + + void setWatermarkValidationDelaySeconds(Long value); } From a1fe33bc122b26960697c87620ca0dc2ed522e56 Mon Sep 17 00:00:00 2001 From: Etienne Chauchot Date: Wed, 15 Mar 2017 15:25:41 +0100 Subject: [PATCH 291/346] Activate monitoring on NexmarkSparkRunner and on specific runners issue #28 Fix compilation issue after rebase + make checkstyle happy again --- integration/java/nexmark/pom.xml | 32 +++++++++---------- .../nexmark/NexmarkApexRunner.java | 2 -- .../nexmark/NexmarkDirectRunner.java | 5 --- .../nexmark/NexmarkFlinkRunner.java | 12 +------ .../nexmark/NexmarkGoogleDriver.java | 2 -- .../nexmark/NexmarkGoogleRunner.java | 2 +- .../integration/nexmark/NexmarkRunner.java | 13 ++++---- .../nexmark/NexmarkSparkDriver.java | 4 +-- .../nexmark/NexmarkSparkRunner.java | 11 +------ .../integration/nexmark/NexmarkUtils.java | 3 +- .../beam/integration/nexmark/Query5.java | 3 +- 11 files changed, 31 insertions(+), 58 deletions(-) diff --git a/integration/java/nexmark/pom.xml b/integration/java/nexmark/pom.xml index 07d14c28bf9de..febd96d313a50 100644 --- a/integration/java/nexmark/pom.xml +++ b/integration/java/nexmark/pom.xml @@ -179,28 +179,28 @@ beam-runners-flink_2.10 - - - - - - + + org.apache.flink + flink-shaded-hadoop2 + ${flink.version} + provided + org.apache.beam beam-runners-spark - - - - - - - - - - + + org.apache.spark + spark-core_2.10 + ${spark.version} + + + org.apache.spark + spark-streaming_2.10 + ${spark.version} + diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkApexRunner.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkApexRunner.java index f2da1c7dce106..ea46082fb20b9 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkApexRunner.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkApexRunner.java @@ -18,8 +18,6 @@ package org.apache.beam.integration.nexmark; import javax.annotation.Nullable; -import org.apache.beam.runners.apex.ApexRunnerResult; -import org.apache.beam.sdk.PipelineResult; /** * Run a query using the Apex runner. diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkDirectRunner.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkDirectRunner.java index ee234b1382c76..c70e41eb02233 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkDirectRunner.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkDirectRunner.java @@ -17,11 +17,6 @@ */ package org.apache.beam.integration.nexmark; -import javax.annotation.Nullable; -import org.apache.beam.runners.dataflow.DataflowPipelineJob; -import org.apache.beam.runners.direct.DirectRunner; -import org.apache.beam.sdk.PipelineResult; - /** * Run a single query using the Direct Runner. */ diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkFlinkRunner.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkFlinkRunner.java index a8b44019e6a90..8e22917fdacd9 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkFlinkRunner.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkFlinkRunner.java @@ -17,10 +17,6 @@ */ package org.apache.beam.integration.nexmark; -import javax.annotation.Nullable; -import org.apache.beam.runners.flink.FlinkRunnerResult; -import org.apache.beam.sdk.PipelineResult; - /** * Run a query using the Flink runner. */ @@ -42,7 +38,7 @@ protected int maxNumWorkers() { @Override protected boolean canMonitor() { - return false; + return true; } @Override @@ -56,12 +52,6 @@ protected void waitForPublisherPreload() { throw new UnsupportedOperationException(); } - @Override - @Nullable - protected NexmarkPerf monitor(NexmarkQuery query) { - return null; - } - public NexmarkFlinkRunner(NexmarkFlinkDriver.NexmarkFlinkOptions options) { super(options); } diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkGoogleDriver.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkGoogleDriver.java index 67c4aeb5fa91a..50c2a7ca0d023 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkGoogleDriver.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkGoogleDriver.java @@ -17,10 +17,8 @@ */ package org.apache.beam.integration.nexmark; -import javax.annotation.Nullable; import org.apache.beam.runners.dataflow.DataflowRunner; import org.apache.beam.runners.dataflow.options.DataflowPipelineOptions; -import org.apache.beam.sdk.options.Description; import org.apache.beam.sdk.options.PipelineOptionsFactory; /** diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkGoogleRunner.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkGoogleRunner.java index c78bb42b1b36c..135d428ad5144 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkGoogleRunner.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkGoogleRunner.java @@ -66,7 +66,7 @@ protected boolean canMonitor() { @Override protected String getJobId(PipelineResult job) { - return ((DataflowPipelineJob)job).getJobId(); + return ((DataflowPipelineJob) job).getJobId(); } @Override diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkRunner.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkRunner.java index 5365dbe94933e..8d4c1f1be1097 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkRunner.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkRunner.java @@ -53,8 +53,6 @@ import org.apache.beam.sdk.values.TupleTagList; import org.joda.time.Duration; -import static org.apache.beam.sdk.metrics.MetricUpdates.MetricUpdate; - /** * Run a single Nexmark query using a given configuration. */ @@ -203,7 +201,8 @@ protected long getTimestamp( * Find a 'steady state' events/sec from {@code snapshots} and * store it in {@code perf} if found. */ - protected void captureSteadyState(NexmarkPerf perf, List snapshots) { + protected void captureSteadyState(NexmarkPerf perf, + List snapshots) { if (!options.isStreaming()) { return; } @@ -365,7 +364,9 @@ private NexmarkPerf currentPerf( return perf; } - String getJobId(PipelineResult job){return "";} + String getJobId(PipelineResult job) { + return ""; + } // TODO specific to dataflow, see if we can find an equivalent /* @@ -926,8 +927,8 @@ private void sinkResultsToBigQuery( new TableFieldSchema().setName("index").setType("INTEGER"), new TableFieldSchema().setName("value").setType("STRING"))))); NexmarkUtils.console("Writing results to BigQuery table %s", tableSpec); - BigQueryIO.Write.Bound io = - BigQueryIO.Write.to(tableSpec) + BigQueryIO.Write io = + BigQueryIO.write().to(tableSpec) .withSchema(tableSchema) .withCreateDisposition(BigQueryIO.Write.CreateDisposition.CREATE_IF_NEEDED) .withWriteDisposition(BigQueryIO.Write.WriteDisposition.WRITE_APPEND); diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkSparkDriver.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkSparkDriver.java index 1ea963d819363..a46d38a2112e1 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkSparkDriver.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkSparkDriver.java @@ -18,7 +18,6 @@ package org.apache.beam.integration.nexmark; import org.apache.beam.runners.spark.SparkPipelineOptions; -import org.apache.beam.runners.spark.SparkRunner; import org.apache.beam.sdk.options.PipelineOptionsFactory; /** @@ -39,7 +38,8 @@ public static void main(String[] args) { PipelineOptionsFactory.fromArgs(args) .withValidation() .as(NexmarkSparkOptions.class); - options.setRunner(SparkRunner.class); +// options.setRunner(org.apache.beam.runners.spark.SparkRunner.class); + options.setRunner(org.apache.beam.runners.spark.SparkRunnerDebugger.class); NexmarkSparkRunner runner = new NexmarkSparkRunner(options); new NexmarkSparkDriver().runAll(options, runner); } diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkSparkRunner.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkSparkRunner.java index 109e8a0080c40..32fee30639330 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkSparkRunner.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkSparkRunner.java @@ -17,10 +17,6 @@ */ package org.apache.beam.integration.nexmark; -import javax.annotation.Nullable; -import org.apache.beam.runners.spark.SparkPipelineResult; -import org.apache.beam.sdk.PipelineResult; - /** * Run a query using the Spark runner. */ @@ -42,7 +38,7 @@ protected int maxNumWorkers() { @Override protected boolean canMonitor() { - return false; + return true; } @Override @@ -56,11 +52,6 @@ protected void waitForPublisherPreload() { throw new UnsupportedOperationException(); } - @Override - @Nullable - protected NexmarkPerf monitor(NexmarkQuery query) { - return null; - } public NexmarkSparkRunner(NexmarkSparkDriver.NexmarkSparkOptions options) { super(options); diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkUtils.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkUtils.java index 6588f8577ac60..8f4cb22845aa5 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkUtils.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkUtils.java @@ -37,7 +37,6 @@ import org.apache.beam.sdk.coders.CustomCoder; import org.apache.beam.sdk.coders.SerializableCoder; import org.apache.beam.sdk.io.Read; -import org.apache.beam.sdk.runners.PipelineRunner; import org.apache.beam.sdk.transforms.Aggregator; import org.apache.beam.sdk.transforms.Combine; import org.apache.beam.sdk.transforms.DoFn; @@ -325,8 +324,8 @@ public static void console(String format, Object... args) { * Setup pipeline with codes and some other options. */ public static void setupPipeline(CoderStrategy coderStrategy, Pipeline p) { - PipelineRunner runner = p.getRunner(); //TODO Ismael check +// PipelineRunner runner = p.getRunner(); // if (runner instanceof DirectRunner) { // // Disable randomization of output since we want to check batch and streaming match the // // model both locally and on the cloud. diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query5.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query5.java index 70019867ba082..9020494e6e611 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query5.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query5.java @@ -67,7 +67,8 @@ private PCollection applyTyped(PCollection events) { // Count the number of bids per auction id. .apply(Count.perElement()) - // We'll want to keep all auctions with the maximal number of bids. + //TODO replace by simple key + // We'll want to keep all auctions with the maximal number of bids. // Start by lifting each into a singleton list. .apply(name + ".ToSingletons", ParDo.of(new DoFn, KV, Long>>() { From 1bd57351f1db9b932b253c36d08098cf57ce652b Mon Sep 17 00:00:00 2001 From: Etienne Chauchot Date: Thu, 16 Mar 2017 11:38:08 +0100 Subject: [PATCH 292/346] Fix Queries tests Workaround for issue #22 + extra cleaning Replace junit asserts by hamcrest asserts Set numEvents in test to the minimum number that makes the tests pass issue #15 comments, improve asserts (hamcrest), reformat For now make generate monothreaded --- integration/java/nexmark/pom.xml | 8 ++- .../nexmark/NexmarkQueryModel.java | 49 ++++++++++--------- .../integration/nexmark/NexmarkUtils.java | 3 +- .../beam/integration/nexmark/Query0Model.java | 1 + .../beam/integration/nexmark/Query1Model.java | 1 + .../beam/integration/nexmark/Query7Model.java | 1 + .../beam/integration/nexmark/Query8Model.java | 2 +- .../nexmark/WinningBidsSimulator.java | 1 + .../beam/integration/nexmark/QueryTest.java | 13 ++--- 9 files changed, 45 insertions(+), 34 deletions(-) diff --git a/integration/java/nexmark/pom.xml b/integration/java/nexmark/pom.xml index febd96d313a50..27abb0ed0ff13 100644 --- a/integration/java/nexmark/pom.xml +++ b/integration/java/nexmark/pom.xml @@ -264,7 +264,13 @@ org.hamcrest hamcrest-all - test + ${hamcrest.version} + + + + org.hamcrest + hamcrest-library + ${hamcrest.version} diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkQueryModel.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkQueryModel.java index a23f82b0654ea..f265e0d96e9cd 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkQueryModel.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkQueryModel.java @@ -17,6 +17,11 @@ */ package org.apache.beam.integration.nexmark; +import static org.hamcrest.CoreMatchers.containsString; +import static org.hamcrest.CoreMatchers.equalTo; +import static org.hamcrest.CoreMatchers.hasItems; +import org.hamcrest.collection.IsIterableContainingInAnyOrder; + import java.io.Serializable; import java.util.ArrayList; import java.util.Collection; @@ -28,16 +33,23 @@ import org.apache.beam.sdk.transforms.SerializableFunction; import org.apache.beam.sdk.values.TimestampedValue; +import org.hamcrest.core.IsCollectionContaining; +import org.hamcrest.core.IsEqual; import org.joda.time.Duration; import org.joda.time.Instant; import org.junit.Assert; /** - * Base class for models of the eight NEXMark queries. Provides an assertion - * function which can be applied against the actual query results to check their consistency - * with the model. + * Base class for models of the eight NEXMark queries. Provides an assertion function which can be + * applied against the actual query results to check their consistency with the model. */ public abstract class NexmarkQueryModel implements Serializable { + protected final NexmarkConfiguration configuration; + + public NexmarkQueryModel(NexmarkConfiguration configuration) { + this.configuration = configuration; + } + /** * Return the start of the most recent window of {@code size} and {@code period} which ends * strictly before {@code timestamp}. @@ -50,15 +62,7 @@ public static Instant windowStart(Duration size, Duration period, Instant timest return new Instant(lim - s); } - protected final NexmarkConfiguration configuration; - - public NexmarkQueryModel(NexmarkConfiguration configuration) { - this.configuration = configuration; - } - - /** - * Convert {@code itr} to strings capturing values, timestamps and order. - */ + /** Convert {@code itr} to strings capturing values, timestamps and order. */ protected static List toValueTimestampOrder(Iterator> itr) { List strings = new ArrayList<>(); while (itr.hasNext()) { @@ -67,9 +71,7 @@ protected static List toValueTimestampOrder(Iterator List toValueOrder(Iterator> itr) { List strings = new ArrayList<>(); while (itr.hasNext()) { @@ -78,9 +80,7 @@ protected static List toValueOrder(Iterator> itr return strings; } - /** - * Convert {@code itr} to strings capturing values only. - */ + /** Convert {@code itr} to strings capturing values only. */ protected static Set toValue(Iterator> itr) { Set strings = new HashSet<>(); while (itr.hasNext()) { @@ -99,22 +99,23 @@ protected Iterable> relevantResults( } /** - * Convert iterator of elements to collection of strings to use when testing coherence - * of model against actual query results. + * Convert iterator of elements to collection of strings to use when testing coherence of model + * against actual query results. */ protected abstract Collection toCollection(Iterator> itr); - /** - * Return assertion to use on results of pipeline for this query. - */ + /** Return assertion to use on results of pipeline for this query. */ public SerializableFunction>, Void> assertionFor() { final Collection expectedStrings = toCollection(simulator().results()); + final String[] expectedStringsArray = expectedStrings.toArray(new String[expectedStrings.size()]); return new SerializableFunction>, Void>() { @Override public Void apply(Iterable> actual) { Collection actualStrings = toCollection(relevantResults(actual).iterator()); - Assert.assertEquals(expectedStrings, actualStrings); + Assert.assertThat("wrong pipeline output", actualStrings, IsEqual.equalTo(expectedStrings)); +//compare without order +// Assert.assertThat("wrong pipeline output", actualStrings, IsIterableContainingInAnyOrder.containsInAnyOrder(expectedStringsArray)); return null; } }; diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkUtils.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkUtils.java index 8f4cb22845aa5..f7417d3c8513e 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkUtils.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkUtils.java @@ -382,8 +382,7 @@ public static Iterator> standardEventIterator( */ public static PTransform> batchEventsSource( NexmarkConfiguration configuration) { - return Read.from(new BoundedEventSource( - NexmarkUtils.standardGeneratorConfig(configuration), configuration.numEventGenerators)); + return Read.from(new BoundedEventSource(standardGeneratorConfig(configuration), configuration.numEventGenerators)); } /** diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query0Model.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query0Model.java index b7cdf1cf861a5..37e3f936e3186 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query0Model.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query0Model.java @@ -42,6 +42,7 @@ protected void run() { return; } addResult(timestampedEvent); + //TODO test fails because offset of some hundreds of ms beween expect and actual } } diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query1Model.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query1Model.java index ace6f7ead6866..16287e68fb029 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query1Model.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query1Model.java @@ -53,6 +53,7 @@ protected void run() { TimestampedValue result = TimestampedValue.of(resultBid, timestampedEvent.getTimestamp()); addResult(result); + //TODO test fails because offset of some hundreds of ms beween expect and actual } } diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query7Model.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query7Model.java index 73e96e24672dc..0033c68ce6dd5 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query7Model.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query7Model.java @@ -107,6 +107,7 @@ protected void run() { } // Keep only the highest bids. captureBid(event.bid); + //TODO test fails because offset of some hundreds of ms between expect and actual } } diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query8Model.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query8Model.java index fdd2a3522f541..261e383db671e 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query8Model.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query8Model.java @@ -115,7 +115,7 @@ public void run() { // Remember auction for future new people. newAuctions.put(event.newAuction.seller, event.newAuction); } - } else { + } else { // event is not an auction, nor a bid, so it is a person // Join new person with existing auctions. for (Auction auction : newAuctions.get(event.newPerson.id)) { addResult(auction, event.newPerson, timestamp); diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/WinningBidsSimulator.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/WinningBidsSimulator.java index 59705562e55bb..dc8094b3c0371 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/WinningBidsSimulator.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/WinningBidsSimulator.java @@ -175,6 +175,7 @@ protected void run() { return; } addResult(result); + //TODO test fails because offset of some hundreds of ms beween expect and actual return; } diff --git a/integration/java/nexmark/src/test/java/org/apache/beam/integration/nexmark/QueryTest.java b/integration/java/nexmark/src/test/java/org/apache/beam/integration/nexmark/QueryTest.java index d4d51f17c71f5..e481eac678683 100644 --- a/integration/java/nexmark/src/test/java/org/apache/beam/integration/nexmark/QueryTest.java +++ b/integration/java/nexmark/src/test/java/org/apache/beam/integration/nexmark/QueryTest.java @@ -25,6 +25,7 @@ import org.apache.beam.sdk.values.TimestampedValue; import org.junit.Ignore; +import org.junit.Rule; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; @@ -33,23 +34,23 @@ * Test the various NEXMark queries yield results coherent with their models. */ @RunWith(JUnit4.class) -@Ignore -//TODO Ismael public class QueryTest { private static final NexmarkConfiguration CONFIG = NexmarkConfiguration.DEFAULT.clone(); + @Rule + public TestPipeline p = TestPipeline.create(); static { - CONFIG.numEvents = 2000; + //careful, results of tests are linked to numEvents value + CONFIG.numEvents = 100; } /** Test {@code query} matches {@code model}. */ - private static void queryMatchesModel(String name, NexmarkQuery query, NexmarkQueryModel model) { - Pipeline p = TestPipeline.create(); + private void queryMatchesModel(String name, NexmarkQuery query, NexmarkQueryModel model) { NexmarkUtils.setupPipeline(NexmarkUtils.CoderStrategy.HAND, p); PCollection> results = p.apply(name + ".ReadBounded", NexmarkUtils.batchEventsSource(CONFIG)).apply(query); //TODO Ismael this should not be called explicitly -// results.setIsBoundedInternal(IsBounded.BOUNDED); + results.setIsBoundedInternal(PCollection.IsBounded.BOUNDED); PAssert.that(results).satisfies(model.assertionFor()); p.run().waitUntilFinish(); } From 9ce9bf076032e1c9aeb3a6dce806ad4b96127157 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Isma=C3=ABl=20Mej=C3=ADa?= Date: Tue, 21 Mar 2017 18:29:20 +0100 Subject: [PATCH 293/346] Fix Apex driver and update execution matrix --- integration/java/nexmark/README.md | 109 ++++++++++-------- integration/java/nexmark/pom.xml | 27 ++++- .../nexmark/NexmarkApexRunner.java | 5 - .../nexmark/NexmarkDirectRunner.java | 5 - .../nexmark/NexmarkFlinkRunner.java | 5 - .../nexmark/NexmarkGoogleRunner.java | 5 - .../integration/nexmark/NexmarkRunner.java | 9 -- .../nexmark/NexmarkSparkRunner.java | 5 - .../beam/integration/nexmark/Query5.java | 1 - .../src/main/resources/log4j.properties | 9 ++ .../nexmark/UnboundedEventSourceTest.java | 4 +- 11 files changed, 100 insertions(+), 84 deletions(-) diff --git a/integration/java/nexmark/README.md b/integration/java/nexmark/README.md index 4c08c28137008..7a91ab22e975a 100644 --- a/integration/java/nexmark/README.md +++ b/integration/java/nexmark/README.md @@ -122,63 +122,80 @@ Number of events generators --numEventGenerators=4 -## Flink specific configuration +## Apex specific configuration ---suite=SMOKE --manageResources=false --monitorJobs=false \ ---flinkMaster=local +--suite=SMOKE --manageResources=false --monitorJobs=true ## Direct specific configuration ---suite=SMOKE --manageResources=false --monitorJobs=false \ +--suite=SMOKE --manageResources=false --monitorJobs=true \ --enforceEncodability=false --enforceImmutability=false +## Flink specific configuration + +--suite=SMOKE --manageResources=false --monitorJobs=true \ +--flinkMaster=local + ## Spark specific configuration ---suite=SMOKE ---manageResources=false --monitorJobs=false --sparkMaster=local --Dspark.ui.enabled=false --DSPARK_LOCAL_IP=localhost --Dsun.io.serialization.extendedDebugInfo=true +--suite=SMOKE --manageResources=false --monitorJobs=true --sparkMaster=local \ +-Dspark.ui.enabled=false -DSPARK_LOCAL_IP=localhost -Dsun.io.serialization.extendedDebugInfo=true # Current Status -Open issues are currently opened on [github](https://github.com/iemejia/beam/issues): - -## Batch Mode / Synthetic / Local - -| Query | Direct | Spark | Flink | Apex | -| ----: | ------ | ------ | ------ | ------ | -| 0 | Ok | #1 | Ok | | -| 1 | Ok | #1 | Ok | | -| 2 | Ok | NEX-01 | Ok | | -| 3 | NEX-07 | NEX-07 | NEX-07 | | -| 4 | Ok | Ok | NEX-02 | | -| 5 | Ok | NEX-03 | Ok | | -| 6 | Ok | OK | NEX-02 | | -| 7 | Ok | NEX-01 | Ok | | -| 8 | Ok | NEX-01 | Ok | | -| 9 | Ok | OK | NEX-02 | | -| 10 | NEX-05 | NEX-04 | Ok | | -| 11 | Ok | NEX-01 | Ok | | -| 12 | Ok | NEX-01 | Ok | | - -## Streaming Mode / Synthetic / Local - -| Query | Direct | Spark | Flink | Apex | -| ----: | ------ | ------ | ------ | ------ | -| 0 | Ok | | | | -| 1 | Ok | | | | -| 2 | Ok | | | | -| 3 | NEX-07 | | | | -| 4 | Ok | | | | -| 5 | Ok | | | | -| 6 | Ok | | | | -| 7 | Ok | | | | -| 8 | Ok | | | | -| 9 | Ok | | | | -| 10 | NEX-05 | | | | -| 11 | Ok | | | | -| 12 | Ok | | | | +Open issues are tracked [here](https://github.com../../../../../issues): + +## Batch / Synthetic / Local + +| Query | Direct | Spark | Flink | Apex | +| ----: | ------------------------------ | ------------------------------ | ------------------------------ | ------------------------------- | +| 0 | ok | [#1](../../../../../issues/1) | ok | ok | +| 1 | ok | [#1](../../../../../issues/1) | ok | ok | +| 2 | ok | [#1](../../../../../issues/1) | ok | ok | +| 3 | [#7](../../../../../issues/7) | [#7](../../../../../issues/7) | [#7](../../../../../issues/7) | [#7](../../../../../issues/7) | +| 4 | ok | ok | [#2](../../../../../issues/2) | ok | +| 5 | ok | [#3](../../../../../issues/3) | ok | ok | +| 6 | ok | ok | [#2](../../../../../issues/2) | ok | +| 7 | ok | [#1](../../../../../issues/1) | ok | [#24](../../../../../issues/24) | +| 8 | ok | [#1](../../../../../issues/1) | ok | ok | +| 9 | ok | ok | [#2](../../../../../issues/2) | ok | +| 10 | [#5](../../../../../issues/5) | [#4](../../../../../issues/4) | ok | ok | +| 11 | ok | [#1](../../../../../issues/1) | ok | ok | +| 12 | ok | [#1](../../../../../issues/1) | ok | ok | + +## Streaming / Synthetic / Local + +| Query | Direct | Spark | Flink | Apex | +| ----: | ------------------------------ | ------------------------------ | ------------------------------ | ------------------------------ | +| 0 | ok | | | ok | +| 1 | ok | | | ok | +| 2 | ok | | | ok | +| 3 | [#7](../../../../../issues/7) | | | [#7](../../../../../issues/7) | +| 4 | ok | | | ok | +| 5 | ok | | | ok | +| 6 | ok | | | ok | +| 7 | ok | | | ? | +| 8 | ok | | | ok | +| 9 | ok | | | ok | +| 10 | [#5](../../../../../issues/5) | | | ? | +| 11 | ok | | | Ok | +| 12 | ok | | | Ok | + +## Batch / Synthetic / Cluster + +TODO + +| Query | Dataflow | Spark | Flink | Apex | +| ----: | ------------------------------ | ------------------------------ | ------------------------------ | ------------------------------ | +| 0 | | | | | + +## Streaming / Synthetic / Cluster + +TODO + +| Query | Dataflow | Spark | Flink | Apex | +| ----: | ------------------------------ | ------------------------------ | ------------------------------ | ------------------------------ | +| 0 | | | | | # Running Nexmark diff --git a/integration/java/nexmark/pom.xml b/integration/java/nexmark/pom.xml index 27abb0ed0ff13..0ecc29839ce17 100644 --- a/integration/java/nexmark/pom.xml +++ b/integration/java/nexmark/pom.xml @@ -28,7 +28,7 @@ beam-integration-java - Apache Beam :: Integration Tests :: Java All + Apache Beam :: Integration Tests :: Java :: Nexmark jar @@ -37,6 +37,7 @@ UTF-8 1.2.0 1.6.3 + 1.9.3 true @@ -207,6 +208,30 @@ org.apache.beam beam-runners-apex + + com.esotericsoftware.kryo + kryo + ${apex.kryo.version} + runtime + + + com.fasterxml.jackson.core + jackson-databind + ${jackson.version} + runtime + + + org.codehaus.jackson + jackson-mapper-asl + ${apex.codehaus.jackson.version} + runtime + + + org.codehaus.jackson + jackson-core-asl + ${apex.codehaus.jackson.version} + runtime + diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkApexRunner.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkApexRunner.java index ea46082fb20b9..3b8993aa5eca6 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkApexRunner.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkApexRunner.java @@ -38,11 +38,6 @@ protected int maxNumWorkers() { return 5; } - @Override - protected boolean canMonitor() { - return false; - } - @Override protected void invokeBuilderForPublishOnlyPipeline( PipelineBuilder builder) { diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkDirectRunner.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkDirectRunner.java index c70e41eb02233..0119bbc5d9e63 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkDirectRunner.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkDirectRunner.java @@ -40,11 +40,6 @@ protected int maxNumWorkers() { return 1; } - @Override - protected boolean canMonitor() { - return true; - } - @Override protected void invokeBuilderForPublishOnlyPipeline(PipelineBuilder builder) { throw new UnsupportedOperationException( diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkFlinkRunner.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkFlinkRunner.java index 8e22917fdacd9..95ab1ad672c8d 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkFlinkRunner.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkFlinkRunner.java @@ -36,11 +36,6 @@ protected int maxNumWorkers() { return 5; } - @Override - protected boolean canMonitor() { - return true; - } - @Override protected void invokeBuilderForPublishOnlyPipeline( PipelineBuilder builder) { diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkGoogleRunner.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkGoogleRunner.java index 135d428ad5144..f4bfb1e6633f0 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkGoogleRunner.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkGoogleRunner.java @@ -59,11 +59,6 @@ protected int maxNumWorkers() { return Math.max(options.getNumWorkers(), options.getMaxNumWorkers()); } - @Override - protected boolean canMonitor() { - return true; - } - @Override protected String getJobId(PipelineResult job) { return ((DataflowPipelineJob) job).getJobId(); diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkRunner.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkRunner.java index 8d4c1f1be1097..d311dc487a61b 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkRunner.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkRunner.java @@ -157,11 +157,6 @@ private PubsubHelper getPubsub() { */ protected abstract int maxNumWorkers(); - /** - * Return true if runner can monitor running jobs. - */ - protected abstract boolean canMonitor(); - /** * Return the current value for a long counter, or -1 if can't be retrieved. */ @@ -1089,10 +1084,6 @@ private void modelResultRates(NexmarkQueryModel model) { */ @Nullable public NexmarkPerf run(NexmarkConfiguration runConfiguration) { - if (options.getMonitorJobs() && !canMonitor()) { - throw new RuntimeException("Cannot use --monitorJobs with this runner since it does not " - + "support monitoring."); - } if (options.getManageResources() && !options.getMonitorJobs()) { throw new RuntimeException("If using --manageResources then must also use --monitorJobs."); } diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkSparkRunner.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkSparkRunner.java index 32fee30639330..30ae9caca3287 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkSparkRunner.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkSparkRunner.java @@ -36,11 +36,6 @@ protected int maxNumWorkers() { return 5; } - @Override - protected boolean canMonitor() { - return true; - } - @Override protected void invokeBuilderForPublishOnlyPipeline( PipelineBuilder builder) { diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query5.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query5.java index 9020494e6e611..2c9fb9bd264e3 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query5.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query5.java @@ -67,7 +67,6 @@ private PCollection applyTyped(PCollection events) { // Count the number of bids per auction id. .apply(Count.perElement()) - //TODO replace by simple key // We'll want to keep all auctions with the maximal number of bids. // Start by lifting each into a singleton list. .apply(name + ".ToSingletons", diff --git a/integration/java/nexmark/src/main/resources/log4j.properties b/integration/java/nexmark/src/main/resources/log4j.properties index 9d20aeadf6a62..bc09794657652 100644 --- a/integration/java/nexmark/src/main/resources/log4j.properties +++ b/integration/java/nexmark/src/main/resources/log4j.properties @@ -35,8 +35,17 @@ log4j.logger.org.apache.spark=WARN log4j.logger.org.spark-project=WARN log4j.logger.io.netty=INFO +# Settings to quiet flink logs log4j.logger.org.apache.flink=WARN +# Settings to quiet apex logs +log4j.logger.org.apache.beam.runners.apex=INFO +log4j.logger.com.datatorrent=ERROR +log4j.logger.org.apache.hadoop.metrics2=WARN +log4j.logger.org.apache.commons=WARN +log4j.logger.org.apache.hadoop.security=WARN +log4j.logger.org.apache.hadoop.util=WARN + # SPARK-9183: Settings to avoid annoying messages when looking up nonexistent UDFs in SparkSQL with Hive support log4j.logger.org.apache.hadoop.hive.metastore.RetryingHMSHandler=FATAL log4j.logger.org.apache.hadoop.hive.ql.exec.FunctionRegistry=ERROR diff --git a/integration/java/nexmark/src/test/java/org/apache/beam/integration/nexmark/UnboundedEventSourceTest.java b/integration/java/nexmark/src/test/java/org/apache/beam/integration/nexmark/UnboundedEventSourceTest.java index 02761d6448967..35b3aeda978d8 100644 --- a/integration/java/nexmark/src/test/java/org/apache/beam/integration/nexmark/UnboundedEventSourceTest.java +++ b/integration/java/nexmark/src/test/java/org/apache/beam/integration/nexmark/UnboundedEventSourceTest.java @@ -87,8 +87,8 @@ public void resumeFromCheckpoint() throws IOException { Generator modelGenerator = new Generator(config); EventIdChecker checker = new EventIdChecker(); - Pipeline p = TestPipeline.create(); - PipelineOptions options = p.getOptions(); + PipelineOptions options = TestPipeline.testingPipelineOptions(); + Pipeline p = TestPipeline.create(options); UnboundedEventSource source = new UnboundedEventSource(config, 1, 0, false); UnboundedReader reader = source.createReader(options, null); From a7f9f7d0784d9ba1f53ac4a0b49d2d81700720d0 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Isma=C3=ABl=20Mej=C3=ADa?= Date: Thu, 23 Mar 2017 19:32:45 +0100 Subject: [PATCH 294/346] Refactor classes into packages The new hierarchy has logically based packages for: - drivers - io - model - queries - sources --- .../java/nexmark/BEAM_ON_FLINK_ON_GCP.md | 2 +- integration/java/nexmark/README.md | 88 ++++++++++++------- integration/java/nexmark/pom.xml | 22 +---- .../nexmark/AbstractSimulator.java | 2 +- .../beam/integration/nexmark/Monitor.java | 1 + .../nexmark/NexmarkConfiguration.java | 6 +- .../integration/nexmark/NexmarkDriver.java | 5 +- .../{Options.java => NexmarkOptions.java} | 2 +- .../beam/integration/nexmark/NexmarkPerf.java | 4 +- .../integration/nexmark/NexmarkQuery.java | 7 +- .../nexmark/NexmarkQueryModel.java | 19 ++-- .../integration/nexmark/NexmarkRunner.java | 40 +++++++-- .../integration/nexmark/NexmarkSuite.java | 2 +- .../integration/nexmark/NexmarkUtils.java | 25 +++++- .../beam/integration/nexmark/WinningBids.java | 11 ++- .../nexmark/WinningBidsSimulator.java | 4 + .../{ => drivers}/NexmarkApexDriver.java | 6 +- .../{ => drivers}/NexmarkApexRunner.java | 6 +- .../{ => drivers}/NexmarkDirectDriver.java | 6 +- .../{ => drivers}/NexmarkDirectRunner.java | 4 +- .../{ => drivers}/NexmarkFlinkDriver.java | 6 +- .../{ => drivers}/NexmarkFlinkRunner.java | 4 +- .../{ => drivers}/NexmarkGoogleDriver.java | 9 +- .../{ => drivers}/NexmarkGoogleRunner.java | 6 +- .../{ => drivers}/NexmarkSparkDriver.java | 10 ++- .../{ => drivers}/NexmarkSparkRunner.java | 4 +- .../nexmark/drivers/package-info.java | 22 +++++ .../nexmark/{ => io}/PubsubHelper.java | 3 +- .../integration/nexmark/io/package-info.java | 22 +++++ .../nexmark/{ => model}/Auction.java | 3 +- .../nexmark/{ => model}/AuctionBid.java | 4 +- .../nexmark/{ => model}/AuctionCount.java | 5 +- .../nexmark/{ => model}/AuctionPrice.java | 5 +- .../integration/nexmark/{ => model}/Bid.java | 3 +- .../nexmark/{ => model}/BidsPerSession.java | 3 +- .../nexmark/{ => model}/CategoryPrice.java | 5 +- .../integration/nexmark/{ => model}/Done.java | 3 +- .../nexmark/{ => model}/Event.java | 2 +- .../nexmark/{ => model}/IdNameReserve.java | 5 +- .../nexmark/{ => model}/KnownSize.java | 2 +- .../nexmark/{ => model}/NameCityStateId.java | 5 +- .../nexmark/{ => model}/Person.java | 3 +- .../nexmark/{ => model}/SellerPrice.java | 5 +- .../nexmark/model/package-info.java | 22 +++++ .../integration/nexmark/package-info.java | 2 +- .../nexmark/{ => queries}/Query0.java | 7 +- .../nexmark/{ => queries}/Query0Model.java | 7 +- .../nexmark/{ => queries}/Query1.java | 10 ++- .../nexmark/{ => queries}/Query10.java | 12 ++- .../nexmark/{ => queries}/Query11.java | 11 ++- .../nexmark/{ => queries}/Query12.java | 11 ++- .../nexmark/{ => queries}/Query1Model.java | 8 +- .../nexmark/{ => queries}/Query2.java | 11 ++- .../nexmark/{ => queries}/Query2Model.java | 9 +- .../nexmark/{ => queries}/Query3.java | 13 ++- .../nexmark/{ => queries}/Query3Model.java | 10 ++- .../nexmark/{ => queries}/Query4.java | 15 +++- .../nexmark/{ => queries}/Query4Model.java | 13 ++- .../nexmark/{ => queries}/Query5.java | 12 ++- .../nexmark/{ => queries}/Query5Model.java | 10 ++- .../nexmark/{ => queries}/Query6.java | 16 +++- .../nexmark/{ => queries}/Query6Model.java | 13 ++- .../nexmark/{ => queries}/Query7.java | 10 ++- .../nexmark/{ => queries}/Query7Model.java | 8 +- .../nexmark/{ => queries}/Query8.java | 13 ++- .../nexmark/{ => queries}/Query8Model.java | 12 ++- .../nexmark/{ => queries}/Query9.java | 11 ++- .../nexmark/{ => queries}/Query9Model.java | 6 +- .../nexmark/queries/package-info.java | 22 +++++ .../{ => sources}/BoundedEventSource.java | 7 +- .../nexmark/{ => sources}/Generator.java | 6 +- .../{ => sources}/GeneratorConfig.java | 6 +- .../{ => sources}/UnboundedEventSource.java | 6 +- .../nexmark/sources/package-info.java | 22 +++++ .../src/main/resources/log4j.properties | 4 + .../nexmark/{ => queries}/QueryTest.java | 16 ++-- .../{ => sources}/BoundedEventSourceTest.java | 3 +- .../nexmark/{ => sources}/GeneratorTest.java | 3 +- .../UnboundedEventSourceTest.java | 4 +- 79 files changed, 578 insertions(+), 194 deletions(-) rename integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/{Options.java => NexmarkOptions.java} (99%) rename integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/{ => drivers}/NexmarkApexDriver.java (86%) rename integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/{ => drivers}/NexmarkApexRunner.java (87%) rename integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/{ => drivers}/NexmarkDirectDriver.java (86%) rename integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/{ => drivers}/NexmarkDirectRunner.java (94%) rename integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/{ => drivers}/NexmarkFlinkDriver.java (86%) rename integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/{ => drivers}/NexmarkFlinkRunner.java (93%) rename integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/{ => drivers}/NexmarkGoogleDriver.java (84%) rename integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/{ => drivers}/NexmarkGoogleRunner.java (95%) rename integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/{ => drivers}/NexmarkSparkDriver.java (81%) rename integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/{ => drivers}/NexmarkSparkRunner.java (93%) create mode 100644 integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/drivers/package-info.java rename integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/{ => io}/PubsubHelper.java (98%) create mode 100644 integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/io/package-info.java rename integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/{ => model}/Auction.java (98%) rename integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/{ => model}/AuctionBid.java (94%) rename integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/{ => model}/AuctionCount.java (95%) rename integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/{ => model}/AuctionPrice.java (95%) rename integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/{ => model}/Bid.java (98%) rename integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/{ => model}/BidsPerSession.java (96%) rename integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/{ => model}/CategoryPrice.java (95%) rename integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/{ => model}/Done.java (95%) rename integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/{ => model}/Event.java (99%) rename integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/{ => model}/IdNameReserve.java (95%) rename integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/{ => model}/KnownSize.java (94%) rename integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/{ => model}/NameCityStateId.java (96%) rename integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/{ => model}/Person.java (98%) rename integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/{ => model}/SellerPrice.java (95%) create mode 100644 integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/package-info.java rename integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/{ => queries}/Query0.java (88%) rename integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/{ => queries}/Query0Model.java (84%) rename integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/{ => queries}/Query1.java (82%) rename integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/{ => queries}/Query10.java (97%) rename integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/{ => queries}/Query11.java (85%) rename integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/{ => queries}/Query12.java (85%) rename integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/{ => queries}/Query1Model.java (85%) rename integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/{ => queries}/Query2.java (83%) rename integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/{ => queries}/Query2Model.java (83%) rename integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/{ => queries}/Query3.java (94%) rename integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/{ => queries}/Query3Model.java (88%) rename integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/{ => queries}/Query4.java (86%) rename integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/{ => queries}/Query4Model.java (90%) rename integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/{ => queries}/Query5.java (90%) rename integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/{ => queries}/Query5Model.java (92%) rename integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/{ => queries}/Query6.java (88%) rename integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/{ => queries}/Query6Model.java (87%) rename integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/{ => queries}/Query7.java (88%) rename integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/{ => queries}/Query7Model.java (90%) rename integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/{ => queries}/Query8.java (86%) rename integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/{ => queries}/Query8Model.java (89%) rename integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/{ => queries}/Query9.java (73%) rename integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/{ => queries}/Query9Model.java (82%) create mode 100644 integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/package-info.java rename integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/{ => sources}/BoundedEventSource.java (96%) rename integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/{ => sources}/Generator.java (98%) rename integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/{ => sources}/GeneratorConfig.java (97%) rename integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/{ => sources}/UnboundedEventSource.java (97%) create mode 100644 integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/sources/package-info.java rename integration/java/nexmark/src/test/java/org/apache/beam/integration/nexmark/{ => queries}/QueryTest.java (87%) rename integration/java/nexmark/src/test/java/org/apache/beam/integration/nexmark/{ => sources}/BoundedEventSourceTest.java (95%) rename integration/java/nexmark/src/test/java/org/apache/beam/integration/nexmark/{ => sources}/GeneratorTest.java (96%) rename integration/java/nexmark/src/test/java/org/apache/beam/integration/nexmark/{ => sources}/UnboundedEventSourceTest.java (95%) diff --git a/integration/java/nexmark/BEAM_ON_FLINK_ON_GCP.md b/integration/java/nexmark/BEAM_ON_FLINK_ON_GCP.md index d1b51e8a02be9..6a7fd3432c5db 100644 --- a/integration/java/nexmark/BEAM_ON_FLINK_ON_GCP.md +++ b/integration/java/nexmark/BEAM_ON_FLINK_ON_GCP.md @@ -243,7 +243,7 @@ $GCLOUD compute ssh \ --zone=$ZONE \ $MASTER \ --command "~/$FLINK_VER/bin/flink run \ - -c org.apache.beam.integration.nexmark.NexmarkFlinkDriver \ + -c org.apache.beam.integration.nexmark.drivers.NexmarkFlinkDriver \ ~/$FLINK_VER/lib/java-integration-all-bundled-0.2.0-incubating-SNAPSHOT.jar \ --project=$PROJECT \ --streaming=true \ diff --git a/integration/java/nexmark/README.md b/integration/java/nexmark/README.md index 7a91ab22e975a..a3549f4173987 100644 --- a/integration/java/nexmark/README.md +++ b/integration/java/nexmark/README.md @@ -74,14 +74,15 @@ We have augmented the original queries with five more: The queries can be executed using a 'Driver' for a given backend. Currently the supported drivers are: +* **NexmarkApexDriver** for running via the Apex runner. * **NexmarkDirectDriver** for running locally on a single machine. -* **NexmarkGoogleDriver** for running on the Google Cloud Dataflow - service. Requires a Google Cloud account. +* **NexmarkGoogleDriver** for running on the Google Cloud Dataflow service. + Requires a Google Cloud account. * **NexmarkFlinkDriver** for running on a Flink cluster. Requires the cluster to be established and the Nexmark jar to be distributed to each worker. * **NexmarkSparkDriver** for running on a Spark cluster. - + Other drivers are straightforward. Test data is deterministically synthesized on demand. The test @@ -103,9 +104,21 @@ the Google Cloud Dataflow driver. # Configuration -Common configuration parameters: +## Common configuration parameters + +Decide if batch or streaming: + + --streaming=true + +Number of events generators + + --numEventGenerators=4 + +Run query N -Available Suites: + --query=N + +## Available Suites - DEFAULT: Test default configuration with query 0. - SMOKE: Run the 12 default configurations. @@ -114,32 +127,39 @@ Available Suites: --suite=SMOKE -Decide if batch or streaming: - - --streaming=true +### Apex specific configuration -Number of events generators + --suite=SMOKE --manageResources=false --monitorJobs=true - --numEventGenerators=4 +### Dataflow specific configuration -## Apex specific configuration + --query=0 --suite=SMOKE --manageResources=false --monitorJobs=true \ + --enforceEncodability=false --enforceImmutability=false + --project= \ + --zone= \ + --workerMachineType=n1-highmem-8 \ + --stagingLocation= ---suite=SMOKE --manageResources=false --monitorJobs=true + --runner=BlockingDataflowRunner \ + --tempLocation=gs://talend-imejia/nexmark/temp/ \ + --stagingLocation=gs://talend-imejia/nexmark/temp/staging/ \ + --filesToStage=target/beam-integration-java-0.7.0-SNAPSHOT.jar -## Direct specific configuration +### Direct specific configuration ---suite=SMOKE --manageResources=false --monitorJobs=true \ ---enforceEncodability=false --enforceImmutability=false + --suite=SMOKE --manageResources=false --monitorJobs=true \ + --enforceEncodability=false --enforceImmutability=false -## Flink specific configuration +### Flink specific configuration ---suite=SMOKE --manageResources=false --monitorJobs=true \ ---flinkMaster=local + --suite=SMOKE --manageResources=false --monitorJobs=true \ + --flinkMaster=[local] --parallelism=#numcores -## Spark specific configuration +### Spark specific configuration ---suite=SMOKE --manageResources=false --monitorJobs=true --sparkMaster=local \ --Dspark.ui.enabled=false -DSPARK_LOCAL_IP=localhost -Dsun.io.serialization.extendedDebugInfo=true + --suite=SMOKE --manageResources=false --monitorJobs=true \ + --sparkMaster=local \ + -Dspark.ui.enabled=false -DSPARK_LOCAL_IP=localhost -Dsun.io.serialization.extendedDebugInfo=true # Current Status @@ -149,19 +169,19 @@ Open issues are tracked [here](https://github.com../../../../../issues): | Query | Direct | Spark | Flink | Apex | | ----: | ------------------------------ | ------------------------------ | ------------------------------ | ------------------------------- | -| 0 | ok | [#1](../../../../../issues/1) | ok | ok | -| 1 | ok | [#1](../../../../../issues/1) | ok | ok | -| 2 | ok | [#1](../../../../../issues/1) | ok | ok | +| 0 | ok | ok | ok | ok | +| 1 | ok | ok | ok | ok | +| 2 | ok | ok | ok | ok | | 3 | [#7](../../../../../issues/7) | [#7](../../../../../issues/7) | [#7](../../../../../issues/7) | [#7](../../../../../issues/7) | | 4 | ok | ok | [#2](../../../../../issues/2) | ok | -| 5 | ok | [#3](../../../../../issues/3) | ok | ok | +| 5 | ok | ok | ok | ok | | 6 | ok | ok | [#2](../../../../../issues/2) | ok | -| 7 | ok | [#1](../../../../../issues/1) | ok | [#24](../../../../../issues/24) | -| 8 | ok | [#1](../../../../../issues/1) | ok | ok | +| 7 | ok | ok | ok | [#24](../../../../../issues/24) | +| 8 | ok | ok | ok | ok | | 9 | ok | ok | [#2](../../../../../issues/2) | ok | -| 10 | [#5](../../../../../issues/5) | [#4](../../../../../issues/4) | ok | ok | -| 11 | ok | [#1](../../../../../issues/1) | ok | ok | -| 12 | ok | [#1](../../../../../issues/1) | ok | ok | +| 10 | [#5](../../../../../issues/5) | ok | ok | ok | +| 11 | ok | ok | ok | ok | +| 12 | ok | ok | ok | ok | ## Streaming / Synthetic / Local @@ -205,11 +225,11 @@ Batch Mode -Dexec.classpathScope="test" - mvn exec:java -Dexec.mainClass=org.apache.beam.integration.nexmark.NexmarkDirectDriver -Dexec.args="--suite=SMOKE --streaming=false --numEventGenerators=4 --manageResources=false --monitorJobs=false --enforceEncodability=false --enforceImmutability=false" + mvn exec:java -Dexec.mainClass=org.apache.beam.integration.nexmark.drivers.NexmarkDirectDriver -Dexec.args="--suite=SMOKE --streaming=false --numEventGenerators=4 --manageResources=false --monitorJobs=false --enforceEncodability=false --enforceImmutability=false" Streaming Mode - mvn exec:java -Dexec.mainClass=org.apache.beam.integration.nexmark.NexmarkDirectDriver -Dexec.args="--suite=SMOKE --streaming=true --numEventGenerators=4 --manageResources=false --monitorJobs=false --enforceEncodability=false --enforceImmutability=false" + mvn exec:java -Dexec.mainClass=org.apache.beam.integration.nexmark.drivers.NexmarkDirectDriver -Dexec.args="--suite=SMOKE --streaming=true --numEventGenerators=4 --manageResources=false --monitorJobs=false --enforceEncodability=false --enforceImmutability=false" ## Running on Google Cloud Dataflow @@ -218,7 +238,7 @@ service. ``` java -cp integration/java/target/java-integration-all-bundled-0.2.0-incubating-SNAPSHOT.jar \ - org.apache.beam.integration.nexmark.NexmarkGoogleDriver \ + org.apache.beam.integration.nexmark.drivers.NexmarkGoogleDriver \ --project= \ --zone= \ --workerMachineType=n1-highmem-8 \ @@ -251,7 +271,7 @@ java -cp integration/java/target/java-integration-all-bundled-0.2.0-incubating-S ``` java -cp integration/java/target/java-integration-all-bundled-0.2.0-incubating-SNAPSHOT.jar \ - org.apache.beam.integration.nexmark.NexmarkGoogleDriver \ + org.apache.beam.integration.nexmark.drivers.NexmarkGoogleDriver \ --project= \ --zone= \ --workerMachineType=n1-highmem-8 \ diff --git a/integration/java/nexmark/pom.xml b/integration/java/nexmark/pom.xml index 0ecc29839ce17..7cd7d392a20cf 100644 --- a/integration/java/nexmark/pom.xml +++ b/integration/java/nexmark/pom.xml @@ -59,11 +59,6 @@ - - org.apache.maven.plugins - maven-dependency-plugin - - org.apache.maven.plugins maven-checkstyle-plugin @@ -139,7 +134,6 @@ org.apache.maven.plugins maven-dependency-plugin - 2.10 analyze-only @@ -196,11 +190,13 @@ org.apache.spark spark-core_2.10 ${spark.version} + runtime org.apache.spark spark-streaming_2.10 ${spark.version} + runtime @@ -214,12 +210,6 @@ ${apex.kryo.version} runtime - - com.fasterxml.jackson.core - jackson-databind - ${jackson.version} - runtime - org.codehaus.jackson jackson-mapper-asl @@ -244,6 +234,7 @@ com.google.apis google-api-services-dataflow ${dataflow.version} + runtime @@ -289,13 +280,6 @@ org.hamcrest hamcrest-all - ${hamcrest.version} - - - - org.hamcrest - hamcrest-library - ${hamcrest.version} diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/AbstractSimulator.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/AbstractSimulator.java index c08cdd349adde..b01284267abcb 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/AbstractSimulator.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/AbstractSimulator.java @@ -34,7 +34,7 @@ * @param Type of input elements. * @param Type of output elements. */ -abstract class AbstractSimulator { +public abstract class AbstractSimulator { /** Window size for action bucket sampling. */ public static final Duration WINDOW_SIZE = Duration.standardMinutes(1); diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Monitor.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Monitor.java index 02660bfe33121..6370e4158bd28 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Monitor.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Monitor.java @@ -19,6 +19,7 @@ import java.io.Serializable; +import org.apache.beam.integration.nexmark.model.KnownSize; import org.apache.beam.sdk.transforms.Aggregator; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.Max; diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkConfiguration.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkConfiguration.java index 09436640a0f4f..e2890eda644fa 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkConfiguration.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkConfiguration.java @@ -29,7 +29,7 @@ * programmatically. We only capture properties which may influence the resulting * pipeline performance, as captured by {@link NexmarkPerf}. */ -class NexmarkConfiguration implements Serializable { +public class NexmarkConfiguration implements Serializable { public static final NexmarkConfiguration DEFAULT = new NexmarkConfiguration(); /** If {@literal true}, include additional debugging and monitoring stats. */ @@ -228,7 +228,7 @@ class NexmarkConfiguration implements Serializable { /** * Replace any properties of this configuration which have been supplied by the command line. */ - public void overrideFromOptions(Options options) { + public void overrideFromOptions(NexmarkOptions options) { if (options.getDebug() != null) { debug = options.getDebug(); } @@ -511,8 +511,6 @@ public String toString() { /** * Parse an object from {@code string}. - * - * @throws IOException */ public static NexmarkConfiguration fromString(String string) { try { diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkDriver.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkDriver.java index e6a7b0b44a35f..4714124a7b041 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkDriver.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkDriver.java @@ -28,6 +28,9 @@ import java.util.Map; import javax.annotation.Nullable; +import org.apache.beam.integration.nexmark.model.Auction; +import org.apache.beam.integration.nexmark.model.Bid; +import org.apache.beam.integration.nexmark.model.Person; import org.joda.time.Duration; import org.joda.time.Instant; @@ -48,7 +51,7 @@ *

                See * http://datalab.cs.pdx.edu/niagaraST/NEXMark/ */ -public class NexmarkDriver { +public class NexmarkDriver { /** * Entry point. diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Options.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkOptions.java similarity index 99% rename from integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Options.java rename to integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkOptions.java index 388473d1d669e..1be974fd4202b 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Options.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkOptions.java @@ -26,7 +26,7 @@ /** * Command line flags. */ -public interface Options extends PubsubOptions { +public interface NexmarkOptions extends PubsubOptions { @Description("Which suite to run. Default is to use command line arguments for one job.") @Default.Enum("DEFAULT") NexmarkSuite getSuite(); diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkPerf.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkPerf.java index 37b621311ab92..e7f59c84d2a8c 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkPerf.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkPerf.java @@ -27,7 +27,7 @@ /** * Summary of performance for a particular run of a configuration. */ -class NexmarkPerf { +public class NexmarkPerf { /** * A sample of the number of events and number of results (if known) generated at * a particular time. @@ -177,8 +177,6 @@ public String toString() { /** * Parse a {@link NexmarkPerf} object from JSON {@code string}. - * - * @throws IOException */ public static NexmarkPerf fromString(String string) { try { diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkQuery.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkQuery.java index 5ef419126db26..c268a3bed84bd 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkQuery.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkQuery.java @@ -18,7 +18,11 @@ package org.apache.beam.integration.nexmark; import javax.annotation.Nullable; - +import org.apache.beam.integration.nexmark.model.Auction; +import org.apache.beam.integration.nexmark.model.Bid; +import org.apache.beam.integration.nexmark.model.Event; +import org.apache.beam.integration.nexmark.model.KnownSize; +import org.apache.beam.integration.nexmark.model.Person; import org.apache.beam.sdk.transforms.Aggregator; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.Filter; @@ -29,7 +33,6 @@ import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.TimestampedValue; import org.apache.beam.sdk.values.TupleTag; - import org.joda.time.Instant; /** diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkQueryModel.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkQueryModel.java index f265e0d96e9cd..b2b1826a8106c 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkQueryModel.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkQueryModel.java @@ -17,11 +17,6 @@ */ package org.apache.beam.integration.nexmark; -import static org.hamcrest.CoreMatchers.containsString; -import static org.hamcrest.CoreMatchers.equalTo; -import static org.hamcrest.CoreMatchers.hasItems; -import org.hamcrest.collection.IsIterableContainingInAnyOrder; - import java.io.Serializable; import java.util.ArrayList; import java.util.Collection; @@ -30,10 +25,11 @@ import java.util.List; import java.util.Set; +import org.apache.beam.integration.nexmark.model.KnownSize; import org.apache.beam.sdk.transforms.SerializableFunction; import org.apache.beam.sdk.values.TimestampedValue; -import org.hamcrest.core.IsCollectionContaining; + import org.hamcrest.core.IsEqual; import org.joda.time.Duration; import org.joda.time.Instant; @@ -107,15 +103,18 @@ protected Iterable> relevantResults( /** Return assertion to use on results of pipeline for this query. */ public SerializableFunction>, Void> assertionFor() { final Collection expectedStrings = toCollection(simulator().results()); - final String[] expectedStringsArray = expectedStrings.toArray(new String[expectedStrings.size()]); + final String[] expectedStringsArray = + expectedStrings.toArray(new String[expectedStrings.size()]); return new SerializableFunction>, Void>() { @Override public Void apply(Iterable> actual) { - Collection actualStrings = toCollection(relevantResults(actual).iterator()); - Assert.assertThat("wrong pipeline output", actualStrings, IsEqual.equalTo(expectedStrings)); + Collection actualStrings = toCollection(relevantResults(actual).iterator()); + Assert.assertThat("wrong pipeline output", actualStrings, + IsEqual.equalTo(expectedStrings)); //compare without order -// Assert.assertThat("wrong pipeline output", actualStrings, IsIterableContainingInAnyOrder.containsInAnyOrder(expectedStringsArray)); +// Assert.assertThat("wrong pipeline output", actualStrings, +// IsIterableContainingInAnyOrder.containsInAnyOrder(expectedStringsArray)); return null; } }; diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkRunner.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkRunner.java index d311dc487a61b..e8d791f510b69 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkRunner.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkRunner.java @@ -26,7 +26,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; @@ -35,6 +34,35 @@ import java.util.List; import java.util.concurrent.ThreadLocalRandom; import javax.annotation.Nullable; +import org.apache.beam.integration.nexmark.io.PubsubHelper; +import org.apache.beam.integration.nexmark.model.Auction; +import org.apache.beam.integration.nexmark.model.Bid; +import org.apache.beam.integration.nexmark.model.Event; +import org.apache.beam.integration.nexmark.model.KnownSize; +import org.apache.beam.integration.nexmark.model.Person; +import org.apache.beam.integration.nexmark.queries.Query0; +import org.apache.beam.integration.nexmark.queries.Query0Model; +import org.apache.beam.integration.nexmark.queries.Query1; +import org.apache.beam.integration.nexmark.queries.Query10; +import org.apache.beam.integration.nexmark.queries.Query11; +import org.apache.beam.integration.nexmark.queries.Query12; +import org.apache.beam.integration.nexmark.queries.Query1Model; +import org.apache.beam.integration.nexmark.queries.Query2; +import org.apache.beam.integration.nexmark.queries.Query2Model; +import org.apache.beam.integration.nexmark.queries.Query3; +import org.apache.beam.integration.nexmark.queries.Query3Model; +import org.apache.beam.integration.nexmark.queries.Query4; +import org.apache.beam.integration.nexmark.queries.Query4Model; +import org.apache.beam.integration.nexmark.queries.Query5; +import org.apache.beam.integration.nexmark.queries.Query5Model; +import org.apache.beam.integration.nexmark.queries.Query6; +import org.apache.beam.integration.nexmark.queries.Query6Model; +import org.apache.beam.integration.nexmark.queries.Query7; +import org.apache.beam.integration.nexmark.queries.Query7Model; +import org.apache.beam.integration.nexmark.queries.Query8; +import org.apache.beam.integration.nexmark.queries.Query8Model; +import org.apache.beam.integration.nexmark.queries.Query9; +import org.apache.beam.integration.nexmark.queries.Query9Model; import org.apache.beam.sdk.AggregatorRetrievalException; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.PipelineResult; @@ -56,7 +84,7 @@ /** * Run a single Nexmark query using a given configuration. */ -public abstract class NexmarkRunner { +public abstract class NexmarkRunner { /** * Minimum number of samples needed for 'stead-state' rate calculation. */ @@ -84,7 +112,7 @@ public abstract class NexmarkRunner { */ private static final Duration STUCK_TERMINATE_DELAY = Duration.standardDays(3); /** - * Options shared by all runs. + * NexmarkOptions shared by all runs. */ protected final OptionT options; @@ -359,7 +387,7 @@ private NexmarkPerf currentPerf( return perf; } - String getJobId(PipelineResult job) { + protected String getJobId(PipelineResult job) { return ""; } @@ -461,7 +489,7 @@ enum MetricType { /** * Build and run a pipeline using specified options. */ - protected interface PipelineBuilder { + protected interface PipelineBuilder { void build(OptionT publishOnlyOptions); } @@ -966,7 +994,7 @@ private PCollection createSource(Pipeline p, final long now) { // We'll shutdown the publisher job when we notice the main job has finished. invokeBuilderForPublishOnlyPipeline(new PipelineBuilder() { @Override - public void build(Options publishOnlyOptions) { + public void build(NexmarkOptions publishOnlyOptions) { Pipeline sp = Pipeline.create(options); NexmarkUtils.setupPipeline(configuration.coderStrategy, sp); publisherMonitor = new Monitor(queryName, "publisher"); diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkSuite.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkSuite.java index bc47540b9a4b1..be7d7b80d3b7c 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkSuite.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkSuite.java @@ -100,7 +100,7 @@ private static List fullThrottle() { * with any set command line flags, except for --isStreaming which is only respected for * the {@link #DEFAULT} suite. */ - public Iterable getConfigurations(Options options) { + public Iterable getConfigurations(NexmarkOptions options) { Set results = new LinkedHashSet<>(); for (NexmarkConfiguration configuration : configurations) { NexmarkConfiguration result = configuration.clone(); diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkUtils.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkUtils.java index f7417d3c8513e..b0421a4c02cfc 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkUtils.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkUtils.java @@ -20,14 +20,30 @@ import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.ImmutableList; import com.google.common.hash.Hashing; - import java.io.IOException; import java.io.InputStream; import java.io.OutputStream; import java.nio.charset.StandardCharsets; import java.util.Iterator; import java.util.List; - +import org.apache.beam.integration.nexmark.model.Auction; +import org.apache.beam.integration.nexmark.model.AuctionBid; +import org.apache.beam.integration.nexmark.model.AuctionCount; +import org.apache.beam.integration.nexmark.model.AuctionPrice; +import org.apache.beam.integration.nexmark.model.Bid; +import org.apache.beam.integration.nexmark.model.BidsPerSession; +import org.apache.beam.integration.nexmark.model.CategoryPrice; +import org.apache.beam.integration.nexmark.model.Done; +import org.apache.beam.integration.nexmark.model.Event; +import org.apache.beam.integration.nexmark.model.IdNameReserve; +import org.apache.beam.integration.nexmark.model.KnownSize; +import org.apache.beam.integration.nexmark.model.NameCityStateId; +import org.apache.beam.integration.nexmark.model.Person; +import org.apache.beam.integration.nexmark.model.SellerPrice; +import org.apache.beam.integration.nexmark.sources.BoundedEventSource; +import org.apache.beam.integration.nexmark.sources.Generator; +import org.apache.beam.integration.nexmark.sources.GeneratorConfig; +import org.apache.beam.integration.nexmark.sources.UnboundedEventSource; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.coders.AvroCoder; import org.apache.beam.sdk.coders.ByteArrayCoder; @@ -67,7 +83,7 @@ public class NexmarkUtils { /** * Mapper for (de)serializing JSON. */ - static final ObjectMapper MAPPER = new ObjectMapper(); + public static final ObjectMapper MAPPER = new ObjectMapper(); /** * Possible sources for events. @@ -382,7 +398,8 @@ public static Iterator> standardEventIterator( */ public static PTransform> batchEventsSource( NexmarkConfiguration configuration) { - return Read.from(new BoundedEventSource(standardGeneratorConfig(configuration), configuration.numEventGenerators)); + return Read.from(new BoundedEventSource(standardGeneratorConfig(configuration), + configuration.numEventGenerators)); } /** diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/WinningBids.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/WinningBids.java index 594195ac5b0e3..9f1ddf898da25 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/WinningBids.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/WinningBids.java @@ -20,7 +20,6 @@ import static com.google.common.base.Preconditions.checkState; import com.fasterxml.jackson.annotation.JsonCreator; - import java.io.IOException; import java.io.InputStream; import java.io.OutputStream; @@ -31,7 +30,11 @@ import java.util.List; import java.util.Map; import java.util.TreeMap; - +import org.apache.beam.integration.nexmark.model.Auction; +import org.apache.beam.integration.nexmark.model.AuctionBid; +import org.apache.beam.integration.nexmark.model.Bid; +import org.apache.beam.integration.nexmark.model.Event; +import org.apache.beam.integration.nexmark.sources.GeneratorConfig; import org.apache.beam.sdk.coders.AtomicCoder; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.CoderException; @@ -45,10 +48,10 @@ import org.apache.beam.sdk.transforms.join.CoGbkResult; import org.apache.beam.sdk.transforms.join.CoGroupByKey; import org.apache.beam.sdk.transforms.join.KeyedPCollectionTuple; -import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.IntervalWindow; import org.apache.beam.sdk.transforms.windowing.Window; import org.apache.beam.sdk.transforms.windowing.WindowFn; +import org.apache.beam.sdk.transforms.windowing.WindowMappingFn; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; import org.joda.time.Instant; @@ -249,7 +252,7 @@ public Coder windowCoder() { } @Override - public AuctionOrBidWindow getSideInputWindow(BoundedWindow window) { + public WindowMappingFn getDefaultWindowMappingFn() { throw new UnsupportedOperationException("AuctionWindowFn not supported for side inputs"); } diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/WinningBidsSimulator.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/WinningBidsSimulator.java index dc8094b3c0371..e7f51b776590f 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/WinningBidsSimulator.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/WinningBidsSimulator.java @@ -26,6 +26,10 @@ import java.util.TreeSet; import javax.annotation.Nullable; +import org.apache.beam.integration.nexmark.model.Auction; +import org.apache.beam.integration.nexmark.model.AuctionBid; +import org.apache.beam.integration.nexmark.model.Bid; +import org.apache.beam.integration.nexmark.model.Event; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.values.TimestampedValue; import org.joda.time.Instant; diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkApexDriver.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/drivers/NexmarkApexDriver.java similarity index 86% rename from integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkApexDriver.java rename to integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/drivers/NexmarkApexDriver.java index 4c2721e220027..265ccf747460c 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkApexDriver.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/drivers/NexmarkApexDriver.java @@ -15,8 +15,10 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.integration.nexmark; +package org.apache.beam.integration.nexmark.drivers; +import org.apache.beam.integration.nexmark.NexmarkDriver; +import org.apache.beam.integration.nexmark.NexmarkOptions; import org.apache.beam.runners.apex.ApexPipelineOptions; import org.apache.beam.runners.apex.ApexRunner; import org.apache.beam.sdk.options.PipelineOptionsFactory; @@ -28,7 +30,7 @@ public class NexmarkApexDriver extends NexmarkDriver LONG_CODER = VarLongCoder.of(); diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/AuctionPrice.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/AuctionPrice.java similarity index 95% rename from integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/AuctionPrice.java rename to integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/AuctionPrice.java index 9bdf11cd7f6a7..f365cc84964ab 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/AuctionPrice.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/AuctionPrice.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.integration.nexmark; +package org.apache.beam.integration.nexmark.model; import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.core.JsonProcessingException; @@ -25,13 +25,14 @@ import java.io.OutputStream; import java.io.Serializable; +import org.apache.beam.integration.nexmark.NexmarkUtils; 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.VarLongCoder; /** - * Result of {@link Query2}. + * Result of Query2. */ public class AuctionPrice implements KnownSize, Serializable { private static final Coder LONG_CODER = VarLongCoder.of(); diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Bid.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/Bid.java similarity index 98% rename from integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Bid.java rename to integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/Bid.java index 04fcfdd043a12..59a33c1cf10ac 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Bid.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/Bid.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.integration.nexmark; +package org.apache.beam.integration.nexmark.model; import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.core.JsonProcessingException; @@ -26,6 +26,7 @@ import java.io.Serializable; import java.util.Comparator; +import org.apache.beam.integration.nexmark.NexmarkUtils; import org.apache.beam.sdk.coders.AtomicCoder; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.CoderException; diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/BidsPerSession.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/BidsPerSession.java similarity index 96% rename from integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/BidsPerSession.java rename to integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/BidsPerSession.java index c6b0fe3a8b3d2..7c4dfae56c9df 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/BidsPerSession.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/BidsPerSession.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.integration.nexmark; +package org.apache.beam.integration.nexmark.model; import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.core.JsonProcessingException; @@ -25,6 +25,7 @@ import java.io.OutputStream; import java.io.Serializable; +import org.apache.beam.integration.nexmark.NexmarkUtils; import org.apache.beam.sdk.coders.AtomicCoder; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.CoderException; diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/CategoryPrice.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/CategoryPrice.java similarity index 95% rename from integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/CategoryPrice.java rename to integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/CategoryPrice.java index c83fb178340d2..6512cc1301d6d 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/CategoryPrice.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/CategoryPrice.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.integration.nexmark; +package org.apache.beam.integration.nexmark.model; import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.core.JsonProcessingException; @@ -25,6 +25,7 @@ import java.io.OutputStream; import java.io.Serializable; +import org.apache.beam.integration.nexmark.NexmarkUtils; import org.apache.beam.sdk.coders.AtomicCoder; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.CoderException; @@ -32,7 +33,7 @@ import org.apache.beam.sdk.coders.VarLongCoder; /** - * Result of {@link Query4}. + * Result of Query4. */ public class CategoryPrice implements KnownSize, Serializable { private static final Coder LONG_CODER = VarLongCoder.of(); diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Done.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/Done.java similarity index 95% rename from integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Done.java rename to integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/Done.java index 3a045f9c9cb59..6009463970feb 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Done.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/Done.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.integration.nexmark; +package org.apache.beam.integration.nexmark.model; import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.core.JsonProcessingException; @@ -25,6 +25,7 @@ import java.io.OutputStream; import java.io.Serializable; +import org.apache.beam.integration.nexmark.NexmarkUtils; import org.apache.beam.sdk.coders.AtomicCoder; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.CoderException; diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Event.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/Event.java similarity index 99% rename from integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Event.java rename to integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/Event.java index 769cedda9df22..8a278bfe3c32d 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Event.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/Event.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.integration.nexmark; +package org.apache.beam.integration.nexmark.model; import java.io.IOException; import java.io.InputStream; diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/IdNameReserve.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/IdNameReserve.java similarity index 95% rename from integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/IdNameReserve.java rename to integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/IdNameReserve.java index 21fa3f402a44a..5d22651500f59 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/IdNameReserve.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/IdNameReserve.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.integration.nexmark; +package org.apache.beam.integration.nexmark.model; import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.core.JsonProcessingException; @@ -25,6 +25,7 @@ import java.io.OutputStream; import java.io.Serializable; +import org.apache.beam.integration.nexmark.NexmarkUtils; import org.apache.beam.sdk.coders.AtomicCoder; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.CoderException; @@ -32,7 +33,7 @@ import org.apache.beam.sdk.coders.VarLongCoder; /** - * Result type of {@link Query8}. + * Result type of Query8. */ public class IdNameReserve implements KnownSize, Serializable { private static final Coder LONG_CODER = VarLongCoder.of(); diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/KnownSize.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/KnownSize.java similarity index 94% rename from integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/KnownSize.java rename to integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/KnownSize.java index 2093c487a7295..c742eac2105cc 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/KnownSize.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/KnownSize.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.integration.nexmark; +package org.apache.beam.integration.nexmark.model; /** * Interface for elements which can quickly estimate their encoded byte size. diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NameCityStateId.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/NameCityStateId.java similarity index 96% rename from integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NameCityStateId.java rename to integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/NameCityStateId.java index fe4687bf3ade5..ac22879d5ceec 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NameCityStateId.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/NameCityStateId.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.integration.nexmark; +package org.apache.beam.integration.nexmark.model; import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.core.JsonProcessingException; @@ -25,6 +25,7 @@ import java.io.OutputStream; import java.io.Serializable; +import org.apache.beam.integration.nexmark.NexmarkUtils; import org.apache.beam.sdk.coders.AtomicCoder; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.CoderException; @@ -32,7 +33,7 @@ import org.apache.beam.sdk.coders.VarLongCoder; /** - * Result of {@link Query3}. + * Result of Query3. */ public class NameCityStateId implements KnownSize, Serializable { private static final Coder LONG_CODER = VarLongCoder.of(); diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Person.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/Person.java similarity index 98% rename from integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Person.java rename to integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/Person.java index 251a6eeb85022..85c71839b32dc 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Person.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/Person.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.integration.nexmark; +package org.apache.beam.integration.nexmark.model; import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.core.JsonProcessingException; @@ -25,6 +25,7 @@ import java.io.OutputStream; import java.io.Serializable; +import org.apache.beam.integration.nexmark.NexmarkUtils; import org.apache.beam.sdk.coders.AtomicCoder; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.CoderException; diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/SellerPrice.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/SellerPrice.java similarity index 95% rename from integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/SellerPrice.java rename to integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/SellerPrice.java index 4081287c88b8d..b7c2b1414a539 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/SellerPrice.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/SellerPrice.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.integration.nexmark; +package org.apache.beam.integration.nexmark.model; import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.core.JsonProcessingException; @@ -25,13 +25,14 @@ import java.io.OutputStream; import java.io.Serializable; +import org.apache.beam.integration.nexmark.NexmarkUtils; 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.VarLongCoder; /** - * Result of {@link Query6}. + * Result of Query6. */ public class SellerPrice implements KnownSize, Serializable { private static final Coder LONG_CODER = VarLongCoder.of(); diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/package-info.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/package-info.java new file mode 100644 index 0000000000000..e1d611337d3e6 --- /dev/null +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/package-info.java @@ -0,0 +1,22 @@ +/* + * 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. + */ + +/** + * Nexmark Benchmark Model. + */ +package org.apache.beam.integration.nexmark.model; diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/package-info.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/package-info.java index 65bf7d4878034..df6f09f0ae24d 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/package-info.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/package-info.java @@ -16,6 +16,6 @@ * limitations under the License. */ /** - * Nexmark Benchmark Integration Queries. + * Nexmark. */ package org.apache.beam.integration.nexmark; diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query0.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query0.java similarity index 88% rename from integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query0.java rename to integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query0.java index e88fce0f8a1ca..f60d5dee5a3d9 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query0.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query0.java @@ -15,12 +15,17 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.integration.nexmark; +package org.apache.beam.integration.nexmark.queries; import java.io.ByteArrayInputStream; import java.io.ByteArrayOutputStream; import java.io.IOException; +import org.apache.beam.integration.nexmark.NexmarkConfiguration; +import org.apache.beam.integration.nexmark.NexmarkQuery; +import org.apache.beam.integration.nexmark.NexmarkUtils; +import org.apache.beam.integration.nexmark.model.Event; +import org.apache.beam.integration.nexmark.model.KnownSize; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.CoderException; import org.apache.beam.sdk.transforms.Aggregator; diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query0Model.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query0Model.java similarity index 84% rename from integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query0Model.java rename to integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query0Model.java index 37e3f936e3186..991b1d42ee862 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query0Model.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query0Model.java @@ -15,11 +15,16 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.integration.nexmark; +package org.apache.beam.integration.nexmark.queries; import java.util.Collection; import java.util.Iterator; +import org.apache.beam.integration.nexmark.AbstractSimulator; +import org.apache.beam.integration.nexmark.NexmarkConfiguration; +import org.apache.beam.integration.nexmark.NexmarkQueryModel; +import org.apache.beam.integration.nexmark.NexmarkUtils; +import org.apache.beam.integration.nexmark.model.Event; import org.apache.beam.sdk.values.TimestampedValue; /** diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query1.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query1.java similarity index 82% rename from integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query1.java rename to integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query1.java index a1ecdeb7ea1ca..0be77ce2d1426 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query1.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query1.java @@ -15,8 +15,14 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.integration.nexmark; +package org.apache.beam.integration.nexmark.queries; +import org.apache.beam.integration.nexmark.NexmarkConfiguration; +import org.apache.beam.integration.nexmark.NexmarkQuery; +import org.apache.beam.integration.nexmark.NexmarkUtils; +import org.apache.beam.integration.nexmark.model.Bid; +import org.apache.beam.integration.nexmark.model.Event; +import org.apache.beam.integration.nexmark.model.KnownSize; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.ParDo; import org.apache.beam.sdk.values.PCollection; @@ -33,7 +39,7 @@ *

                To make things more interesting, allow the 'currency conversion' to be arbitrarily * slowed down. */ -class Query1 extends NexmarkQuery { +public class Query1 extends NexmarkQuery { public Query1(NexmarkConfiguration configuration) { super(configuration, "Query1"); } diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query10.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query10.java similarity index 97% rename from integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query10.java rename to integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query10.java index 7bdcb367937ae..6912ed1a54806 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query10.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query10.java @@ -15,12 +15,11 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.integration.nexmark; +package org.apache.beam.integration.nexmark.queries; import static com.google.common.base.Preconditions.checkState; import com.google.cloud.hadoop.gcsio.GoogleCloudStorageWriteChannel; - import java.io.IOException; import java.io.OutputStream; import java.io.Serializable; @@ -28,7 +27,12 @@ import java.nio.channels.WritableByteChannel; import java.util.concurrent.ThreadLocalRandom; import javax.annotation.Nullable; - +import org.apache.beam.integration.nexmark.NexmarkConfiguration; +import org.apache.beam.integration.nexmark.NexmarkQuery; +import org.apache.beam.integration.nexmark.NexmarkUtils; +import org.apache.beam.integration.nexmark.model.Done; +import org.apache.beam.integration.nexmark.model.Event; +import org.apache.beam.integration.nexmark.model.KnownSize; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.options.GcsOptions; import org.apache.beam.sdk.transforms.Aggregator; @@ -61,7 +65,7 @@ * *

                Every windowSizeSec, save all events from the last period into 2*maxWorkers log files. */ -class Query10 extends NexmarkQuery { +public class Query10 extends NexmarkQuery { private static final Logger LOG = LoggerFactory.getLogger(Query10.class); private static final int CHANNEL_BUFFER = 8 << 20; // 8MB private static final int NUM_SHARDS_PER_WORKER = 5; diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query11.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query11.java similarity index 85% rename from integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query11.java rename to integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query11.java index d610b7cf53b4b..4da99ebefef63 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query11.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query11.java @@ -15,8 +15,15 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.integration.nexmark; +package org.apache.beam.integration.nexmark.queries; +import org.apache.beam.integration.nexmark.NexmarkConfiguration; +import org.apache.beam.integration.nexmark.NexmarkQuery; +import org.apache.beam.integration.nexmark.NexmarkUtils; +import org.apache.beam.integration.nexmark.model.Bid; +import org.apache.beam.integration.nexmark.model.BidsPerSession; +import org.apache.beam.integration.nexmark.model.Event; +import org.apache.beam.integration.nexmark.model.KnownSize; import org.apache.beam.sdk.transforms.Count; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.ParDo; @@ -35,7 +42,7 @@ * However limit the session to at most {@code maxLogEvents}. Emit the number of * bids per session. */ -class Query11 extends NexmarkQuery { +public class Query11 extends NexmarkQuery { public Query11(NexmarkConfiguration configuration) { super(configuration, "Query11"); } diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query12.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query12.java similarity index 85% rename from integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query12.java rename to integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query12.java index 72fbb57da6006..c67401bf7be81 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query12.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query12.java @@ -15,8 +15,15 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.integration.nexmark; +package org.apache.beam.integration.nexmark.queries; +import org.apache.beam.integration.nexmark.NexmarkConfiguration; +import org.apache.beam.integration.nexmark.NexmarkQuery; +import org.apache.beam.integration.nexmark.NexmarkUtils; +import org.apache.beam.integration.nexmark.model.Bid; +import org.apache.beam.integration.nexmark.model.BidsPerSession; +import org.apache.beam.integration.nexmark.model.Event; +import org.apache.beam.integration.nexmark.model.KnownSize; import org.apache.beam.sdk.transforms.Count; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.ParDo; @@ -34,7 +41,7 @@ *

                Group bids by the same user into processing time windows of windowSize. Emit the count * of bids per window. */ -class Query12 extends NexmarkQuery { +public class Query12 extends NexmarkQuery { public Query12(NexmarkConfiguration configuration) { super(configuration, "Query12"); } diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query1Model.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query1Model.java similarity index 85% rename from integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query1Model.java rename to integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query1Model.java index 16287e68fb029..58037d33a49b2 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query1Model.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query1Model.java @@ -15,12 +15,18 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.integration.nexmark; +package org.apache.beam.integration.nexmark.queries; import java.io.Serializable; import java.util.Collection; import java.util.Iterator; +import org.apache.beam.integration.nexmark.AbstractSimulator; +import org.apache.beam.integration.nexmark.NexmarkConfiguration; +import org.apache.beam.integration.nexmark.NexmarkQueryModel; +import org.apache.beam.integration.nexmark.NexmarkUtils; +import org.apache.beam.integration.nexmark.model.Bid; +import org.apache.beam.integration.nexmark.model.Event; import org.apache.beam.sdk.values.TimestampedValue; /** diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query2.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query2.java similarity index 83% rename from integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query2.java rename to integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query2.java index 828cdf5f91f41..4c8f878c30648 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query2.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query2.java @@ -15,8 +15,15 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.integration.nexmark; +package org.apache.beam.integration.nexmark.queries; +import org.apache.beam.integration.nexmark.NexmarkConfiguration; +import org.apache.beam.integration.nexmark.NexmarkQuery; +import org.apache.beam.integration.nexmark.NexmarkUtils; +import org.apache.beam.integration.nexmark.model.AuctionPrice; +import org.apache.beam.integration.nexmark.model.Bid; +import org.apache.beam.integration.nexmark.model.Event; +import org.apache.beam.integration.nexmark.model.KnownSize; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.Filter; import org.apache.beam.sdk.transforms.ParDo; @@ -37,7 +44,7 @@ * arbitrary size. To make it more interesting we instead choose bids for every * {@code auctionSkip}'th auction. */ -class Query2 extends NexmarkQuery { +public class Query2 extends NexmarkQuery { public Query2(NexmarkConfiguration configuration) { super(configuration, "Query2"); } diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query2Model.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query2Model.java similarity index 83% rename from integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query2Model.java rename to integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query2Model.java index 7769e521e651e..f578e4c571178 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query2Model.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query2Model.java @@ -15,12 +15,19 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.integration.nexmark; +package org.apache.beam.integration.nexmark.queries; import java.io.Serializable; import java.util.Collection; import java.util.Iterator; +import org.apache.beam.integration.nexmark.AbstractSimulator; +import org.apache.beam.integration.nexmark.NexmarkConfiguration; +import org.apache.beam.integration.nexmark.NexmarkQueryModel; +import org.apache.beam.integration.nexmark.NexmarkUtils; +import org.apache.beam.integration.nexmark.model.AuctionPrice; +import org.apache.beam.integration.nexmark.model.Bid; +import org.apache.beam.integration.nexmark.model.Event; import org.apache.beam.sdk.values.TimestampedValue; /** diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query3.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query3.java similarity index 94% rename from integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query3.java rename to integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query3.java index 71969c46eb016..128c2b74a2010 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query3.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query3.java @@ -15,13 +15,20 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.integration.nexmark; +package org.apache.beam.integration.nexmark.queries; import java.io.IOException; import java.util.ArrayList; import java.util.List; import javax.annotation.Nullable; - +import org.apache.beam.integration.nexmark.NexmarkConfiguration; +import org.apache.beam.integration.nexmark.NexmarkQuery; +import org.apache.beam.integration.nexmark.NexmarkUtils; +import org.apache.beam.integration.nexmark.model.Auction; +import org.apache.beam.integration.nexmark.model.Event; +import org.apache.beam.integration.nexmark.model.KnownSize; +import org.apache.beam.integration.nexmark.model.NameCityStateId; +import org.apache.beam.integration.nexmark.model.Person; import org.apache.beam.sdk.coders.ListCoder; import org.apache.beam.sdk.transforms.Aggregator; import org.apache.beam.sdk.transforms.DoFn; @@ -60,7 +67,7 @@ * *

                A real system would use an external system to maintain the id-to-person association. */ -class Query3 extends NexmarkQuery { +public class Query3 extends NexmarkQuery { private static final Logger LOG = LoggerFactory.getLogger(Query3.class); // private static final StateContext GLOBAL_NAMESPACE = StateContexts.global(); private static final StateSpec>> AUCTION_LIST_CODED_TAG = diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query3Model.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query3Model.java similarity index 88% rename from integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query3Model.java rename to integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query3Model.java index 85796ee95f668..e4b72d277ff03 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query3Model.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query3Model.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.integration.nexmark; +package org.apache.beam.integration.nexmark.queries; import com.google.common.collect.ArrayListMultimap; import com.google.common.collect.Multimap; @@ -26,6 +26,14 @@ import java.util.Iterator; import java.util.Map; +import org.apache.beam.integration.nexmark.AbstractSimulator; +import org.apache.beam.integration.nexmark.NexmarkConfiguration; +import org.apache.beam.integration.nexmark.NexmarkQueryModel; +import org.apache.beam.integration.nexmark.NexmarkUtils; +import org.apache.beam.integration.nexmark.model.Auction; +import org.apache.beam.integration.nexmark.model.Event; +import org.apache.beam.integration.nexmark.model.NameCityStateId; +import org.apache.beam.integration.nexmark.model.Person; import org.apache.beam.sdk.values.TimestampedValue; import org.joda.time.Instant; diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query4.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query4.java similarity index 86% rename from integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query4.java rename to integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query4.java index b24410d0a7e46..61991c87f40b0 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query4.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query4.java @@ -15,8 +15,19 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.integration.nexmark; +package org.apache.beam.integration.nexmark.queries; +import org.apache.beam.integration.nexmark.Monitor; +import org.apache.beam.integration.nexmark.NexmarkConfiguration; +import org.apache.beam.integration.nexmark.NexmarkQuery; +import org.apache.beam.integration.nexmark.NexmarkUtils; +import org.apache.beam.integration.nexmark.WinningBids; +import org.apache.beam.integration.nexmark.model.Auction; +import org.apache.beam.integration.nexmark.model.AuctionBid; +import org.apache.beam.integration.nexmark.model.Bid; +import org.apache.beam.integration.nexmark.model.CategoryPrice; +import org.apache.beam.integration.nexmark.model.Event; +import org.apache.beam.integration.nexmark.model.KnownSize; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.Mean; import org.apache.beam.sdk.transforms.ParDo; @@ -50,7 +61,7 @@ * period {@code windowPeriodSec}. *

              */ -class Query4 extends NexmarkQuery { +public class Query4 extends NexmarkQuery { private final Monitor winningBidsMonitor; public Query4(NexmarkConfiguration configuration) { diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query4Model.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query4Model.java similarity index 90% rename from integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query4Model.java rename to integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query4Model.java index afab7e86307cd..9405ac8e40e61 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query4Model.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query4Model.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.integration.nexmark; +package org.apache.beam.integration.nexmark.queries; import java.io.Serializable; import java.util.ArrayList; @@ -24,7 +24,16 @@ import java.util.List; import java.util.Map; import java.util.TreeMap; - +import org.apache.beam.integration.nexmark.AbstractSimulator; +import org.apache.beam.integration.nexmark.NexmarkConfiguration; +import org.apache.beam.integration.nexmark.NexmarkQueryModel; +import org.apache.beam.integration.nexmark.NexmarkUtils; +import org.apache.beam.integration.nexmark.WinningBidsSimulator; +import org.apache.beam.integration.nexmark.model.Auction; +import org.apache.beam.integration.nexmark.model.AuctionBid; +import org.apache.beam.integration.nexmark.model.Bid; +import org.apache.beam.integration.nexmark.model.CategoryPrice; +import org.apache.beam.integration.nexmark.model.KnownSize; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.values.TimestampedValue; import org.joda.time.Duration; diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query5.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query5.java similarity index 90% rename from integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query5.java rename to integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query5.java index 2c9fb9bd264e3..9f02ddb7dc5a2 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query5.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query5.java @@ -15,12 +15,18 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.integration.nexmark; +package org.apache.beam.integration.nexmark.queries; import java.util.ArrayList; import java.util.Arrays; import java.util.List; - +import org.apache.beam.integration.nexmark.NexmarkConfiguration; +import org.apache.beam.integration.nexmark.NexmarkQuery; +import org.apache.beam.integration.nexmark.NexmarkUtils; +import org.apache.beam.integration.nexmark.model.AuctionCount; +import org.apache.beam.integration.nexmark.model.Bid; +import org.apache.beam.integration.nexmark.model.Event; +import org.apache.beam.integration.nexmark.model.KnownSize; import org.apache.beam.sdk.transforms.Combine; import org.apache.beam.sdk.transforms.Count; import org.apache.beam.sdk.transforms.DoFn; @@ -48,7 +54,7 @@ *

              To make things a bit more dynamic and easier to test we use much shorter windows, and * we'll also preserve the bid counts. */ -class Query5 extends NexmarkQuery { +public class Query5 extends NexmarkQuery { public Query5(NexmarkConfiguration configuration) { super(configuration, "Query5"); } diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query5Model.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query5Model.java similarity index 92% rename from integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query5Model.java rename to integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query5Model.java index f8e466e2b6597..6bf65dc2d5fa4 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query5Model.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query5Model.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.integration.nexmark; +package org.apache.beam.integration.nexmark.queries; import java.io.Serializable; import java.util.ArrayList; @@ -24,7 +24,13 @@ import java.util.List; import java.util.Map; import java.util.TreeMap; - +import org.apache.beam.integration.nexmark.AbstractSimulator; +import org.apache.beam.integration.nexmark.NexmarkConfiguration; +import org.apache.beam.integration.nexmark.NexmarkQueryModel; +import org.apache.beam.integration.nexmark.NexmarkUtils; +import org.apache.beam.integration.nexmark.model.AuctionCount; +import org.apache.beam.integration.nexmark.model.Bid; +import org.apache.beam.integration.nexmark.model.Event; import org.apache.beam.sdk.values.TimestampedValue; import org.joda.time.Duration; import org.joda.time.Instant; diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query6.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query6.java similarity index 88% rename from integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query6.java rename to integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query6.java index d5bcc301063fd..2a5ab7029bfdb 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query6.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query6.java @@ -15,14 +15,22 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.integration.nexmark; +package org.apache.beam.integration.nexmark.queries; import com.google.common.collect.Lists; - import java.util.ArrayList; import java.util.Collections; import java.util.List; - +import org.apache.beam.integration.nexmark.NexmarkConfiguration; +import org.apache.beam.integration.nexmark.NexmarkQuery; +import org.apache.beam.integration.nexmark.NexmarkUtils; +import org.apache.beam.integration.nexmark.WinningBids; +import org.apache.beam.integration.nexmark.model.Auction; +import org.apache.beam.integration.nexmark.model.AuctionBid; +import org.apache.beam.integration.nexmark.model.Bid; +import org.apache.beam.integration.nexmark.model.Event; +import org.apache.beam.integration.nexmark.model.KnownSize; +import org.apache.beam.integration.nexmark.model.SellerPrice; import org.apache.beam.sdk.transforms.Combine; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.ParDo; @@ -49,7 +57,7 @@ * *

              We are a little more exact with selecting winning bids: see {@link WinningBids}. */ -class Query6 extends NexmarkQuery { +public class Query6 extends NexmarkQuery { /** * Combiner to keep track of up to {@code maxNumBids} of the most recent wining bids and calculate * their average selling price. diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query6Model.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query6Model.java similarity index 87% rename from integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query6Model.java rename to integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query6Model.java index d03f0fec18cce..432533702199e 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query6Model.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query6Model.java @@ -15,14 +15,23 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.integration.nexmark; +package org.apache.beam.integration.nexmark.queries; import java.io.Serializable; import java.util.Collection; import java.util.Iterator; import java.util.Map; import java.util.TreeMap; - +import org.apache.beam.integration.nexmark.AbstractSimulator; +import org.apache.beam.integration.nexmark.NexmarkConfiguration; +import org.apache.beam.integration.nexmark.NexmarkQueryModel; +import org.apache.beam.integration.nexmark.NexmarkUtils; +import org.apache.beam.integration.nexmark.WinningBidsSimulator; +import org.apache.beam.integration.nexmark.model.Auction; +import org.apache.beam.integration.nexmark.model.AuctionBid; +import org.apache.beam.integration.nexmark.model.Bid; +import org.apache.beam.integration.nexmark.model.KnownSize; +import org.apache.beam.integration.nexmark.model.SellerPrice; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.values.TimestampedValue; import org.joda.time.Instant; diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query7.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query7.java similarity index 88% rename from integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query7.java rename to integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query7.java index 7c51c189250d4..2835737b70a57 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query7.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query7.java @@ -15,8 +15,14 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.integration.nexmark; +package org.apache.beam.integration.nexmark.queries; +import org.apache.beam.integration.nexmark.NexmarkConfiguration; +import org.apache.beam.integration.nexmark.NexmarkQuery; +import org.apache.beam.integration.nexmark.NexmarkUtils; +import org.apache.beam.integration.nexmark.model.Bid; +import org.apache.beam.integration.nexmark.model.Event; +import org.apache.beam.integration.nexmark.model.KnownSize; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.Max; import org.apache.beam.sdk.transforms.ParDo; @@ -42,7 +48,7 @@ * a side-input in order to exercise that functionality. (A combiner, as used in Query 5, is * a more efficient approach.). */ -class Query7 extends NexmarkQuery { +public class Query7 extends NexmarkQuery { public Query7(NexmarkConfiguration configuration) { super(configuration, "Query7"); } diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query7Model.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query7Model.java similarity index 90% rename from integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query7Model.java rename to integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query7Model.java index 0033c68ce6dd5..0a80e590d7991 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query7Model.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query7Model.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.integration.nexmark; +package org.apache.beam.integration.nexmark.queries; import java.io.Serializable; import java.util.ArrayList; @@ -23,6 +23,12 @@ import java.util.Iterator; import java.util.List; +import org.apache.beam.integration.nexmark.AbstractSimulator; +import org.apache.beam.integration.nexmark.NexmarkConfiguration; +import org.apache.beam.integration.nexmark.NexmarkQueryModel; +import org.apache.beam.integration.nexmark.NexmarkUtils; +import org.apache.beam.integration.nexmark.model.Bid; +import org.apache.beam.integration.nexmark.model.Event; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.values.TimestampedValue; import org.joda.time.Duration; diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query8.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query8.java similarity index 86% rename from integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query8.java rename to integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query8.java index ee5c26c8ad40c..e7daccdea947e 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query8.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query8.java @@ -15,8 +15,16 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.integration.nexmark; +package org.apache.beam.integration.nexmark.queries; +import org.apache.beam.integration.nexmark.NexmarkConfiguration; +import org.apache.beam.integration.nexmark.NexmarkQuery; +import org.apache.beam.integration.nexmark.NexmarkUtils; +import org.apache.beam.integration.nexmark.model.Auction; +import org.apache.beam.integration.nexmark.model.Event; +import org.apache.beam.integration.nexmark.model.IdNameReserve; +import org.apache.beam.integration.nexmark.model.KnownSize; +import org.apache.beam.integration.nexmark.model.Person; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.ParDo; import org.apache.beam.sdk.transforms.join.CoGbkResult; @@ -26,7 +34,6 @@ 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; /** @@ -41,7 +48,7 @@ * *

              To make things a bit more dynamic and easier to test we'll use a much shorter window. */ -class Query8 extends NexmarkQuery { +public class Query8 extends NexmarkQuery { public Query8(NexmarkConfiguration configuration) { super(configuration, "Query8"); } diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query8Model.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query8Model.java similarity index 89% rename from integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query8Model.java rename to integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query8Model.java index 261e383db671e..11619942990fb 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query8Model.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query8Model.java @@ -15,17 +15,23 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.integration.nexmark; +package org.apache.beam.integration.nexmark.queries; import com.google.common.collect.ArrayListMultimap; import com.google.common.collect.Multimap; - import java.io.Serializable; import java.util.Collection; import java.util.HashMap; import java.util.Iterator; import java.util.Map; - +import org.apache.beam.integration.nexmark.AbstractSimulator; +import org.apache.beam.integration.nexmark.NexmarkConfiguration; +import org.apache.beam.integration.nexmark.NexmarkQueryModel; +import org.apache.beam.integration.nexmark.NexmarkUtils; +import org.apache.beam.integration.nexmark.model.Auction; +import org.apache.beam.integration.nexmark.model.Event; +import org.apache.beam.integration.nexmark.model.IdNameReserve; +import org.apache.beam.integration.nexmark.model.Person; import org.apache.beam.sdk.values.TimestampedValue; import org.joda.time.Duration; import org.joda.time.Instant; diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query9.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query9.java similarity index 73% rename from integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query9.java rename to integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query9.java index 64bf653a765c7..aed827b8a1853 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query9.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query9.java @@ -15,15 +15,22 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.integration.nexmark; +package org.apache.beam.integration.nexmark.queries; +import org.apache.beam.integration.nexmark.NexmarkConfiguration; +import org.apache.beam.integration.nexmark.NexmarkQuery; +import org.apache.beam.integration.nexmark.NexmarkUtils; +import org.apache.beam.integration.nexmark.WinningBids; +import org.apache.beam.integration.nexmark.model.AuctionBid; +import org.apache.beam.integration.nexmark.model.Event; +import org.apache.beam.integration.nexmark.model.KnownSize; import org.apache.beam.sdk.values.PCollection; /** * Query "9", 'Winning bids'. Select just the winning bids. Not in original NEXMark suite, but * handy for testing. See {@link WinningBids} for the details. */ -class Query9 extends NexmarkQuery { +public class Query9 extends NexmarkQuery { public Query9(NexmarkConfiguration configuration) { super(configuration, "Query9"); } diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query9Model.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query9Model.java similarity index 82% rename from integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query9Model.java rename to integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query9Model.java index 338f02a5bd9c1..b88d60a829b0e 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Query9Model.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query9Model.java @@ -15,12 +15,16 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.integration.nexmark; +package org.apache.beam.integration.nexmark.queries; import java.io.Serializable; import java.util.Collection; import java.util.Iterator; +import org.apache.beam.integration.nexmark.AbstractSimulator; +import org.apache.beam.integration.nexmark.NexmarkConfiguration; +import org.apache.beam.integration.nexmark.NexmarkQueryModel; +import org.apache.beam.integration.nexmark.WinningBidsSimulator; import org.apache.beam.sdk.values.TimestampedValue; /** diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/package-info.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/package-info.java new file mode 100644 index 0000000000000..7a56733ea85b6 --- /dev/null +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/package-info.java @@ -0,0 +1,22 @@ +/* + * 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. + */ + +/** + * Nexmark Queries. + */ +package org.apache.beam.integration.nexmark.queries; diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/BoundedEventSource.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/sources/BoundedEventSource.java similarity index 96% rename from integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/BoundedEventSource.java rename to integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/sources/BoundedEventSource.java index 7dc1bcc855b52..be741516d79c9 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/BoundedEventSource.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/sources/BoundedEventSource.java @@ -15,14 +15,15 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.integration.nexmark; +package org.apache.beam.integration.nexmark.sources; import java.io.IOException; import java.util.ArrayList; import java.util.List; import java.util.NoSuchElementException; import javax.annotation.Nullable; - +import org.apache.beam.integration.nexmark.NexmarkUtils; +import org.apache.beam.integration.nexmark.model.Event; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.io.BoundedSource; import org.apache.beam.sdk.options.PipelineOptions; @@ -32,7 +33,7 @@ /** * A custom, bounded source of event records. */ -class BoundedEventSource extends BoundedSource { +public class BoundedEventSource extends BoundedSource { /** Configuration we generate events against. */ private final GeneratorConfig config; diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Generator.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/sources/Generator.java similarity index 98% rename from integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Generator.java rename to integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/sources/Generator.java index 7adb1b2af9c8f..cffc7a5c97717 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Generator.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/sources/Generator.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.integration.nexmark; +package org.apache.beam.integration.nexmark.sources; import static com.google.common.base.Preconditions.checkNotNull; @@ -28,6 +28,10 @@ import java.util.List; import java.util.Random; +import org.apache.beam.integration.nexmark.model.Auction; +import org.apache.beam.integration.nexmark.model.Bid; +import org.apache.beam.integration.nexmark.model.Event; +import org.apache.beam.integration.nexmark.model.Person; import org.apache.beam.sdk.coders.AtomicCoder; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.CoderException; diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/GeneratorConfig.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/sources/GeneratorConfig.java similarity index 97% rename from integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/GeneratorConfig.java rename to integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/sources/GeneratorConfig.java index dceff4f4778e1..3caaf5179cf96 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/GeneratorConfig.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/sources/GeneratorConfig.java @@ -15,18 +15,20 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.integration.nexmark; +package org.apache.beam.integration.nexmark.sources; import java.io.Serializable; import java.util.ArrayList; import java.util.List; +import org.apache.beam.integration.nexmark.NexmarkConfiguration; +import org.apache.beam.integration.nexmark.model.Event; import org.apache.beam.sdk.values.KV; /** * Parameters controlling how {@link Generator} synthesizes {@link Event} elements. */ -class GeneratorConfig implements Serializable { +public class GeneratorConfig implements Serializable { /** * We start the ids at specific values to help ensure the queries find a match even on * small synthesized dataset sizes. diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/UnboundedEventSource.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/sources/UnboundedEventSource.java similarity index 97% rename from integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/UnboundedEventSource.java rename to integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/sources/UnboundedEventSource.java index 9573ef79fcc5a..286c5768f2ea5 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/UnboundedEventSource.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/sources/UnboundedEventSource.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.integration.nexmark; +package org.apache.beam.integration.nexmark.sources; import java.util.ArrayList; import java.util.List; @@ -25,6 +25,8 @@ import java.util.concurrent.ThreadLocalRandom; import javax.annotation.Nullable; +import org.apache.beam.integration.nexmark.NexmarkUtils; +import org.apache.beam.integration.nexmark.model.Event; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.io.UnboundedSource; import org.apache.beam.sdk.options.PipelineOptions; @@ -42,7 +44,7 @@ * that the overall rate respect the {@code interEventDelayUs} period if possible. Otherwise, * events are returned every time the system asks for one. */ -class UnboundedEventSource extends UnboundedSource { +public class UnboundedEventSource extends UnboundedSource { private static final Duration BACKLOG_PERIOD = Duration.standardSeconds(30); private static final Logger LOG = LoggerFactory.getLogger(UnboundedEventSource.class); diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/sources/package-info.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/sources/package-info.java new file mode 100644 index 0000000000000..ceaec9d46d69c --- /dev/null +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/sources/package-info.java @@ -0,0 +1,22 @@ +/* + * 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. + */ + +/** + * Nexmark Synthetic Sources. + */ +package org.apache.beam.integration.nexmark.sources; diff --git a/integration/java/nexmark/src/main/resources/log4j.properties b/integration/java/nexmark/src/main/resources/log4j.properties index bc09794657652..30d0a9df7f73f 100644 --- a/integration/java/nexmark/src/main/resources/log4j.properties +++ b/integration/java/nexmark/src/main/resources/log4j.properties @@ -22,9 +22,13 @@ log4j.appender.console.target=System.err log4j.appender.console.layout=org.apache.log4j.PatternLayout log4j.appender.console.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss} %p %c: %m%n +# General Beam loggers log4j.logger.org.apache.beam.runners.direct=WARN log4j.logger.org.apache.beam.sdk=WARN +# Nexmark specific +log4j.logger.org.apache.beam.integration.nexmark=ALL + # Settings to quiet third party logs that are too verbose log4j.logger.org.spark_project.jetty=WARN log4j.logger.org.spark_project.jetty.util.component.AbstractLifeCycle=ERROR diff --git a/integration/java/nexmark/src/test/java/org/apache/beam/integration/nexmark/QueryTest.java b/integration/java/nexmark/src/test/java/org/apache/beam/integration/nexmark/queries/QueryTest.java similarity index 87% rename from integration/java/nexmark/src/test/java/org/apache/beam/integration/nexmark/QueryTest.java rename to integration/java/nexmark/src/test/java/org/apache/beam/integration/nexmark/queries/QueryTest.java index e481eac678683..5cf42879e38d6 100644 --- a/integration/java/nexmark/src/test/java/org/apache/beam/integration/nexmark/QueryTest.java +++ b/integration/java/nexmark/src/test/java/org/apache/beam/integration/nexmark/queries/QueryTest.java @@ -15,16 +15,18 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.integration.nexmark; +package org.apache.beam.integration.nexmark.queries; -import org.apache.beam.sdk.Pipeline; +import org.apache.beam.integration.nexmark.NexmarkConfiguration; +import org.apache.beam.integration.nexmark.NexmarkQuery; +import org.apache.beam.integration.nexmark.NexmarkQueryModel; +import org.apache.beam.integration.nexmark.NexmarkUtils; +import org.apache.beam.integration.nexmark.model.KnownSize; +import org.apache.beam.sdk.PipelineResult; 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.PCollection.IsBounded; import org.apache.beam.sdk.values.TimestampedValue; - -import org.junit.Ignore; import org.junit.Rule; import org.junit.Test; import org.junit.runner.RunWith; @@ -41,6 +43,7 @@ public class QueryTest { static { //careful, results of tests are linked to numEvents value + CONFIG.numEventGenerators = 1; CONFIG.numEvents = 100; } @@ -52,7 +55,8 @@ private void queryMatchesModel(String name, NexmarkQuery query, NexmarkQueryMode //TODO Ismael this should not be called explicitly results.setIsBoundedInternal(PCollection.IsBounded.BOUNDED); PAssert.that(results).satisfies(model.assertionFor()); - p.run().waitUntilFinish(); + PipelineResult result = p.run(); + result.waitUntilFinish(); } @Test diff --git a/integration/java/nexmark/src/test/java/org/apache/beam/integration/nexmark/BoundedEventSourceTest.java b/integration/java/nexmark/src/test/java/org/apache/beam/integration/nexmark/sources/BoundedEventSourceTest.java similarity index 95% rename from integration/java/nexmark/src/test/java/org/apache/beam/integration/nexmark/BoundedEventSourceTest.java rename to integration/java/nexmark/src/test/java/org/apache/beam/integration/nexmark/sources/BoundedEventSourceTest.java index 77957e5d50b61..3f85bab14f07a 100644 --- a/integration/java/nexmark/src/test/java/org/apache/beam/integration/nexmark/BoundedEventSourceTest.java +++ b/integration/java/nexmark/src/test/java/org/apache/beam/integration/nexmark/sources/BoundedEventSourceTest.java @@ -15,8 +15,9 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.integration.nexmark; +package org.apache.beam.integration.nexmark.sources; +import org.apache.beam.integration.nexmark.NexmarkConfiguration; import org.apache.beam.runners.dataflow.options.DataflowPipelineOptions; import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.testing.SourceTestUtils; diff --git a/integration/java/nexmark/src/test/java/org/apache/beam/integration/nexmark/GeneratorTest.java b/integration/java/nexmark/src/test/java/org/apache/beam/integration/nexmark/sources/GeneratorTest.java similarity index 96% rename from integration/java/nexmark/src/test/java/org/apache/beam/integration/nexmark/GeneratorTest.java rename to integration/java/nexmark/src/test/java/org/apache/beam/integration/nexmark/sources/GeneratorTest.java index 4b821ea2fb1fc..b0dff2f8f2300 100644 --- a/integration/java/nexmark/src/test/java/org/apache/beam/integration/nexmark/GeneratorTest.java +++ b/integration/java/nexmark/src/test/java/org/apache/beam/integration/nexmark/sources/GeneratorTest.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.integration.nexmark; +package org.apache.beam.integration.nexmark.sources; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; @@ -24,6 +24,7 @@ import java.util.Iterator; import java.util.List; +import org.apache.beam.integration.nexmark.NexmarkConfiguration; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; diff --git a/integration/java/nexmark/src/test/java/org/apache/beam/integration/nexmark/UnboundedEventSourceTest.java b/integration/java/nexmark/src/test/java/org/apache/beam/integration/nexmark/sources/UnboundedEventSourceTest.java similarity index 95% rename from integration/java/nexmark/src/test/java/org/apache/beam/integration/nexmark/UnboundedEventSourceTest.java rename to integration/java/nexmark/src/test/java/org/apache/beam/integration/nexmark/sources/UnboundedEventSourceTest.java index 35b3aeda978d8..15e17a8b64e41 100644 --- a/integration/java/nexmark/src/test/java/org/apache/beam/integration/nexmark/UnboundedEventSourceTest.java +++ b/integration/java/nexmark/src/test/java/org/apache/beam/integration/nexmark/sources/UnboundedEventSourceTest.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.integration.nexmark; +package org.apache.beam.integration.nexmark.sources; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; @@ -26,6 +26,8 @@ import java.util.Random; import java.util.Set; +import org.apache.beam.integration.nexmark.NexmarkConfiguration; +import org.apache.beam.integration.nexmark.model.Event; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.io.UnboundedSource.CheckpointMark; import org.apache.beam.sdk.io.UnboundedSource.UnboundedReader; From 7bfc982c77de52f49ba1b304a81bb0d53de5f44a Mon Sep 17 00:00:00 2001 From: Etienne Chauchot Date: Fri, 24 Mar 2017 14:29:08 +0100 Subject: [PATCH 295/346] Improve query5, query10 and query11 query5: Add comment on key lifting (issue #30) query10: Add comment for strange groupByKey (issue #31) query11: Replace Count.perKey by Count.perElement (issue #32) --- .../integration/nexmark/queries/Query10.java | 3 +- .../integration/nexmark/queries/Query11.java | 47 ++++++++++--------- .../integration/nexmark/queries/Query5.java | 2 + 3 files changed, 27 insertions(+), 25 deletions(-) diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query10.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query10.java index 6912ed1a54806..5246427c5643e 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query10.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query10.java @@ -322,8 +322,7 @@ public void processElement(ProcessContext c, BoundedWindow window) // We expect no late data here, but we'll assume the worst so we can detect any. .withAllowedLateness(Duration.standardDays(1)) .discardingFiredPanes()) - // TODO etienne: unnecessary groupByKey? because aggregators are shared in parallel - // and Pardo is also in parallel, why group all elements in memory of the same executor? + // this GroupByKey allows to have one file per window .apply(name + ".GroupByKey2", GroupByKey.create()) .apply(name + ".Index", ParDo.of(new DoFn>, Done>() { diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query11.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query11.java index 4da99ebefef63..a8a61aebec1a5 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query11.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query11.java @@ -48,29 +48,30 @@ public Query11(NexmarkConfiguration configuration) { } private PCollection applyTyped(PCollection events) { - return events.apply(JUST_BIDS) - .apply(name + ".Rekey", - // TODO etienne: why not avoid this ParDo and do a Cont.perElement? - ParDo.of(new DoFn>() { - @ProcessElement - public void processElement(ProcessContext c) { - Bid bid = c.element(); - c.output(KV.of(bid.bidder, (Void) null)); - } - })) - .apply(Window.>into( - Sessions.withGapDuration(Duration.standardSeconds(configuration.windowSizeSec))) - .triggering(Repeatedly.forever(AfterPane.elementCountAtLeast(configuration.maxLogEvents))) - .discardingFiredPanes() - .withAllowedLateness(Duration.standardSeconds(configuration.occasionalDelaySec / 2))) - .apply(Count.perKey()) - .apply(name + ".ToResult", - ParDo.of(new DoFn, BidsPerSession>() { - @ProcessElement - public void processElement(ProcessContext c) { - c.output(new BidsPerSession(c.element().getKey(), c.element().getValue())); - } - })); + PCollection bidders = events.apply(JUST_BIDS).apply(name + ".Rekey", + ParDo.of(new DoFn() { + + @ProcessElement public void processElement(ProcessContext c) { + Bid bid = c.element(); + c.output(bid.bidder); + } + })); + + PCollection biddersWindowed = bidders.apply( + Window.into( + Sessions.withGapDuration(Duration.standardSeconds(configuration.windowSizeSec))) + .triggering( + Repeatedly.forever(AfterPane.elementCountAtLeast(configuration.maxLogEvents))) + .discardingFiredPanes() + .withAllowedLateness(Duration.standardSeconds(configuration.occasionalDelaySec / 2))); + PCollection bidsPerSession = biddersWindowed.apply(Count.perElement()) + .apply(name + ".ToResult", ParDo.of(new DoFn, BidsPerSession>() { + + @ProcessElement public void processElement(ProcessContext c) { + c.output(new BidsPerSession(c.element().getKey(), c.element().getValue())); + } + })); + return bidsPerSession; } @Override diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query5.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query5.java index 9f02ddb7dc5a2..34b7b50e40660 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query5.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query5.java @@ -75,6 +75,8 @@ private PCollection applyTyped(PCollection events) { // We'll want to keep all auctions with the maximal number of bids. // Start by lifting each into a singleton list. + // need to do so because bellow combine returns a list of auctions in the key in case of + // equal number of bids. Combine needs to have same input type and return type. .apply(name + ".ToSingletons", ParDo.of(new DoFn, KV, Long>>() { @ProcessElement From bd93c8b55ba6f81c87b74364b26d64e0f8c1103f Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Isma=C3=ABl=20Mej=C3=ADa?= Date: Wed, 29 Mar 2017 10:10:13 +0200 Subject: [PATCH 296/346] Fix compile after ParDo refactor --- .../integration/nexmark/NexmarkQuery.java | 14 ++++++------ .../integration/nexmark/NexmarkRunner.java | 3 +-- .../integration/nexmark/NexmarkUtils.java | 16 +++++++------- .../integration/nexmark/queries/Query7.java | 22 +++++++++---------- 4 files changed, 27 insertions(+), 28 deletions(-) diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkQuery.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkQuery.java index c268a3bed84bd..e1cd493ed7e6c 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkQuery.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkQuery.java @@ -97,7 +97,7 @@ public void processElement(ProcessContext c) { }; /** Transform to key each person by their id. */ - protected static final ParDo.Bound> PERSON_BY_ID = + protected static final ParDo.SingleOutput> PERSON_BY_ID = ParDo.of(new DoFn>() { @ProcessElement public void processElement(ProcessContext c) { @@ -106,7 +106,7 @@ public void processElement(ProcessContext c) { }); /** Transform to key each auction by its id. */ - protected static final ParDo.Bound> AUCTION_BY_ID = + protected static final ParDo.SingleOutput> AUCTION_BY_ID = ParDo.of(new DoFn>() { @ProcessElement public void processElement(ProcessContext c) { @@ -115,7 +115,7 @@ public void processElement(ProcessContext c) { }); /** Transform to key each auction by its seller id. */ - protected static final ParDo.Bound> AUCTION_BY_SELLER = + protected static final ParDo.SingleOutput> AUCTION_BY_SELLER = ParDo.of(new DoFn>() { @ProcessElement public void processElement(ProcessContext c) { @@ -124,7 +124,7 @@ public void processElement(ProcessContext c) { }); /** Transform to key each bid by it's auction id. */ - protected static final ParDo.Bound> BID_BY_AUCTION = + protected static final ParDo.SingleOutput> BID_BY_AUCTION = ParDo.of(new DoFn>() { @ProcessElement public void processElement(ProcessContext c) { @@ -133,7 +133,7 @@ public void processElement(ProcessContext c) { }); /** Transform to project the auction id from each bid. */ - protected static final ParDo.Bound BID_TO_AUCTION = + protected static final ParDo.SingleOutput BID_TO_AUCTION = ParDo.of(new DoFn() { @ProcessElement public void processElement(ProcessContext c) { @@ -142,7 +142,7 @@ public void processElement(ProcessContext c) { }); /** Transform to project the price from each bid. */ - protected static final ParDo.Bound BID_TO_PRICE = + protected static final ParDo.SingleOutput BID_TO_PRICE = ParDo.of(new DoFn() { @ProcessElement public void processElement(ProcessContext c) { @@ -151,7 +151,7 @@ public void processElement(ProcessContext c) { }); /** Transform to emit each event with the timestamp embedded within it. */ - public static final ParDo.Bound EVENT_TIMESTAMP_FROM_DATA = + public static final ParDo.SingleOutput EVENT_TIMESTAMP_FROM_DATA = ParDo.of(new DoFn() { @ProcessElement public void processElement(ProcessContext c) { diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkRunner.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkRunner.java index e8d791f510b69..df1000abb3734 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkRunner.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkRunner.java @@ -1073,8 +1073,7 @@ private void sink(PCollection> results, long now) { case BIGQUERY: // Multiple BigQuery backends to mimic what most customers do. PCollectionTuple res = formattedResults.apply(queryName + ".Partition", - ParDo.withOutputTags(MAIN, TupleTagList.of(SIDE)) - .of(new PartitionDoFn())); + ParDo.of(new PartitionDoFn()).withOutputTags(MAIN, TupleTagList.of(SIDE))); sinkResultsToBigQuery(res.get(MAIN), now, "main"); sinkResultsToBigQuery(res.get(SIDE), now, "side"); sinkResultsToBigQuery(formattedResults, now, "copy"); diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkUtils.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkUtils.java index b0421a4c02cfc..a47ebcc2ec4c3 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkUtils.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkUtils.java @@ -417,7 +417,7 @@ public static PTransform> streamEventsSource( /** * Return a transform to pass-through events, but count them as they go by. */ - public static ParDo.Bound snoop(final String name) { + public static ParDo.SingleOutput snoop(final String name) { return ParDo.of(new DoFn() { final Aggregator eventCounter = createAggregator("events", Sum.ofLongs()); @@ -451,7 +451,7 @@ public void processElement(ProcessContext c) { /** * Return a transform to count and discard each element. */ - public static ParDo.Bound devNull(String name) { + public static ParDo.SingleOutput devNull(String name) { return ParDo.of(new DoFn() { final Aggregator discardCounter = createAggregator("discarded", Sum.ofLongs()); @@ -466,7 +466,7 @@ public void processElement(ProcessContext c) { /** * Return a transform to log each element, passing it through unchanged. */ - public static ParDo.Bound log(final String name) { + public static ParDo.SingleOutput log(final String name) { return ParDo.of(new DoFn() { @ProcessElement public void processElement(ProcessContext c) { @@ -479,7 +479,7 @@ public void processElement(ProcessContext c) { /** * Return a transform to format each element as a string. */ - public static ParDo.Bound format(String name) { + public static ParDo.SingleOutput format(String name) { return ParDo.of(new DoFn() { final Aggregator recordCounter = createAggregator("records", Sum.ofLongs()); @@ -495,7 +495,7 @@ public void processElement(ProcessContext c) { /** * Return a transform to make explicit the timestamp of each element. */ - public static ParDo.Bound> stamp(String name) { + public static ParDo.SingleOutput> stamp(String name) { return ParDo.of(new DoFn>() { @ProcessElement public void processElement(ProcessContext c) { @@ -548,7 +548,7 @@ public Long apply(Long left, Long right) { /** * Return a transform to keep the CPU busy for given milliseconds on every record. */ - public static ParDo.Bound cpuDelay(String name, final long delayMs) { + public static ParDo.SingleOutput cpuDelay(String name, final long delayMs) { return ParDo.of(new DoFn() { @ProcessElement public void processElement(ProcessContext c) { @@ -580,7 +580,7 @@ public void processElement(ProcessContext c) { /** * Return a transform to write given number of bytes to durable store on every record. */ - public static ParDo.Bound diskBusy(String name, final long bytes) { + public static ParDo.SingleOutput diskBusy(String name, final long bytes) { return ParDo.of(new DoFn() { @ProcessElement public void processElement(ProcessContext c) { @@ -608,7 +608,7 @@ public void processElement(ProcessContext c) { /** * Return a transform to cast each element to {@link KnownSize}. */ - private static ParDo.Bound castToKnownSize() { + private static ParDo.SingleOutput castToKnownSize() { return ParDo.of(new DoFn() { @ProcessElement public void processElement(ProcessContext c) { diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query7.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query7.java index 2835737b70a57..f3d1ba4f3a2e4 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query7.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query7.java @@ -70,18 +70,18 @@ private PCollection applyTyped(PCollection events) { return slidingBids // Select all bids which have that maximum price (there may be more than one). - .apply(name + ".Select", - ParDo.withSideInputs(maxPriceView) - .of(new DoFn() { - @ProcessElement - public void processElement(ProcessContext c) { - long maxPrice = c.sideInput(maxPriceView); - Bid bid = c.element(); - if (bid.price == maxPrice) { - c.output(bid); - } + .apply(name + ".Select", ParDo + .of(new DoFn() { + @ProcessElement + public void processElement(ProcessContext c) { + long maxPrice = c.sideInput(maxPriceView); + Bid bid = c.element(); + if (bid.price == maxPrice) { + c.output(bid); } - })); + } + }) + .withSideInputs(maxPriceView)); } @Override From 7c28b492aa17160d9a84914814e618716b7beb9f Mon Sep 17 00:00:00 2001 From: Etienne Chauchot Date: Mon, 3 Apr 2017 15:18:04 +0200 Subject: [PATCH 297/346] Fix and improve query3 and query12 query3: Use GlobalWindow to comply with the State/Timer APIs (issue #7). Use timer for personState expiration in GlobalWindow (issue #29). Add trigger to GlobalWindow query12: Replace Count.perKey by Count.perElement (issue #34) --- .../nexmark/NexmarkConfiguration.java | 19 +- .../integration/nexmark/NexmarkOptions.java | 7 + .../integration/nexmark/queries/Query12.java | 19 +- .../integration/nexmark/queries/Query3.java | 263 +++++++++++------- .../nexmark/queries/QueryTest.java | 4 + 5 files changed, 195 insertions(+), 117 deletions(-) diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkConfiguration.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkConfiguration.java index e2890eda644fa..d6cd80801163e 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkConfiguration.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkConfiguration.java @@ -194,6 +194,13 @@ public class NexmarkConfiguration implements Serializable { @JsonProperty public int fanout = 5; + /** + * Maximum waiting time to clean personState in query3 + * (ie maximum waiting of the auctions related to person in state in seconds in event time). + */ + @JsonProperty + public int maxAuctionsWaitingTime = 600; + /** * Length of occasional delay to impose on events (in seconds). */ @@ -322,6 +329,9 @@ public void overrideFromOptions(NexmarkOptions options) { if (options.getFanout() != null) { fanout = options.getFanout(); } + if (options.getMaxAuctionsWaitingTime() != null) { + fanout = options.getMaxAuctionsWaitingTime(); + } if (options.getOccasionalDelaySec() != null) { occasionalDelaySec = options.getOccasionalDelaySec(); } @@ -376,6 +386,7 @@ public NexmarkConfiguration clone() { result.diskBusyBytes = diskBusyBytes; result.auctionSkip = auctionSkip; result.fanout = fanout; + result.maxAuctionsWaitingTime = maxAuctionsWaitingTime; result.occasionalDelaySec = occasionalDelaySec; result.probDelayedEvent = probDelayedEvent; result.maxLogEvents = maxLogEvents; @@ -479,6 +490,9 @@ public String toShortString() { if (fanout != DEFAULT.fanout) { sb.append(String.format("; fanout:%d", fanout)); } + if (maxAuctionsWaitingTime != DEFAULT.maxAuctionsWaitingTime) { + sb.append(String.format("; maxAuctionsWaitingTime:%d", fanout)); + } if (occasionalDelaySec != DEFAULT.occasionalDelaySec) { sb.append(String.format("; occasionalDelaySec:%d", occasionalDelaySec)); } @@ -527,7 +541,7 @@ public int hashCode() { ratePeriodSec, preloadSeconds, isRateLimited, useWallclockEventTime, avgPersonByteSize, avgAuctionByteSize, avgBidByteSize, hotAuctionRatio, hotSellersRatio, hotBiddersRatio, windowSizeSec, windowPeriodSec, watermarkHoldbackSec, numInFlightAuctions, numActivePeople, - coderStrategy, cpuDelayMs, diskBusyBytes, auctionSkip, fanout, + coderStrategy, cpuDelayMs, diskBusyBytes, auctionSkip, fanout, maxAuctionsWaitingTime, occasionalDelaySec, probDelayedEvent, maxLogEvents, usePubsubPublishTime, outOfOrderGroupSize); } @@ -571,6 +585,9 @@ public boolean equals(Object obj) { if (fanout != other.fanout) { return false; } + if (maxAuctionsWaitingTime != other.maxAuctionsWaitingTime) { + return false; + } if (firstEventRate != other.firstEventRate) { return false; } diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkOptions.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkOptions.java index 1be974fd4202b..e39f0a48dd109 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkOptions.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkOptions.java @@ -309,6 +309,13 @@ public interface NexmarkOptions extends PubsubOptions { void setFanout(Integer fanout); + @Description("Maximum waiting time to clean personState in query3 " + + "(ie maximum waiting of the auctions related to person in state in seconds in event time).") + @Nullable + Integer getMaxAuctionsWaitingTime(); + + void setMaxAuctionsWaitingTime(Integer fanout); + @Description("Length of occasional delay to impose on events (in seconds).") @Nullable Long getOccasionalDelaySec(); diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query12.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query12.java index c67401bf7be81..a5db5047b5e63 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query12.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query12.java @@ -49,16 +49,13 @@ public Query12(NexmarkConfiguration configuration) { private PCollection applyTyped(PCollection events) { return events .apply(JUST_BIDS) - .apply(name + ".Rekey", - // TODO etienne: why not avoid this ParDo and do a Cont.perElement? - ParDo.of(new DoFn>() { - @ProcessElement - public void processElement(ProcessContext c) { - Bid bid = c.element(); - c.output(KV.of(bid.bidder, (Void) null)); - } - })) - .apply(Window.>into(new GlobalWindows()) + .apply(ParDo.of(new DoFn() { + @ProcessElement + public void processElement(ProcessContext c){ + c.output(c.element().bidder); + } + })) + .apply(Window.into(new GlobalWindows()) .triggering( Repeatedly.forever( AfterProcessingTime.pastFirstElementInPane() @@ -66,7 +63,7 @@ public void processElement(ProcessContext c) { Duration.standardSeconds(configuration.windowSizeSec)))) .discardingFiredPanes() .withAllowedLateness(Duration.ZERO)) - .apply(Count.perKey()) + .apply(Count.perElement()) .apply(name + ".ToResult", ParDo.of(new DoFn, BidsPerSession>() { @ProcessElement diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query3.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query3.java index 128c2b74a2010..ba31e9ff5aba3 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query3.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query3.java @@ -39,14 +39,21 @@ import org.apache.beam.sdk.transforms.join.CoGbkResult; import org.apache.beam.sdk.transforms.join.CoGroupByKey; import org.apache.beam.sdk.transforms.join.KeyedPCollectionTuple; -import org.apache.beam.sdk.transforms.windowing.FixedWindows; +import org.apache.beam.sdk.transforms.windowing.AfterPane; +import org.apache.beam.sdk.transforms.windowing.GlobalWindows; +import org.apache.beam.sdk.transforms.windowing.Repeatedly; 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.state.StateSpec; import org.apache.beam.sdk.util.state.StateSpecs; import org.apache.beam.sdk.util.state.ValueState; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; import org.joda.time.Duration; +import org.joda.time.Instant; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -62,31 +69,141 @@ * * *

              We'll implement this query to allow 'new auction' events to come before the 'new person' - * events for the auction seller. Those auctions will be stored until the matching person is - * seen. Then all subsequent auctions for a person will use the stored person record. + * events for the auction seller. Those auctions will be stored until the matching person is seen. + * Then all subsequent auctions for a person will use the stored person record. * *

              A real system would use an external system to maintain the id-to-person association. */ public class Query3 extends NexmarkQuery { + private static final Logger LOG = LoggerFactory.getLogger(Query3.class); -// private static final StateContext GLOBAL_NAMESPACE = StateContexts.global(); - private static final StateSpec>> AUCTION_LIST_CODED_TAG = - StateSpecs.value(ListCoder.of(Auction.CODER)); - private static final StateSpec> PERSON_CODED_TAG = - StateSpecs.value(Person.CODER); + private final JoinDoFn joinDoFn; + + public Query3(NexmarkConfiguration configuration) { + super(configuration, "Query3"); + joinDoFn = new JoinDoFn(configuration.maxAuctionsWaitingTime); + + } + + @Override + @Nullable + public Aggregator getFatalCount() { + return joinDoFn.fatalCounter; + } + + private PCollection applyTyped(PCollection events) { + int numEventsInPane = 30; + + PCollection eventsWindowed = + events.apply( + Window.into(new GlobalWindows()) + .triggering(Repeatedly.forever((AfterPane.elementCountAtLeast(numEventsInPane)))) + .discardingFiredPanes() + .withAllowedLateness(Duration.ZERO)); + PCollection> auctionsBySellerId = + eventsWindowed + // Only want the new auction events. + .apply(JUST_NEW_AUCTIONS) + + // We only want auctions in category 10. + .apply( + name + ".InCategory", + Filter.by( + new SerializableFunction() { + + @Override + public Boolean apply(Auction auction) { + return auction.category == 10; + } + })) + + // Key auctions by their seller id. + .apply("AuctionBySeller", AUCTION_BY_SELLER); + + PCollection> personsById = + eventsWindowed + // Only want the new people events. + .apply(JUST_NEW_PERSONS) + + // We only want people in OR, ID, CA. + .apply( + name + ".InState", + Filter.by( + new SerializableFunction() { + + @Override + public Boolean apply(Person person) { + return person.state.equals("OR") + || person.state.equals("ID") + || person.state.equals("CA"); + } + })) + + // Key people by their id. + .apply("PersonById", PERSON_BY_ID); + + return + // Join auctions and people. + // concatenate KeyedPCollections + KeyedPCollectionTuple.of(AUCTION_TAG, auctionsBySellerId) + .and(PERSON_TAG, personsById) + // group auctions and persons by personId + .apply(CoGroupByKey.create()) + .apply(name + ".Join", ParDo.of(joinDoFn)) + + // Project what we want. + .apply( + name + ".Project", + ParDo.of( + new DoFn, NameCityStateId>() { + + @ProcessElement + public void processElement(ProcessContext c) { + Auction auction = c.element().getKey(); + Person person = c.element().getValue(); + c.output( + new NameCityStateId(person.name, person.city, person.state, auction.id)); + } + })); + } + + @Override + protected PCollection applyPrim(PCollection events) { + return NexmarkUtils.castToKnownSize(name, applyTyped(events)); + } /** - * Join {@code auctions} and {@code people} by person id and emit their cross-product one pair - * at a time. + * Join {@code auctions} and {@code people} by person id and emit their cross-product one pair at + * a time. * *

              We know a person may submit any number of auctions. Thus new person event must have the * person record stored in persistent state in order to match future auctions by that person. * - *

              However we know that each auction is associated with at most one person, so only need - * to store auction records in persistent state until we have seen the corresponding person - * record. And of course may have already seen that record. + *

              However we know that each auction is associated with at most one person, so only need to + * store auction records in persistent state until we have seen the corresponding person record. + * And of course may have already seen that record. */ private static class JoinDoFn extends DoFn, KV> { + + private int maxAuctionsWaitingTime; + private static final String AUCTIONS = "auctions"; + private static final String PERSON = "person"; + + @StateId(PERSON) + private static final StateSpec> personSpec = + StateSpecs.value(Person.CODER); + + private static final String PERSON_STATE_EXPIRING = "personStateExpiring"; + + public final Aggregator fatalCounter = createAggregator("fatal", Sum.ofLongs()); + + @StateId(AUCTIONS) + private final StateSpec>> auctionsSpec = + StateSpecs.value(ListCoder.of(Auction.CODER)); + + @TimerId(PERSON_STATE_EXPIRING) + private final TimerSpec timerSpec = TimerSpecs.timer(TimeDomain.EVENT_TIME); + private final Aggregator newAuctionCounter = createAggregator("newAuction", Sum.ofLongs()); private final Aggregator newPersonCounter = @@ -97,20 +214,25 @@ private static class JoinDoFn extends DoFn, KV oldNewOutputCounter = createAggregator("oldNewOutput", Sum.ofLongs()); - public final Aggregator fatalCounter = createAggregator("fatal", Sum.ofLongs()); + + private JoinDoFn(int maxAuctionsWaitingTime) { + this.maxAuctionsWaitingTime = maxAuctionsWaitingTime; + } @ProcessElement - public void processElement(ProcessContext c) throws IOException { - //TODO: This is using the internal state API. Rework to use the - //TODO Ismael this is broken for not access to state + public void processElement( + ProcessContext c, + @TimerId(PERSON_STATE_EXPIRING) Timer timer, + @StateId(PERSON) ValueState personState, + @StateId(AUCTIONS) ValueState> auctionsState) + throws IOException { // We would *almost* implement this by rewindowing into the global window and // running a combiner over the result. The combiner's accumulator would be the // state we use below. However, combiners cannot emit intermediate results, thus // we need to wait for the pending ReduceFn API. -// StateInternals stateInternals = c.windowingInternals().stateInternals(); -// ValueState personState = stateInternals.state(GLOBAL_NAMESPACE, PERSON_CODED_TAG); -// Person existingPerson = personState.read(); - Person existingPerson = null; + + Person existingPerson = personState.read(); + if (existingPerson != null) { // We've already seen the new person event for this person id. // We can join with any new auctions on-the-fly without needing any @@ -123,8 +245,6 @@ public void processElement(ProcessContext c) throws IOException { return; } -// ValueState> auctionsState = -// stateInternals.state(GLOBAL_NAMESPACE, AUCTION_LIST_CODED_TAG); Person theNewPerson = null; for (Person newPerson : c.element().getValue().getAll(PERSON_TAG)) { if (theNewPerson == null) { @@ -140,14 +260,14 @@ public void processElement(ProcessContext c) throws IOException { } newPersonCounter.addValue(1L); // We've now seen the person for this person id so can flush any - // pending auctions for the same seller id. - List pendingAuctions = null; //auctionsState.read(); + // pending auctions for the same seller id (an auction is done by only one seller). + List pendingAuctions = auctionsState.read(); if (pendingAuctions != null) { for (Auction pendingAuction : pendingAuctions) { oldNewOutputCounter.addValue(1L); c.output(KV.of(pendingAuction, newPerson)); } -// auctionsState.clear(); + auctionsState.clear(); } // Also deal with any new auctions. for (Auction newAuction : c.element().getValue().getAll(AUCTION_TAG)) { @@ -156,8 +276,11 @@ public void processElement(ProcessContext c) throws IOException { c.output(KV.of(newAuction, newPerson)); } // Remember this person for any future auctions. - -// personState.write(newPerson); + personState.write(newPerson); + //set a time out to clear this state + Instant firingTime = new Instant(newPerson.dateTime) + .plus(Duration.standardSeconds(maxAuctionsWaitingTime)); + timer.set(firingTime); } if (theNewPerson != null) { return; @@ -165,7 +288,7 @@ public void processElement(ProcessContext c) throws IOException { // We'll need to remember the auctions until we see the corresponding // new person event. - List pendingAuctions = null; //auctionsState.read(); + List pendingAuctions = auctionsState.read(); if (pendingAuctions == null) { pendingAuctions = new ArrayList<>(); } @@ -173,84 +296,14 @@ public void processElement(ProcessContext c) throws IOException { newAuctionCounter.addValue(1L); pendingAuctions.add(newAuction); } -// auctionsState.write(pendingAuctions); + auctionsState.write(pendingAuctions); } + @OnTimer(PERSON_STATE_EXPIRING) + public void onTimerCallback( + OnTimerContext context, + @StateId(PERSON) ValueState personState) { + personState.clear(); } - private final JoinDoFn joinDoFn = new JoinDoFn(); - - public Query3(NexmarkConfiguration configuration) { - super(configuration, "Query3"); - } - - @Override - @Nullable - public Aggregator getFatalCount() { - return joinDoFn.fatalCounter; - } - - private PCollection applyTyped(PCollection events) { - // Batch into incremental results windows. - events = events.apply( - Window.into(FixedWindows.of(Duration.standardSeconds(configuration.windowSizeSec)))); - - PCollection> auctionsBySellerId = - events - // Only want the new auction events. - .apply(JUST_NEW_AUCTIONS) - - // We only want auctions in category 10. - .apply(name + ".InCategory", Filter.by(new SerializableFunction() { - @Override - public Boolean apply(Auction auction) { - return auction.category == 10; - } - })) - - // Key auctions by their seller id. - .apply("AuctionBySeller", AUCTION_BY_SELLER); - - PCollection> personsById = - events - // Only want the new people events. - .apply(JUST_NEW_PERSONS) - - // We only want people in OR, ID, CA. - .apply(name + ".InState", Filter.by(new SerializableFunction() { - @Override - public Boolean apply(Person person) { - return person.state.equals("OR") || person.state.equals("ID") - || person.state.equals("CA"); - } - })) - - // Key people by their id. - .apply("PersonById", PERSON_BY_ID); - - return - // Join auctions and people. - // concatenate KeyedPCollections - KeyedPCollectionTuple.of(AUCTION_TAG, auctionsBySellerId) - .and(PERSON_TAG, personsById) - // group auctions and persons by personId - .apply(CoGroupByKey.create()) - .apply(name + ".Join", ParDo.of(joinDoFn)) - - // Project what we want. - .apply(name + ".Project", - ParDo.of(new DoFn, NameCityStateId>() { - @ProcessElement - public void processElement(ProcessContext c) { - Auction auction = c.element().getKey(); - Person person = c.element().getValue(); - c.output(new NameCityStateId( - person.name, person.city, person.state, auction.id)); - } - })); - } - - @Override - protected PCollection applyPrim(PCollection events) { - return NexmarkUtils.castToKnownSize(name, applyTyped(events)); } } diff --git a/integration/java/nexmark/src/test/java/org/apache/beam/integration/nexmark/queries/QueryTest.java b/integration/java/nexmark/src/test/java/org/apache/beam/integration/nexmark/queries/QueryTest.java index 5cf42879e38d6..dca2887bc000e 100644 --- a/integration/java/nexmark/src/test/java/org/apache/beam/integration/nexmark/queries/QueryTest.java +++ b/integration/java/nexmark/src/test/java/org/apache/beam/integration/nexmark/queries/QueryTest.java @@ -25,10 +25,13 @@ import org.apache.beam.sdk.PipelineResult; import org.apache.beam.sdk.testing.PAssert; import org.apache.beam.sdk.testing.TestPipeline; +import org.apache.beam.sdk.testing.UsesStatefulParDo; +import org.apache.beam.sdk.testing.UsesTimersInParDo; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.TimestampedValue; import org.junit.Rule; import org.junit.Test; +import org.junit.experimental.categories.Category; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; @@ -95,6 +98,7 @@ public void query6MatchesModel() { } @Test + @Category({UsesStatefulParDo.class, UsesTimersInParDo.class}) public void query7MatchesModel() { queryMatchesModel("Query7Test", new Query7(CONFIG), new Query7Model(CONFIG)); } From 7ef49dc3706c3a2543284e17eb39782c783d30cf Mon Sep 17 00:00:00 2001 From: Etienne Chauchot Date: Mon, 3 Apr 2017 16:50:51 +0200 Subject: [PATCH 298/346] Improve queries tests Fix Runner categories in tests Add streaming unit tests and corresponding labels issue #37 Update numEvents: results are no more linked to the number of events issue #22 --- .../src/main/resources/log4j.properties | 2 +- .../nexmark/queries/QueryTest.java | 142 ++++++++++++++---- 2 files changed, 110 insertions(+), 34 deletions(-) diff --git a/integration/java/nexmark/src/main/resources/log4j.properties b/integration/java/nexmark/src/main/resources/log4j.properties index 30d0a9df7f73f..7dd57b542f863 100644 --- a/integration/java/nexmark/src/main/resources/log4j.properties +++ b/integration/java/nexmark/src/main/resources/log4j.properties @@ -27,7 +27,7 @@ log4j.logger.org.apache.beam.runners.direct=WARN log4j.logger.org.apache.beam.sdk=WARN # Nexmark specific -log4j.logger.org.apache.beam.integration.nexmark=ALL +log4j.logger.org.apache.beam.integration.nexmark=WARN # Settings to quiet third party logs that are too verbose log4j.logger.org.spark_project.jetty=WARN diff --git a/integration/java/nexmark/src/test/java/org/apache/beam/integration/nexmark/queries/QueryTest.java b/integration/java/nexmark/src/test/java/org/apache/beam/integration/nexmark/queries/QueryTest.java index dca2887bc000e..284aa7e4f2c9b 100644 --- a/integration/java/nexmark/src/test/java/org/apache/beam/integration/nexmark/queries/QueryTest.java +++ b/integration/java/nexmark/src/test/java/org/apache/beam/integration/nexmark/queries/QueryTest.java @@ -23,6 +23,7 @@ import org.apache.beam.integration.nexmark.NexmarkUtils; import org.apache.beam.integration.nexmark.model.KnownSize; import org.apache.beam.sdk.PipelineResult; +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.testing.UsesStatefulParDo; @@ -35,81 +36,156 @@ import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -/** - * Test the various NEXMark queries yield results coherent with their models. - */ +/** Test the various NEXMark queries yield results coherent with their models. */ @RunWith(JUnit4.class) public class QueryTest { private static final NexmarkConfiguration CONFIG = NexmarkConfiguration.DEFAULT.clone(); - @Rule - public TestPipeline p = TestPipeline.create(); static { - //careful, results of tests are linked to numEvents value + // careful, results of tests are linked to numEventGenerators because of timestamp generation CONFIG.numEventGenerators = 1; - CONFIG.numEvents = 100; + CONFIG.numEvents = 1000; } + @Rule public TestPipeline p = TestPipeline.create(); + /** Test {@code query} matches {@code model}. */ - private void queryMatchesModel(String name, NexmarkQuery query, NexmarkQueryModel model) { + private void queryMatchesModel( + String name, NexmarkQuery query, NexmarkQueryModel model, boolean streamingMode) { NexmarkUtils.setupPipeline(NexmarkUtils.CoderStrategy.HAND, p); - PCollection> results = - p.apply(name + ".ReadBounded", NexmarkUtils.batchEventsSource(CONFIG)).apply(query); - //TODO Ismael this should not be called explicitly - results.setIsBoundedInternal(PCollection.IsBounded.BOUNDED); + PCollection> results; + if (streamingMode) { + results = + p.apply(name + ".ReadUnBounded", NexmarkUtils.streamEventsSource(CONFIG)).apply(query); + //TODO Ismael this should not be called explicitly + results.setIsBoundedInternal(PCollection.IsBounded.UNBOUNDED); + } else { + results = p.apply(name + ".ReadBounded", NexmarkUtils.batchEventsSource(CONFIG)).apply(query); + //TODO Ismael this should not be called explicitly + results.setIsBoundedInternal(PCollection.IsBounded.BOUNDED); + } PAssert.that(results).satisfies(model.assertionFor()); PipelineResult result = p.run(); result.waitUntilFinish(); } @Test - public void query0MatchesModel() { - queryMatchesModel("Query0Test", new Query0(CONFIG), new Query0Model(CONFIG)); + @Category(NeedsRunner.class) + public void query0MatchesModelBatch() { + queryMatchesModel("Query0TestBatch", new Query0(CONFIG), new Query0Model(CONFIG), false); + } + + @Test + @Category(NeedsRunner.class) + public void query0MatchesModelStreaming() { + queryMatchesModel("Query0TestStreaming", new Query0(CONFIG), new Query0Model(CONFIG), true); + } + + @Test + @Category(NeedsRunner.class) + public void query1MatchesModelBatch() { + queryMatchesModel("Query1TestBatch", new Query1(CONFIG), new Query1Model(CONFIG), false); + } + + @Test + @Category(NeedsRunner.class) + public void query1MatchesModelStreaming() { + queryMatchesModel("Query1TestStreaming", new Query1(CONFIG), new Query1Model(CONFIG), true); + } + + @Test + @Category(NeedsRunner.class) + public void query2MatchesModelBatch() { + queryMatchesModel("Query2TestBatch", new Query2(CONFIG), new Query2Model(CONFIG), false); + } + + @Test + @Category(NeedsRunner.class) + public void query2MatchesModelStreaming() { + queryMatchesModel("Query2TestStreaming", new Query2(CONFIG), new Query2Model(CONFIG), true); + } + + @Test + @Category({NeedsRunner.class, UsesStatefulParDo.class, UsesTimersInParDo.class}) + public void query3MatchesModelBatch() { + queryMatchesModel("Query3TestBatch", new Query3(CONFIG), new Query3Model(CONFIG), false); + } + + @Test + @Category({NeedsRunner.class, UsesStatefulParDo.class, UsesTimersInParDo.class}) + public void query3MatchesModelStreaming() { + queryMatchesModel("Query3TestStreaming", new Query3(CONFIG), new Query3Model(CONFIG), true); + } + + @Test + @Category(NeedsRunner.class) + public void query4MatchesModelBatch() { + queryMatchesModel("Query4TestBatch", new Query4(CONFIG), new Query4Model(CONFIG), false); + } + + @Test + @Category(NeedsRunner.class) + public void query4MatchesModelStreaming() { + queryMatchesModel("Query4TestStreaming", new Query4(CONFIG), new Query4Model(CONFIG), true); + } + + @Test + @Category(NeedsRunner.class) + public void query5MatchesModelBatch() { + queryMatchesModel("Query5TestBatch", new Query5(CONFIG), new Query5Model(CONFIG), false); } @Test - public void query1MatchesModel() { - queryMatchesModel("Query1Test", new Query1(CONFIG), new Query1Model(CONFIG)); + @Category(NeedsRunner.class) + public void query5MatchesModelStreaming() { + queryMatchesModel("Query5TestStreaming", new Query5(CONFIG), new Query5Model(CONFIG), true); } @Test - public void query2MatchesModel() { - queryMatchesModel("Query2Test", new Query2(CONFIG), new Query2Model(CONFIG)); + @Category(NeedsRunner.class) + public void query6MatchesModelBatch() { + queryMatchesModel("Query6TestBatch", new Query6(CONFIG), new Query6Model(CONFIG), false); } @Test - public void query3MatchesModel() { - queryMatchesModel("Query3Test", new Query3(CONFIG), new Query3Model(CONFIG)); + @Category(NeedsRunner.class) + public void query6MatchesModelStreaming() { + queryMatchesModel("Query6TestStreaming", new Query6(CONFIG), new Query6Model(CONFIG), true); } @Test - public void query4MatchesModel() { - queryMatchesModel("Query4Test", new Query4(CONFIG), new Query4Model(CONFIG)); + @Category(NeedsRunner.class) + public void query7MatchesModelBatch() { + queryMatchesModel("Query7TestBatch", new Query7(CONFIG), new Query7Model(CONFIG), false); } @Test - public void query5MatchesModel() { - queryMatchesModel("Query5Test", new Query5(CONFIG), new Query5Model(CONFIG)); + @Category(NeedsRunner.class) + public void query7MatchesModelStreaming() { + queryMatchesModel("Query7TestStreaming", new Query7(CONFIG), new Query7Model(CONFIG), true); } @Test - public void query6MatchesModel() { - queryMatchesModel("Query6Test", new Query6(CONFIG), new Query6Model(CONFIG)); + @Category(NeedsRunner.class) + public void query8MatchesModelBatch() { + queryMatchesModel("Query8TestBatch", new Query8(CONFIG), new Query8Model(CONFIG), false); } @Test - @Category({UsesStatefulParDo.class, UsesTimersInParDo.class}) - public void query7MatchesModel() { - queryMatchesModel("Query7Test", new Query7(CONFIG), new Query7Model(CONFIG)); + @Category(NeedsRunner.class) + public void query8MatchesModelStreaming() { + queryMatchesModel("Query8TestStreaming", new Query8(CONFIG), new Query8Model(CONFIG), true); } @Test - public void query8MatchesModel() { - queryMatchesModel("Query8Test", new Query8(CONFIG), new Query8Model(CONFIG)); + @Category(NeedsRunner.class) + public void query9MatchesModelBatch() { + queryMatchesModel("Query9TestBatch", new Query9(CONFIG), new Query9Model(CONFIG), false); } @Test - public void query9MatchesModel() { - queryMatchesModel("Query9Test", new Query9(CONFIG), new Query9Model(CONFIG)); + @Category(NeedsRunner.class) + public void query9MatchesModelStreaming() { + queryMatchesModel("Query9TestStreaming", new Query9(CONFIG), new Query9Model(CONFIG), true); } } From 8098bb1dbcc22153960d9b4483327e2977641148 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Isma=C3=ABl=20Mej=C3=ADa?= Date: Thu, 13 Apr 2017 10:47:54 +0200 Subject: [PATCH 299/346] Change Nexmark pom structure to mirror other modules on Beam Fix compile after PubsubIO refactor --- integration/java/nexmark/pom.xml | 48 +- .../integration/nexmark/NexmarkRunner.java | 2 +- .../integration/nexmark/io/PubsubClient.java | 543 ++++++++++++++++++ .../integration/nexmark/io/PubsubHelper.java | 2 - .../nexmark/io/PubsubJsonClient.java | 318 ++++++++++ .../nexmark/io/PubsubTestClient.java | 436 ++++++++++++++ integration/java/pom.xml | 37 ++ integration/pom.xml | 37 ++ pom.xml | 2 +- 9 files changed, 1401 insertions(+), 24 deletions(-) create mode 100644 integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/io/PubsubClient.java create mode 100644 integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/io/PubsubJsonClient.java create mode 100644 integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/io/PubsubTestClient.java create mode 100644 integration/java/pom.xml create mode 100644 integration/pom.xml diff --git a/integration/java/nexmark/pom.xml b/integration/java/nexmark/pom.xml index 7cd7d392a20cf..67d6117cf3185 100644 --- a/integration/java/nexmark/pom.xml +++ b/integration/java/nexmark/pom.xml @@ -22,19 +22,17 @@ org.apache.beam - beam-parent + beam-integration-java-parent 0.7.0-SNAPSHOT - ../../../pom.xml + ../pom.xml - beam-integration-java + beam-integration-java-nexmark Apache Beam :: Integration Tests :: Java :: Nexmark jar - UTF-8 - UTF-8 1.2.0 1.6.3 1.9.3 @@ -252,11 +250,36 @@ google-api-services-bigquery + + com.google.apis + google-api-services-pubsub + + + + com.google.auth + google-auth-library-credentials + + + + com.google.auth + google-auth-library-oauth2-http + + com.google.cloud.bigdataoss gcsio + + com.google.cloud.bigdataoss + util + + + + com.google.http-client + google-http-client + + com.fasterxml.jackson.core jackson-annotations @@ -288,13 +311,6 @@ compile - - - - - - - com.google.code.findbugs jsr305 @@ -305,13 +321,5 @@ junit compile - - - - - - - - diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkRunner.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkRunner.java index df1000abb3734..3a0452f182b5d 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkRunner.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkRunner.java @@ -67,9 +67,9 @@ import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.PipelineResult; import org.apache.beam.sdk.io.AvroIO; -import org.apache.beam.sdk.io.PubsubIO; import org.apache.beam.sdk.io.TextIO; import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO; +import org.apache.beam.sdk.io.gcp.pubsub.PubsubIO; import org.apache.beam.sdk.testing.PAssert; import org.apache.beam.sdk.transforms.Aggregator; import org.apache.beam.sdk.transforms.DoFn; diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/io/PubsubClient.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/io/PubsubClient.java new file mode 100644 index 0000000000000..687aa35d9cc6c --- /dev/null +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/io/PubsubClient.java @@ -0,0 +1,543 @@ +/* + * 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.integration.nexmark.io; + +import static com.google.common.base.Preconditions.checkArgument; +import static com.google.common.base.Preconditions.checkState; + +import com.google.api.client.util.DateTime; +import com.google.common.base.Objects; +import com.google.common.base.Strings; +import java.io.Closeable; +import java.io.IOException; +import java.io.Serializable; +import java.util.Arrays; +import java.util.List; +import java.util.Map; +import java.util.concurrent.ThreadLocalRandom; +import javax.annotation.Nullable; +import org.apache.beam.sdk.options.PubsubOptions; + +/** + * An (abstract) helper class for talking to Pubsub via an underlying transport. + */ +abstract class PubsubClient implements Closeable { + /** + * Factory for creating clients. + */ + public interface PubsubClientFactory extends Serializable { + /** + * Construct a new Pubsub client. It should be closed via {@link #close} in order + * to ensure tidy cleanup of underlying netty resources (or use the try-with-resources + * construct). Uses {@code options} to derive pubsub endpoints and application credentials. + * If non-{@literal null}, use {@code timestampLabel} and {@code idLabel} to store custom + * timestamps/ids within message metadata. + */ + PubsubClient newClient(@Nullable String timestampLabel, + @Nullable String idLabel, PubsubOptions options) throws IOException; + + /** + * Return the display name for this factory. Eg "Json", "gRPC". + */ + String getKind(); + } + + /** + * Return timestamp as ms-since-unix-epoch corresponding to {@code timestamp}. + * Return {@literal null} if no timestamp could be found. Throw {@link IllegalArgumentException} + * if timestamp cannot be recognized. + */ + @Nullable + private static Long asMsSinceEpoch(@Nullable String timestamp) { + if (Strings.isNullOrEmpty(timestamp)) { + return null; + } + try { + // Try parsing as milliseconds since epoch. Note there is no way to parse a + // string in RFC 3339 format here. + // Expected IllegalArgumentException if parsing fails; we use that to fall back + // to RFC 3339. + return Long.parseLong(timestamp); + } catch (IllegalArgumentException e1) { + // Try parsing as RFC3339 string. DateTime.parseRfc3339 will throw an + // IllegalArgumentException if parsing fails, and the caller should handle. + return DateTime.parseRfc3339(timestamp).getValue(); + } + } + + /** + * Return the timestamp (in ms since unix epoch) to use for a Pubsub message with {@code + * attributes} and {@code pubsubTimestamp}. + * + *

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

              NOTE: This class is {@link Serializable} only to support the {@link PubsubTestClient}. + * Java serialization is never used for non-test clients. + */ + static class OutgoingMessage implements Serializable { + /** + * Underlying (encoded) element. + */ + public final byte[] elementBytes; + + public final Map attributes; + + /** + * Timestamp for element (ms since epoch). + */ + public final long timestampMsSinceEpoch; + + /** + * If using an id label, the record id to associate with this record's metadata so the receiver + * can reject duplicates. Otherwise {@literal null}. + */ + @Nullable + public final String recordId; + + public OutgoingMessage(byte[] elementBytes, Map attributes, + long timestampMsSinceEpoch, @Nullable String recordId) { + this.elementBytes = elementBytes; + this.attributes = attributes; + this.timestampMsSinceEpoch = timestampMsSinceEpoch; + this.recordId = recordId; + } + + @Override + public String toString() { + return String.format("OutgoingMessage(%db, %dms)", + elementBytes.length, timestampMsSinceEpoch); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + + OutgoingMessage that = (OutgoingMessage) o; + + return timestampMsSinceEpoch == that.timestampMsSinceEpoch + && Arrays.equals(elementBytes, that.elementBytes) + && Objects.equal(attributes, that.attributes) + && Objects.equal(recordId, that.recordId); + } + + @Override + public int hashCode() { + return Objects.hashCode(Arrays.hashCode(elementBytes), attributes, timestampMsSinceEpoch, + recordId); + } + } + + /** + * A message received from Pubsub. + * + *

              NOTE: This class is {@link Serializable} only to support the {@link PubsubTestClient}. + * Java serialization is never used for non-test clients. + */ + static class IncomingMessage implements Serializable { + /** + * Underlying (encoded) element. + */ + public final byte[] elementBytes; + + public Map attributes; + + /** + * Timestamp for element (ms since epoch). Either Pubsub's processing time, + * or the custom timestamp associated with the message. + */ + public final long timestampMsSinceEpoch; + + /** + * Timestamp (in system time) at which we requested the message (ms since epoch). + */ + public final long requestTimeMsSinceEpoch; + + /** + * Id to pass back to Pubsub to acknowledge receipt of this message. + */ + public final String ackId; + + /** + * Id to pass to the runner to distinguish this message from all others. + */ + public final String recordId; + + public IncomingMessage( + byte[] elementBytes, + Map attributes, + long timestampMsSinceEpoch, + long requestTimeMsSinceEpoch, + String ackId, + String recordId) { + this.elementBytes = elementBytes; + this.attributes = attributes; + this.timestampMsSinceEpoch = timestampMsSinceEpoch; + this.requestTimeMsSinceEpoch = requestTimeMsSinceEpoch; + this.ackId = ackId; + this.recordId = recordId; + } + + public IncomingMessage withRequestTime(long requestTimeMsSinceEpoch) { + return new IncomingMessage(elementBytes, attributes, timestampMsSinceEpoch, + requestTimeMsSinceEpoch, ackId, recordId); + } + + @Override + public String toString() { + return String.format("IncomingMessage(%db, %dms)", + elementBytes.length, timestampMsSinceEpoch); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + + IncomingMessage that = (IncomingMessage) o; + + return timestampMsSinceEpoch == that.timestampMsSinceEpoch + && requestTimeMsSinceEpoch == that.requestTimeMsSinceEpoch + && ackId.equals(that.ackId) + && recordId.equals(that.recordId) + && Arrays.equals(elementBytes, that.elementBytes) + && Objects.equal(attributes, that.attributes); + } + + @Override + public int hashCode() { + return Objects.hashCode(Arrays.hashCode(elementBytes), attributes, timestampMsSinceEpoch, + requestTimeMsSinceEpoch, + ackId, recordId); + } + } + + /** + * Publish {@code outgoingMessages} to Pubsub {@code topic}. Return number of messages + * published. + */ + public abstract int publish(TopicPath topic, List outgoingMessages) + throws IOException; + + /** + * Request the next batch of up to {@code batchSize} messages from {@code subscription}. + * Return the received messages, or empty collection if none were available. Does not + * wait for messages to arrive if {@code returnImmediately} is {@literal true}. + * Returned messages will record their request time as {@code requestTimeMsSinceEpoch}. + */ + public abstract List pull( + long requestTimeMsSinceEpoch, + SubscriptionPath subscription, + int batchSize, + boolean returnImmediately) + throws IOException; + + /** + * Acknowldege messages from {@code subscription} with {@code ackIds}. + */ + public abstract void acknowledge(SubscriptionPath subscription, List ackIds) + throws IOException; + + /** + * Modify the ack deadline for messages from {@code subscription} with {@code ackIds} to + * be {@code deadlineSeconds} from now. + */ + public abstract void modifyAckDeadline( + SubscriptionPath subscription, List ackIds, + int deadlineSeconds) throws IOException; + + /** + * Create {@code topic}. + */ + public abstract void createTopic(TopicPath topic) throws IOException; + + /* + * Delete {@code topic}. + */ + public abstract void deleteTopic(TopicPath topic) throws IOException; + + /** + * Return a list of topics for {@code project}. + */ + public abstract List listTopics(ProjectPath project) throws IOException; + + /** + * Create {@code subscription} to {@code topic}. + */ + public abstract void createSubscription( + TopicPath topic, SubscriptionPath subscription, int ackDeadlineSeconds) throws IOException; + + /** + * Create a random subscription for {@code topic}. Return the {@link SubscriptionPath}. It + * is the responsibility of the caller to later delete the subscription. + */ + public SubscriptionPath createRandomSubscription( + ProjectPath project, TopicPath topic, int ackDeadlineSeconds) throws IOException { + // Create a randomized subscription derived from the topic name. + String subscriptionName = topic.getName() + "_beam_" + ThreadLocalRandom.current().nextLong(); + SubscriptionPath subscription = + PubsubClient + .subscriptionPathFromName(project.getId(), subscriptionName); + createSubscription(topic, subscription, ackDeadlineSeconds); + return subscription; + } + + /** + * Delete {@code subscription}. + */ + public abstract void deleteSubscription(SubscriptionPath subscription) throws IOException; + + /** + * Return a list of subscriptions for {@code topic} in {@code project}. + */ + public abstract List listSubscriptions(ProjectPath project, TopicPath topic) + throws IOException; + + /** + * Return the ack deadline, in seconds, for {@code subscription}. + */ + public abstract int ackDeadlineSeconds(SubscriptionPath subscription) throws IOException; + + /** + * Return {@literal true} if {@link #pull} will always return empty list. Actual clients + * will return {@literal false}. Test clients may return {@literal true} to signal that all + * expected messages have been pulled and the test may complete. + */ + public abstract boolean isEOF(); +} diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/io/PubsubHelper.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/io/PubsubHelper.java index f5cfc2baae443..15401b72a54b0 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/io/PubsubHelper.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/io/PubsubHelper.java @@ -24,8 +24,6 @@ import org.apache.beam.integration.nexmark.NexmarkUtils; import org.apache.beam.sdk.options.PubsubOptions; -import org.apache.beam.sdk.util.PubsubClient; -import org.apache.beam.sdk.util.PubsubJsonClient; /** * Helper for working with pubsub. diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/io/PubsubJsonClient.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/io/PubsubJsonClient.java new file mode 100644 index 0000000000000..b778a094022e7 --- /dev/null +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/io/PubsubJsonClient.java @@ -0,0 +1,318 @@ +/* + * 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.integration.nexmark.io; + +import static com.google.common.base.Preconditions.checkState; + +import com.google.api.client.http.HttpRequestInitializer; +import com.google.api.services.pubsub.Pubsub; +import com.google.api.services.pubsub.Pubsub.Builder; +import com.google.api.services.pubsub.model.AcknowledgeRequest; +import com.google.api.services.pubsub.model.ListSubscriptionsResponse; +import com.google.api.services.pubsub.model.ListTopicsResponse; +import com.google.api.services.pubsub.model.ModifyAckDeadlineRequest; +import com.google.api.services.pubsub.model.PublishRequest; +import com.google.api.services.pubsub.model.PublishResponse; +import com.google.api.services.pubsub.model.PubsubMessage; +import com.google.api.services.pubsub.model.PullRequest; +import com.google.api.services.pubsub.model.PullResponse; +import com.google.api.services.pubsub.model.ReceivedMessage; +import com.google.api.services.pubsub.model.Subscription; +import com.google.api.services.pubsub.model.Topic; +import com.google.auth.Credentials; +import com.google.auth.http.HttpCredentialsAdapter; +import com.google.cloud.hadoop.util.ChainingHttpRequestInitializer; +import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Strings; +import com.google.common.collect.ImmutableList; +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.TreeMap; +import javax.annotation.Nullable; +import org.apache.beam.sdk.options.PubsubOptions; +import org.apache.beam.sdk.util.RetryHttpRequestInitializer; +import org.apache.beam.sdk.util.Transport; + +/** + * A Pubsub client using JSON transport. + */ +class PubsubJsonClient extends PubsubClient { + + private static class PubsubJsonClientFactory implements PubsubClientFactory { + private static HttpRequestInitializer chainHttpRequestInitializer( + Credentials credential, HttpRequestInitializer httpRequestInitializer) { + if (credential == null) { + return httpRequestInitializer; + } else { + return new ChainingHttpRequestInitializer( + new HttpCredentialsAdapter(credential), + httpRequestInitializer); + } + } + + @Override + public PubsubClient newClient( + @Nullable String timestampLabel, @Nullable String idLabel, PubsubOptions options) + throws IOException { + Pubsub pubsub = new Builder( + Transport.getTransport(), + Transport.getJsonFactory(), + chainHttpRequestInitializer( + options.getGcpCredential(), + // Do not log 404. It clutters the output and is possibly even required by the caller. + new RetryHttpRequestInitializer(ImmutableList.of(404)))) + .setRootUrl(options.getPubsubRootUrl()) + .setApplicationName(options.getAppName()) + .setGoogleClientRequestInitializer(options.getGoogleApiTrace()) + .build(); + return new PubsubJsonClient(timestampLabel, idLabel, pubsub); + } + + @Override + public String getKind() { + return "Json"; + } + } + + /** + * Factory for creating Pubsub clients using Json transport. + */ + public static final PubsubClientFactory FACTORY = new PubsubJsonClientFactory(); + + /** + * Label to use for custom timestamps, or {@literal null} if should use Pubsub publish time + * instead. + */ + @Nullable + private final String timestampLabel; + + /** + * Label to use for custom ids, or {@literal null} if should use Pubsub provided ids. + */ + @Nullable + private final String idLabel; + + /** + * Underlying JSON transport. + */ + private Pubsub pubsub; + + @VisibleForTesting PubsubJsonClient( + @Nullable String timestampLabel, + @Nullable String idLabel, + Pubsub pubsub) { + this.timestampLabel = timestampLabel; + this.idLabel = idLabel; + this.pubsub = pubsub; + } + + @Override + public void close() { + // Nothing to close. + } + + @Override + public int publish(TopicPath topic, List outgoingMessages) + throws IOException { + List pubsubMessages = new ArrayList<>(outgoingMessages.size()); + for (OutgoingMessage outgoingMessage : outgoingMessages) { + PubsubMessage pubsubMessage = new PubsubMessage().encodeData(outgoingMessage.elementBytes); + + Map attributes = outgoingMessage.attributes; + if ((timestampLabel != null || idLabel != null) && attributes == null) { + attributes = new TreeMap<>(); + } + if (attributes != null) { + pubsubMessage.setAttributes(attributes); + } + + if (timestampLabel != null) { + attributes.put(timestampLabel, String.valueOf(outgoingMessage.timestampMsSinceEpoch)); + } + + if (idLabel != null && !Strings.isNullOrEmpty(outgoingMessage.recordId)) { + attributes.put(idLabel, outgoingMessage.recordId); + } + + pubsubMessages.add(pubsubMessage); + } + PublishRequest request = new PublishRequest().setMessages(pubsubMessages); + PublishResponse response = pubsub.projects() + .topics() + .publish(topic.getPath(), request) + .execute(); + return response.getMessageIds().size(); + } + + @Override + public List pull( + long requestTimeMsSinceEpoch, + SubscriptionPath subscription, + int batchSize, + boolean returnImmediately) throws IOException { + PullRequest request = new PullRequest() + .setReturnImmediately(returnImmediately) + .setMaxMessages(batchSize); + PullResponse response = pubsub.projects() + .subscriptions() + .pull(subscription.getPath(), request) + .execute(); + if (response.getReceivedMessages() == null || response.getReceivedMessages().size() == 0) { + return ImmutableList.of(); + } + List incomingMessages = new ArrayList<>(response.getReceivedMessages().size()); + for (ReceivedMessage message : response.getReceivedMessages()) { + PubsubMessage pubsubMessage = message.getMessage(); + @Nullable Map attributes = pubsubMessage.getAttributes(); + + // Payload. + byte[] elementBytes = pubsubMessage.decodeData(); + + // Timestamp. + long timestampMsSinceEpoch = + extractTimestamp(timestampLabel, message.getMessage().getPublishTime(), attributes); + + // Ack id. + String ackId = message.getAckId(); + checkState(!Strings.isNullOrEmpty(ackId)); + + // Record id, if any. + @Nullable String recordId = null; + if (idLabel != null && attributes != null) { + recordId = attributes.get(idLabel); + } + if (Strings.isNullOrEmpty(recordId)) { + // Fall back to the Pubsub provided message id. + recordId = pubsubMessage.getMessageId(); + } + + incomingMessages.add(new IncomingMessage(elementBytes, attributes, timestampMsSinceEpoch, + requestTimeMsSinceEpoch, ackId, recordId)); + } + + return incomingMessages; + } + + @Override + public void acknowledge(SubscriptionPath subscription, List ackIds) throws IOException { + AcknowledgeRequest request = new AcknowledgeRequest().setAckIds(ackIds); + pubsub.projects() + .subscriptions() + .acknowledge(subscription.getPath(), request) + .execute(); // ignore Empty result. + } + + @Override + public void modifyAckDeadline( + SubscriptionPath subscription, List ackIds, int deadlineSeconds) + throws IOException { + ModifyAckDeadlineRequest request = + new ModifyAckDeadlineRequest().setAckIds(ackIds) + .setAckDeadlineSeconds(deadlineSeconds); + pubsub.projects() + .subscriptions() + .modifyAckDeadline(subscription.getPath(), request) + .execute(); // ignore Empty result. + } + + @Override + public void createTopic(TopicPath topic) throws IOException { + pubsub.projects() + .topics() + .create(topic.getPath(), new Topic()) + .execute(); // ignore Topic result. + } + + @Override + public void deleteTopic(TopicPath topic) throws IOException { + pubsub.projects() + .topics() + .delete(topic.getPath()) + .execute(); // ignore Empty result. + } + + @Override + public List listTopics(ProjectPath project) throws IOException { + ListTopicsResponse response = pubsub.projects() + .topics() + .list(project.getPath()) + .execute(); + if (response.getTopics() == null || response.getTopics().isEmpty()) { + return ImmutableList.of(); + } + List topics = new ArrayList<>(response.getTopics().size()); + for (Topic topic : response.getTopics()) { + topics.add(topicPathFromPath(topic.getName())); + } + return topics; + } + + @Override + public void createSubscription( + TopicPath topic, SubscriptionPath subscription, + int ackDeadlineSeconds) throws IOException { + Subscription request = new Subscription() + .setTopic(topic.getPath()) + .setAckDeadlineSeconds(ackDeadlineSeconds); + pubsub.projects() + .subscriptions() + .create(subscription.getPath(), request) + .execute(); // ignore Subscription result. + } + + @Override + public void deleteSubscription(SubscriptionPath subscription) throws IOException { + pubsub.projects() + .subscriptions() + .delete(subscription.getPath()) + .execute(); // ignore Empty result. + } + + @Override + public List listSubscriptions(ProjectPath project, TopicPath topic) + throws IOException { + ListSubscriptionsResponse response = pubsub.projects() + .subscriptions() + .list(project.getPath()) + .execute(); + if (response.getSubscriptions() == null || response.getSubscriptions().isEmpty()) { + return ImmutableList.of(); + } + List subscriptions = new ArrayList<>(response.getSubscriptions().size()); + for (Subscription subscription : response.getSubscriptions()) { + if (subscription.getTopic().equals(topic.getPath())) { + subscriptions.add(subscriptionPathFromPath(subscription.getName())); + } + } + return subscriptions; + } + + @Override + public int ackDeadlineSeconds(SubscriptionPath subscription) throws IOException { + Subscription response = pubsub.projects().subscriptions().get(subscription.getPath()).execute(); + return response.getAckDeadlineSeconds(); + } + + @Override + public boolean isEOF() { + return false; + } +} diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/io/PubsubTestClient.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/io/PubsubTestClient.java new file mode 100644 index 0000000000000..125a8d69c4ae7 --- /dev/null +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/io/PubsubTestClient.java @@ -0,0 +1,436 @@ +/* + * 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.integration.nexmark.io; + +import static com.google.common.base.Preconditions.checkState; + +import com.google.api.client.util.Clock; +import com.google.common.collect.Lists; +import com.google.common.collect.Sets; +import java.io.Closeable; +import java.io.IOException; +import java.io.Serializable; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Set; +import javax.annotation.Nullable; +import org.apache.beam.sdk.options.PubsubOptions; + +/** + * A (partial) implementation of {@link PubsubClient} for use by unit tests. Only suitable for + * testing {@link #publish}, {@link #pull}, {@link #acknowledge} and {@link #modifyAckDeadline} + * methods. Relies on statics to mimic the Pubsub service, though we try to hide that. + */ +class PubsubTestClient extends PubsubClient implements Serializable { + /** + * Mimic the state of the simulated Pubsub 'service'. + * + *

              Note that the {@link PubsubTestClientFactory} is serialized/deserialized even when running + * test pipelines. Meanwhile it is valid for multiple {@link PubsubTestClient}s to be created + * from the same client factory and run in parallel. Thus we can't enforce aliasing of the + * following data structures over all clients and must resort to a static. + */ + private static class State { + /** + * True if has been primed for a test but not yet validated. + */ + boolean isActive; + + /** + * Publish mode only: Only publish calls for this topic are allowed. + */ + @Nullable + TopicPath expectedTopic; + + /** + * Publish mode only: Messages yet to seen in a {@link #publish} call. + */ + @Nullable + Set remainingExpectedOutgoingMessages; + + /** + * Publish mode only: Messages which should throw when first sent to simulate transient publish + * failure. + */ + @Nullable + Set remainingFailingOutgoingMessages; + + /** + * Pull mode only: Clock from which to get current time. + */ + @Nullable + Clock clock; + + /** + * Pull mode only: Only pull calls for this subscription are allowed. + */ + @Nullable + SubscriptionPath expectedSubscription; + + /** + * Pull mode only: Timeout to simulate. + */ + int ackTimeoutSec; + + /** + * Pull mode only: Messages waiting to be received by a {@link #pull} call. + */ + @Nullable + List remainingPendingIncomingMessages; + + /** + * Pull mode only: Messages which have been returned from a {@link #pull} call and + * not yet ACKed by an {@link #acknowledge} call. + */ + @Nullable + Map pendingAckIncomingMessages; + + /** + * Pull mode only: When above messages are due to have their ACK deadlines expire. + */ + @Nullable + Map ackDeadline; + } + + private static final State STATE = new State(); + + /** Closing the factory will validate all expected messages were processed. */ + public interface PubsubTestClientFactory + extends PubsubClientFactory, Closeable, Serializable { + } + + /** + * Return a factory for testing publishers. Only one factory may be in-flight at a time. + * The factory must be closed when the test is complete, at which point final validation will + * occur. + */ + static PubsubTestClientFactory createFactoryForPublish( + final TopicPath expectedTopic, + final Iterable expectedOutgoingMessages, + final Iterable failingOutgoingMessages) { + synchronized (STATE) { + checkState(!STATE.isActive, "Test still in flight"); + STATE.expectedTopic = expectedTopic; + STATE.remainingExpectedOutgoingMessages = Sets.newHashSet(expectedOutgoingMessages); + STATE.remainingFailingOutgoingMessages = Sets.newHashSet(failingOutgoingMessages); + STATE.isActive = true; + } + return new PubsubTestClientFactory() { + @Override + public PubsubClient newClient( + @Nullable String timestampLabel, @Nullable String idLabel, PubsubOptions options) + throws IOException { + return new PubsubTestClient(); + } + + @Override + public String getKind() { + return "PublishTest"; + } + + @Override + public void close() { + synchronized (STATE) { + checkState(STATE.isActive, "No test still in flight"); + checkState(STATE.remainingExpectedOutgoingMessages.isEmpty(), + "Still waiting for %s messages to be published", + STATE.remainingExpectedOutgoingMessages.size()); + STATE.isActive = false; + STATE.remainingExpectedOutgoingMessages = null; + } + } + }; + } + + /** + * Return a factory for testing subscribers. Only one factory may be in-flight at a time. + * The factory must be closed when the test in complete + */ + public static PubsubTestClientFactory createFactoryForPull( + final Clock clock, + final SubscriptionPath expectedSubscription, + final int ackTimeoutSec, + final Iterable expectedIncomingMessages) { + synchronized (STATE) { + checkState(!STATE.isActive, "Test still in flight"); + STATE.clock = clock; + STATE.expectedSubscription = expectedSubscription; + STATE.ackTimeoutSec = ackTimeoutSec; + STATE.remainingPendingIncomingMessages = Lists.newArrayList(expectedIncomingMessages); + STATE.pendingAckIncomingMessages = new HashMap<>(); + STATE.ackDeadline = new HashMap<>(); + STATE.isActive = true; + } + return new PubsubTestClientFactory() { + @Override + public PubsubClient newClient( + @Nullable String timestampLabel, @Nullable String idLabel, PubsubOptions options) + throws IOException { + return new PubsubTestClient(); + } + + @Override + public String getKind() { + return "PullTest"; + } + + @Override + public void close() { + synchronized (STATE) { + checkState(STATE.isActive, "No test still in flight"); + checkState(STATE.remainingPendingIncomingMessages.isEmpty(), + "Still waiting for %s messages to be pulled", + STATE.remainingPendingIncomingMessages.size()); + checkState(STATE.pendingAckIncomingMessages.isEmpty(), + "Still waiting for %s messages to be ACKed", + STATE.pendingAckIncomingMessages.size()); + checkState(STATE.ackDeadline.isEmpty(), + "Still waiting for %s messages to be ACKed", + STATE.ackDeadline.size()); + STATE.isActive = false; + STATE.remainingPendingIncomingMessages = null; + STATE.pendingAckIncomingMessages = null; + STATE.ackDeadline = null; + } + } + }; + } + + public static PubsubTestClientFactory createFactoryForCreateSubscription() { + return new PubsubTestClientFactory() { + int numCalls = 0; + + @Override + public void close() throws IOException { + checkState( + numCalls == 1, "Expected exactly one subscription to be created, got %s", numCalls); + } + + @Override + public PubsubClient newClient( + @Nullable String timestampLabel, @Nullable String idLabel, PubsubOptions options) + throws IOException { + return new PubsubTestClient() { + @Override + public void createSubscription( + TopicPath topic, SubscriptionPath subscription, int ackDeadlineSeconds) + throws IOException { + checkState(numCalls == 0, "Expected at most one subscription to be created"); + numCalls++; + } + }; + } + + @Override + public String getKind() { + return "CreateSubscriptionTest"; + } + }; + } + + /** + * Return true if in pull mode. + */ + private boolean inPullMode() { + checkState(STATE.isActive, "No test is active"); + return STATE.expectedSubscription != null; + } + + /** + * Return true if in publish mode. + */ + private boolean inPublishMode() { + checkState(STATE.isActive, "No test is active"); + return STATE.expectedTopic != null; + } + + /** + * For subscription mode only: + * Track progression of time according to the {@link Clock} passed . This will simulate Pubsub + * expiring + * outstanding ACKs. + */ + public void advance() { + synchronized (STATE) { + checkState(inPullMode(), "Can only advance in pull mode"); + // Any messages who's ACKs timed out are available for re-pulling. + Iterator> deadlineItr = STATE.ackDeadline.entrySet().iterator(); + while (deadlineItr.hasNext()) { + Map.Entry entry = deadlineItr.next(); + if (entry.getValue() <= STATE.clock.currentTimeMillis()) { + STATE.remainingPendingIncomingMessages.add( + STATE.pendingAckIncomingMessages.remove(entry.getKey())); + deadlineItr.remove(); + } + } + } + } + + @Override + public void close() { + } + + @Override + public int publish( + TopicPath topic, List outgoingMessages) throws IOException { + synchronized (STATE) { + checkState(inPublishMode(), "Can only publish in publish mode"); + checkState(topic.equals(STATE.expectedTopic), "Topic %s does not match expected %s", topic, + STATE.expectedTopic); + for (OutgoingMessage outgoingMessage : outgoingMessages) { + if (STATE.remainingFailingOutgoingMessages.remove(outgoingMessage)) { + throw new RuntimeException("Simulating failure for " + outgoingMessage); + } + checkState(STATE.remainingExpectedOutgoingMessages.remove(outgoingMessage), + "Unexpected outgoing message %s", outgoingMessage); + } + return outgoingMessages.size(); + } + } + + @Override + public List pull( + long requestTimeMsSinceEpoch, SubscriptionPath subscription, int batchSize, + boolean returnImmediately) throws IOException { + synchronized (STATE) { + checkState(inPullMode(), "Can only pull in pull mode"); + long now = STATE.clock.currentTimeMillis(); + checkState(requestTimeMsSinceEpoch == now, + "Simulated time %s does not match request time %s", now, requestTimeMsSinceEpoch); + checkState(subscription.equals(STATE.expectedSubscription), + "Subscription %s does not match expected %s", subscription, + STATE.expectedSubscription); + checkState(returnImmediately, "Pull only supported if returning immediately"); + + List incomingMessages = new ArrayList<>(); + Iterator pendItr = STATE.remainingPendingIncomingMessages.iterator(); + while (pendItr.hasNext()) { + IncomingMessage incomingMessage = pendItr.next(); + pendItr.remove(); + IncomingMessage incomingMessageWithRequestTime = + incomingMessage.withRequestTime(requestTimeMsSinceEpoch); + incomingMessages.add(incomingMessageWithRequestTime); + STATE.pendingAckIncomingMessages.put(incomingMessageWithRequestTime.ackId, + incomingMessageWithRequestTime); + STATE.ackDeadline.put(incomingMessageWithRequestTime.ackId, + requestTimeMsSinceEpoch + STATE.ackTimeoutSec * 1000); + if (incomingMessages.size() >= batchSize) { + break; + } + } + return incomingMessages; + } + } + + @Override + public void acknowledge( + SubscriptionPath subscription, + List ackIds) throws IOException { + synchronized (STATE) { + checkState(inPullMode(), "Can only acknowledge in pull mode"); + checkState(subscription.equals(STATE.expectedSubscription), + "Subscription %s does not match expected %s", subscription, + STATE.expectedSubscription); + + for (String ackId : ackIds) { + checkState(STATE.ackDeadline.remove(ackId) != null, + "No message with ACK id %s is waiting for an ACK", ackId); + checkState(STATE.pendingAckIncomingMessages.remove(ackId) != null, + "No message with ACK id %s is waiting for an ACK", ackId); + } + } + } + + @Override + public void modifyAckDeadline( + SubscriptionPath subscription, List ackIds, int deadlineSeconds) throws IOException { + synchronized (STATE) { + checkState(inPullMode(), "Can only modify ack deadline in pull mode"); + checkState(subscription.equals(STATE.expectedSubscription), + "Subscription %s does not match expected %s", subscription, + STATE.expectedSubscription); + + for (String ackId : ackIds) { + if (deadlineSeconds > 0) { + checkState(STATE.ackDeadline.remove(ackId) != null, + "No message with ACK id %s is waiting for an ACK", ackId); + checkState(STATE.pendingAckIncomingMessages.containsKey(ackId), + "No message with ACK id %s is waiting for an ACK", ackId); + STATE.ackDeadline.put(ackId, STATE.clock.currentTimeMillis() + deadlineSeconds * 1000); + } else { + checkState(STATE.ackDeadline.remove(ackId) != null, + "No message with ACK id %s is waiting for an ACK", ackId); + IncomingMessage message = STATE.pendingAckIncomingMessages.remove(ackId); + checkState(message != null, "No message with ACK id %s is waiting for an ACK", ackId); + STATE.remainingPendingIncomingMessages.add(message); + } + } + } + } + + @Override + public void createTopic(TopicPath topic) throws IOException { + throw new UnsupportedOperationException(); + } + + @Override + public void deleteTopic(TopicPath topic) throws IOException { + throw new UnsupportedOperationException(); + } + + @Override + public List listTopics(ProjectPath project) throws IOException { + throw new UnsupportedOperationException(); + } + + @Override + public void createSubscription( + TopicPath topic, SubscriptionPath subscription, int ackDeadlineSeconds) throws IOException { + throw new UnsupportedOperationException(); + } + + @Override + public void deleteSubscription(SubscriptionPath subscription) throws IOException { + throw new UnsupportedOperationException(); + } + + @Override + public List listSubscriptions( + ProjectPath project, TopicPath topic) throws IOException { + throw new UnsupportedOperationException(); + } + + @Override + public int ackDeadlineSeconds(SubscriptionPath subscription) throws IOException { + synchronized (STATE) { + return STATE.ackTimeoutSec; + } + } + + @Override + public boolean isEOF() { + synchronized (STATE) { + checkState(inPullMode(), "Can only check EOF in pull mode"); + return STATE.remainingPendingIncomingMessages.isEmpty(); + } + } +} diff --git a/integration/java/pom.xml b/integration/java/pom.xml new file mode 100644 index 0000000000000..dcad4c3a76625 --- /dev/null +++ b/integration/java/pom.xml @@ -0,0 +1,37 @@ + + + + + 4.0.0 + + + org.apache.beam + beam-integration-parent + 0.7.0-SNAPSHOT + ../pom.xml + + + beam-integration-java-parent + pom + Apache Beam :: Integration Tests :: Java + + + nexmark + + + diff --git a/integration/pom.xml b/integration/pom.xml new file mode 100644 index 0000000000000..4839da5acbb4f --- /dev/null +++ b/integration/pom.xml @@ -0,0 +1,37 @@ + + + + + 4.0.0 + + + org.apache.beam + beam-parent + 0.7.0-SNAPSHOT + ../pom.xml + + + beam-integration-parent + pom + Apache Beam :: Integration Tests + + + java + + + diff --git a/pom.xml b/pom.xml index c92d391ccefba..bddbf1f90ae02 100644 --- a/pom.xml +++ b/pom.xml @@ -187,7 +187,7 @@ sdks runners examples - integration/java/nexmark + integration sdks/java/javadoc From e10d5783d8c8ed32008e29d99d5a4b1dd3e408a6 Mon Sep 17 00:00:00 2001 From: Etienne Chauchot Date: Fri, 14 Apr 2017 17:13:59 +0200 Subject: [PATCH 300/346] Fix Spark streaming termination via waitUntilFinish and timeout config issue #39 --- .../nexmark/NexmarkConfiguration.java | 61 ++++++++++++++++--- .../integration/nexmark/NexmarkOptions.java | 7 +++ .../integration/nexmark/NexmarkRunner.java | 2 +- 3 files changed, 62 insertions(+), 8 deletions(-) diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkConfiguration.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkConfiguration.java index d6cd80801163e..1da08b410cbde 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkConfiguration.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkConfiguration.java @@ -105,6 +105,12 @@ public class NexmarkConfiguration implements Serializable { @JsonProperty public int preloadSeconds = 0; + /** + * Timeout for stream pipelines to stop in seconds. + */ + @JsonProperty + public int streamTimeout = 240; + /** * If true, and in streaming mode, generate events only when they are due according to their * timestamp. @@ -275,6 +281,9 @@ public void overrideFromOptions(NexmarkOptions options) { if (options.getPreloadSeconds() != null) { preloadSeconds = options.getPreloadSeconds(); } + if (options.getStreamTimeout() != null) { + streamTimeout = options.getStreamTimeout(); + } if (options.getIsRateLimited() != null) { isRateLimited = options.getIsRateLimited(); } @@ -368,6 +377,7 @@ public NexmarkConfiguration clone() { result.rateUnit = rateUnit; result.ratePeriodSec = ratePeriodSec; result.preloadSeconds = preloadSeconds; + result.streamTimeout = streamTimeout; result.isRateLimited = isRateLimited; result.useWallclockEventTime = useWallclockEventTime; result.avgPersonByteSize = avgPersonByteSize; @@ -436,6 +446,9 @@ public String toShortString() { if (preloadSeconds != DEFAULT.preloadSeconds) { sb.append(String.format("; preloadSeconds:%d", preloadSeconds)); } + if (streamTimeout != DEFAULT.streamTimeout) { + sb.append(String.format("; streamTimeout:%d", streamTimeout)); + } if (isRateLimited != DEFAULT.isRateLimited) { sb.append(String.format("; isRateLimited:%s", isRateLimited)); } @@ -536,13 +549,44 @@ public static NexmarkConfiguration fromString(String string) { @Override public int hashCode() { - return Objects.hash(debug, query, sourceType, sinkType, pubSubMode, - numEvents, numEventGenerators, rateShape, firstEventRate, nextEventRate, rateUnit, - ratePeriodSec, preloadSeconds, isRateLimited, useWallclockEventTime, avgPersonByteSize, - avgAuctionByteSize, avgBidByteSize, hotAuctionRatio, hotSellersRatio, hotBiddersRatio, - windowSizeSec, windowPeriodSec, watermarkHoldbackSec, numInFlightAuctions, numActivePeople, - coderStrategy, cpuDelayMs, diskBusyBytes, auctionSkip, fanout, maxAuctionsWaitingTime, - occasionalDelaySec, probDelayedEvent, maxLogEvents, usePubsubPublishTime, + return Objects.hash( + debug, + query, + sourceType, + sinkType, + pubSubMode, + numEvents, + numEventGenerators, + rateShape, + firstEventRate, + nextEventRate, + rateUnit, + ratePeriodSec, + preloadSeconds, + streamTimeout, + isRateLimited, + useWallclockEventTime, + avgPersonByteSize, + avgAuctionByteSize, + avgBidByteSize, + hotAuctionRatio, + hotSellersRatio, + hotBiddersRatio, + windowSizeSec, + windowPeriodSec, + watermarkHoldbackSec, + numInFlightAuctions, + numActivePeople, + coderStrategy, + cpuDelayMs, + diskBusyBytes, + auctionSkip, + fanout, + maxAuctionsWaitingTime, + occasionalDelaySec, + probDelayedEvent, + maxLogEvents, + usePubsubPublishTime, outOfOrderGroupSize); } @@ -630,6 +674,9 @@ public boolean equals(Object obj) { if (preloadSeconds != other.preloadSeconds) { return false; } + if (streamTimeout != other.streamTimeout) { + return false; + } if (Double.doubleToLongBits(probDelayedEvent) != Double.doubleToLongBits(other.probDelayedEvent)) { return false; diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkOptions.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkOptions.java index e39f0a48dd109..5d093ae64bc43 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkOptions.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkOptions.java @@ -110,6 +110,13 @@ public interface NexmarkOptions extends PubsubOptions { void setPreloadSeconds(Integer preloadSeconds); + @Description( + "Time in seconds to wait in pipelineResult.waitUntilFinish(), useful in streaming mode") + @Nullable + Integer getStreamTimeout(); + + void setStreamTimeout(Integer preloadSeconds); + @Description("Number of unbounded sources to create events.") @Nullable Integer getNumEventGenerators(); diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkRunner.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkRunner.java index 3a0452f182b5d..ef5f0e26a30bf 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkRunner.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkRunner.java @@ -1230,7 +1230,7 @@ public NexmarkPerf run(NexmarkConfiguration runConfiguration) { waitForPublisherPreload(); } mainResult = p.run(); - mainResult.waitUntilFinish(); + mainResult.waitUntilFinish(Duration.standardSeconds(configuration.streamTimeout)); return monitor(query); } finally { // From b438fa7df16e5181f73b6103ac2f57430cd9e6f3 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Isma=C3=ABl=20Mej=C3=ADa?= Date: Wed, 19 Apr 2017 11:22:42 +0200 Subject: [PATCH 301/346] Remove Accumulators and switch to the Metrics API Fix compile after sideOutput and split refactor --- integration/java/nexmark/pom.xml | 6 +- .../beam/integration/nexmark/Monitor.java | 77 ++-- .../integration/nexmark/NexmarkQuery.java | 16 +- .../integration/nexmark/NexmarkRunner.java | 129 +++++-- .../integration/nexmark/NexmarkUtils.java | 107 +++--- .../beam/integration/nexmark/WinningBids.java | 102 +++-- .../nexmark/drivers/NexmarkGoogleRunner.java | 4 +- .../integration/nexmark/queries/Query0.java | 10 +- .../integration/nexmark/queries/Query10.java | 363 +++++++++--------- .../integration/nexmark/queries/Query3.java | 73 ++-- .../nexmark/sources/BoundedEventSource.java | 2 +- .../nexmark/sources/UnboundedEventSource.java | 2 +- .../sources/BoundedEventSourceTest.java | 2 +- 13 files changed, 448 insertions(+), 445 deletions(-) diff --git a/integration/java/nexmark/pom.xml b/integration/java/nexmark/pom.xml index 67d6117cf3185..103c18f28f703 100644 --- a/integration/java/nexmark/pom.xml +++ b/integration/java/nexmark/pom.xml @@ -29,7 +29,6 @@ beam-integration-java-nexmark Apache Beam :: Integration Tests :: Java :: Nexmark - jar @@ -227,6 +226,11 @@ beam-sdks-java-io-google-cloud-platform + + org.apache.beam + beam-sdks-java-extensions-gcp-core + + com.google.apis diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Monitor.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Monitor.java index 6370e4158bd28..cb4d71c957c83 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Monitor.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Monitor.java @@ -20,54 +20,55 @@ import java.io.Serializable; import org.apache.beam.integration.nexmark.model.KnownSize; -import org.apache.beam.sdk.transforms.Aggregator; +import org.apache.beam.sdk.metrics.Counter; +import org.apache.beam.sdk.metrics.Distribution; +import org.apache.beam.sdk.metrics.Metrics; 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.PTransform; import org.apache.beam.sdk.transforms.ParDo; -import org.apache.beam.sdk.transforms.Sum; import org.apache.beam.sdk.values.PCollection; /** - * A monitor of elements with support for later retrieving their aggregators. + * A monitor of elements with support for later retrieving their metrics. * * @param Type of element we are monitoring. */ public class Monitor implements Serializable { private class MonitorDoFn extends DoFn { - public final Aggregator elementCounter = - createAggregator(counterNamePrefix + "_elements", Sum.ofLongs()); - public final Aggregator bytesCounter = - createAggregator(counterNamePrefix + "_bytes", Sum.ofLongs()); - public final Aggregator startTime = - createAggregator(counterNamePrefix + "_startTime", Min.ofLongs()); - public final Aggregator endTime = - createAggregator(counterNamePrefix + "_endTime", Max.ofLongs()); - public final Aggregator startTimestamp = - createAggregator("startTimestamp", Min.ofLongs()); - public final Aggregator endTimestamp = - createAggregator("endTimestamp", Max.ofLongs()); + final Counter elementCounter = + Metrics.counter(name , prefix + ".elements"); + final Counter bytesCounter = + Metrics.counter(name , prefix + ".bytes"); + final Distribution startTime = + Metrics.distribution(name , prefix + ".startTime"); + final Distribution endTime = + Metrics.distribution(name , prefix + ".endTime"); + final Distribution startTimestamp = + Metrics.distribution(name , prefix + ".startTimestamp"); + final Distribution endTimestamp = + Metrics.distribution(name , prefix + ".endTimestamp"); @ProcessElement public void processElement(ProcessContext c) { - elementCounter.addValue(1L); - bytesCounter.addValue(c.element().sizeInBytes()); + elementCounter.inc(); + bytesCounter.inc(c.element().sizeInBytes()); long now = System.currentTimeMillis(); - startTime.addValue(now); - endTime.addValue(now); - startTimestamp.addValue(c.timestamp().getMillis()); - endTimestamp.addValue(c.timestamp().getMillis()); + startTime.update(now); + endTime.update(now); + startTimestamp.update(c.timestamp().getMillis()); + endTimestamp.update(c.timestamp().getMillis()); c.output(c.element()); } } + public final String name; + public final String prefix; final MonitorDoFn doFn; final PTransform, PCollection> transform; - private String counterNamePrefix; - public Monitor(String name, String counterNamePrefix) { - this.counterNamePrefix = counterNamePrefix; + public Monitor(String name, String prefix) { + this.name = name; + this.prefix = prefix; doFn = new MonitorDoFn(); transform = ParDo.of(doFn); } @@ -75,28 +76,4 @@ public Monitor(String name, String counterNamePrefix) { public PTransform, PCollection> getTransform() { return transform; } - - public Aggregator getElementCounter() { - return doFn.elementCounter; - } - - public Aggregator getBytesCounter() { - return doFn.bytesCounter; - } - - public Aggregator getStartTime() { - return doFn.startTime; - } - - public Aggregator getEndTime() { - return doFn.endTime; - } - - public Aggregator getStartTimestamp() { - return doFn.startTimestamp; - } - - public Aggregator getEndTimestamp() { - return doFn.endTimestamp; - } } diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkQuery.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkQuery.java index e1cd493ed7e6c..ab1c3052d2b06 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkQuery.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkQuery.java @@ -17,13 +17,13 @@ */ package org.apache.beam.integration.nexmark; -import javax.annotation.Nullable; import org.apache.beam.integration.nexmark.model.Auction; import org.apache.beam.integration.nexmark.model.Bid; import org.apache.beam.integration.nexmark.model.Event; import org.apache.beam.integration.nexmark.model.KnownSize; import org.apache.beam.integration.nexmark.model.Person; -import org.apache.beam.sdk.transforms.Aggregator; +import org.apache.beam.sdk.metrics.Counter; +import org.apache.beam.sdk.metrics.Metrics; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.Filter; import org.apache.beam.sdk.transforms.PTransform; @@ -206,6 +206,7 @@ public PCollection expand(PCollection input) { public final Monitor eventMonitor; public final Monitor resultMonitor; public final Monitor endOfStreamMonitor; + protected final Counter fatalCounter; protected NexmarkQuery(NexmarkConfiguration configuration, String name) { super(name); @@ -214,22 +215,15 @@ protected NexmarkQuery(NexmarkConfiguration configuration, String name) { eventMonitor = new Monitor<>(name + ".Events", "event"); resultMonitor = new Monitor<>(name + ".Results", "result"); endOfStreamMonitor = new Monitor<>(name + ".EndOfStream", "end"); + fatalCounter = Metrics.counter(name , "fatal"); } else { eventMonitor = null; resultMonitor = null; endOfStreamMonitor = null; + fatalCounter = null; } } - /** - * Return the aggregator which counts fatal errors in this query. Return null if no such - * aggregator. - */ - @Nullable - public Aggregator getFatalCount() { - return null; - } - /** * Implement the actual query. All we know about the result is it has a known encoded size. */ diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkRunner.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkRunner.java index ef5f0e26a30bf..87314cebddce8 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkRunner.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkRunner.java @@ -24,14 +24,13 @@ import com.google.api.services.bigquery.model.TableSchema; import com.google.common.base.Strings; 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; import java.util.Collections; import java.util.List; +import java.util.NoSuchElementException; import java.util.concurrent.ThreadLocalRandom; import javax.annotation.Nullable; import org.apache.beam.integration.nexmark.io.PubsubHelper; @@ -63,15 +62,18 @@ import org.apache.beam.integration.nexmark.queries.Query8Model; import org.apache.beam.integration.nexmark.queries.Query9; import org.apache.beam.integration.nexmark.queries.Query9Model; -import org.apache.beam.sdk.AggregatorRetrievalException; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.PipelineResult; import org.apache.beam.sdk.io.AvroIO; import org.apache.beam.sdk.io.TextIO; import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO; import org.apache.beam.sdk.io.gcp.pubsub.PubsubIO; +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.MetricResult; +import org.apache.beam.sdk.metrics.MetricsFilter; 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.values.PCollection; @@ -186,38 +188,59 @@ private PubsubHelper getPubsub() { protected abstract int maxNumWorkers(); /** - * Return the current value for a long counter, or -1 if can't be retrieved. + * Return the current value for a long counter, or a default value if can't be retrieved. + * Note this uses only attempted metrics because some runners don't support committed metrics. */ - protected long getLong(PipelineResult job, Aggregator aggregator) { + protected long getCounterMetric(PipelineResult result, String namespace, String name, + long defaultValue) { + //TODO Ismael calc this only once + MetricQueryResults metrics = result.metrics().queryMetrics( + MetricsFilter.builder().addNameFilter(MetricNameFilter.named(namespace, name)).build()); + Iterable> counters = metrics.counters(); try { - Collection values = job.getAggregatorValues(aggregator).getValues(); - if (values.size() != 1) { - return -1; - } - return Iterables.getOnlyElement(values); - } catch (AggregatorRetrievalException e) { - return -1; + MetricResult metricResult = counters.iterator().next(); + return metricResult.attempted(); + } catch (NoSuchElementException e) { + //TODO Ismael } + return defaultValue; } /** - * Return the current value for a time counter, or -1 if can't be retrieved. + * Return the current value for a long counter, or a default value if can't be retrieved. + * Note this uses only attempted metrics because some runners don't support committed metrics. */ - protected long getTimestamp( - long now, PipelineResult job, Aggregator aggregator) { + protected long getDistributionMetric(PipelineResult result, String namespace, String name, + DistributionType distType, long defaultValue) { + MetricQueryResults metrics = result.metrics().queryMetrics( + MetricsFilter.builder().addNameFilter(MetricNameFilter.named(namespace, name)).build()); + Iterable> distributions = metrics.distributions(); try { - Collection values = job.getAggregatorValues(aggregator).getValues(); - if (values.size() != 1) { - return -1; - } - long value = Iterables.getOnlyElement(values); - if (Math.abs(value - now) > Duration.standardDays(10000).getMillis()) { - return -1; + MetricResult distributionResult = distributions.iterator().next(); + if (distType.equals(DistributionType.MIN)) { + return distributionResult.attempted().min(); + } else if (distType.equals(DistributionType.MAX)) { + return distributionResult.attempted().max(); + } else { + //TODO Ismael } - return value; - } catch (AggregatorRetrievalException e) { + } catch (NoSuchElementException e) { + //TODO Ismael + } + return defaultValue; + } + + private enum DistributionType {MIN, MAX} + + /** + * Return the current value for a time counter, or -1 if can't be retrieved. + */ + protected long getTimestampMetric(long now, long value) { + //TODO Ismael improve doc + if (Math.abs(value - now) > Duration.standardDays(10000).getMillis()) { return -1; } + return value; } /** @@ -294,21 +317,46 @@ protected void captureSteadyState(NexmarkPerf perf, * Return the current performance given {@code eventMonitor} and {@code resultMonitor}. */ private NexmarkPerf currentPerf( - long startMsSinceEpoch, long now, PipelineResult job, + long startMsSinceEpoch, long now, PipelineResult result, List snapshots, Monitor eventMonitor, Monitor resultMonitor) { NexmarkPerf perf = new NexmarkPerf(); - long numEvents = getLong(job, eventMonitor.getElementCounter()); - long numEventBytes = getLong(job, eventMonitor.getBytesCounter()); - long eventStart = getTimestamp(now, job, eventMonitor.getStartTime()); - long eventEnd = getTimestamp(now, job, eventMonitor.getEndTime()); - long numResults = getLong(job, resultMonitor.getElementCounter()); - long numResultBytes = getLong(job, resultMonitor.getBytesCounter()); - long resultStart = getTimestamp(now, job, resultMonitor.getStartTime()); - long resultEnd = getTimestamp(now, job, resultMonitor.getEndTime()); - long timestampStart = getTimestamp(now, job, resultMonitor.getStartTimestamp()); - long timestampEnd = getTimestamp(now, job, resultMonitor.getEndTimestamp()); + long numEvents = + getCounterMetric(result, eventMonitor.name, eventMonitor.prefix + ".elements", -1); + long numEventBytes = + getCounterMetric(result, eventMonitor.name, eventMonitor.prefix + ".bytes", -1); + long eventStart = + getTimestampMetric(now, + getDistributionMetric(result, eventMonitor.name, eventMonitor.prefix + ".startTime", + DistributionType.MIN, -1)); + long eventEnd = + getTimestampMetric(now, + getDistributionMetric(result, eventMonitor.name, eventMonitor.prefix + ".endTime", + DistributionType.MAX, -1)); + + long numResults = + getCounterMetric(result, resultMonitor.name, resultMonitor.prefix + ".elements", -1); + long numResultBytes = + getCounterMetric(result, resultMonitor.name, resultMonitor.prefix + ".bytes", -1); + long resultStart = + getTimestampMetric(now, + getDistributionMetric(result, resultMonitor.name, resultMonitor.prefix + ".startTime", + DistributionType.MIN, -1)); + long resultEnd = + getTimestampMetric(now, + getDistributionMetric(result, resultMonitor.name, resultMonitor.prefix + ".endTime", + DistributionType.MAX, -1)); + long timestampStart = + getTimestampMetric(now, + getDistributionMetric(result, + resultMonitor.name, resultMonitor.prefix + ".startTimestamp", + DistributionType.MIN, -1)); + long timestampEnd = + getTimestampMetric(now, + getDistributionMetric(result, + resultMonitor.name, resultMonitor.prefix + ".endTimestamp", + DistributionType.MAX, -1)); long effectiveEnd = -1; if (eventEnd >= 0 && resultEnd >= 0) { @@ -372,7 +420,7 @@ private NexmarkPerf currentPerf( perf.shutdownDelaySec = (now - resultEnd) / 1000.0; } - perf.jobId = getJobId(job); + perf.jobId = getJobId(result); // As soon as available, try to capture cumulative cost at this point too. NexmarkPerf.ProgressSnapshot snapshot = new NexmarkPerf.ProgressSnapshot(); @@ -574,9 +622,10 @@ protected NexmarkPerf monitor(NexmarkQuery query) { if (options.isStreaming() && !waitingForShutdown) { Duration quietFor = new Duration(lastActivityMsSinceEpoch, now); - if (query.getFatalCount() != null && getLong(job, query.getFatalCount()) > 0) { + long fatalCount = getCounterMetric(job, query.getName(), "fatal", 0); + if (fatalCount > 0) { NexmarkUtils.console("job has fatal errors, cancelling."); - errors.add(String.format("Pipeline reported %s fatal errors", query.getFatalCount())); + errors.add(String.format("Pipeline reported %s fatal errors", fatalCount)); waitingForShutdown = true; } else if (configuration.debug && configuration.numEvents > 0 && currPerf.numEvents == configuration.numEvents @@ -1033,7 +1082,7 @@ public void processElement(ProcessContext c) { if (c.element().hashCode() % 2 == 0) { c.output(c.element()); } else { - c.sideOutput(SIDE, c.element()); + c.output(SIDE, c.element()); } } } diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkUtils.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkUtils.java index a47ebcc2ec4c3..18589c44e46de 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkUtils.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkUtils.java @@ -53,12 +53,12 @@ import org.apache.beam.sdk.coders.CustomCoder; import org.apache.beam.sdk.coders.SerializableCoder; import org.apache.beam.sdk.io.Read; -import org.apache.beam.sdk.transforms.Aggregator; +import org.apache.beam.sdk.metrics.Counter; +import org.apache.beam.sdk.metrics.Metrics; import org.apache.beam.sdk.transforms.Combine; 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; import org.apache.beam.sdk.transforms.windowing.AfterPane; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.GlobalWindows; @@ -419,48 +419,42 @@ public static PTransform> streamEventsSource( */ public static ParDo.SingleOutput snoop(final String name) { return ParDo.of(new DoFn() { - final Aggregator eventCounter = - createAggregator("events", Sum.ofLongs()); - final Aggregator newPersonCounter = - createAggregator("newPersons", Sum.ofLongs()); - final Aggregator newAuctionCounter = - createAggregator("newAuctions", Sum.ofLongs()); - final Aggregator bidCounter = - createAggregator("bids", Sum.ofLongs()); - final Aggregator endOfStreamCounter = - createAggregator("endOfStream", Sum.ofLongs()); - - @ProcessElement - public void processElement(ProcessContext c) { - eventCounter.addValue(1L); - if (c.element().newPerson != null) { - newPersonCounter.addValue(1L); - } else if (c.element().newAuction != null) { - newAuctionCounter.addValue(1L); - } else if (c.element().bid != null) { - bidCounter.addValue(1L); - } else { - endOfStreamCounter.addValue(1L); - } - info("%s snooping element %s", name, c.element()); - c.output(c.element()); - } - }); + final Counter eventCounter = Metrics.counter(name, "events"); + final Counter newPersonCounter = Metrics.counter(name, "newPersons"); + final Counter newAuctionCounter = Metrics.counter(name, "newAuctions"); + final Counter bidCounter = Metrics.counter(name, "bids"); + final Counter endOfStreamCounter = Metrics.counter(name, "endOfStream"); + + @ProcessElement + public void processElement(ProcessContext c) { + eventCounter.inc(); + if (c.element().newPerson != null) { + newPersonCounter.inc(); + } else if (c.element().newAuction != null) { + newAuctionCounter.inc(); + } else if (c.element().bid != null) { + bidCounter.inc(); + } else { + endOfStreamCounter.inc(); + } + info("%s snooping element %s", name, c.element()); + c.output(c.element()); + } + }); } /** * Return a transform to count and discard each element. */ - public static ParDo.SingleOutput devNull(String name) { + public static ParDo.SingleOutput devNull(final String name) { return ParDo.of(new DoFn() { - final Aggregator discardCounter = - createAggregator("discarded", Sum.ofLongs()); + final Counter discardedCounterMetric = Metrics.counter(name, "discarded"); - @ProcessElement - public void processElement(ProcessContext c) { - discardCounter.addValue(1L); - } - }); + @ProcessElement + public void processElement(ProcessContext c) { + discardedCounterMetric.inc(); + } + }); } /** @@ -468,28 +462,27 @@ public void processElement(ProcessContext c) { */ public static ParDo.SingleOutput log(final String name) { return ParDo.of(new DoFn() { - @ProcessElement - public void processElement(ProcessContext c) { - LOG.info("%s: %s", name, c.element()); - c.output(c.element()); - } - }); + @ProcessElement + public void processElement(ProcessContext c) { + LOG.info("%s: %s", name, c.element()); + c.output(c.element()); + } + }); } /** * Return a transform to format each element as a string. */ - public static ParDo.SingleOutput format(String name) { + public static ParDo.SingleOutput format(final String name) { return ParDo.of(new DoFn() { - final Aggregator recordCounter = - createAggregator("records", Sum.ofLongs()); + final Counter recordCounterMetric = Metrics.counter(name, "records"); - @ProcessElement - public void processElement(ProcessContext c) { - recordCounter.addValue(1L); - c.output(c.element().toString()); - } - }); + @ProcessElement + public void processElement(ProcessContext c) { + recordCounterMetric.inc(); + c.output(c.element().toString()); + } + }); } /** @@ -497,11 +490,11 @@ public void processElement(ProcessContext c) { */ public static ParDo.SingleOutput> stamp(String name) { return ParDo.of(new DoFn>() { - @ProcessElement - public void processElement(ProcessContext c) { - c.output(TimestampedValue.of(c.element(), c.timestamp())); - } - }); + @ProcessElement + public void processElement(ProcessContext c) { + c.output(TimestampedValue.of(c.element(), c.timestamp())); + } + }); } /** diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/WinningBids.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/WinningBids.java index 9f1ddf898da25..f2566b805cc9a 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/WinningBids.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/WinningBids.java @@ -40,11 +40,11 @@ import org.apache.beam.sdk.coders.CoderException; import org.apache.beam.sdk.coders.VarIntCoder; import org.apache.beam.sdk.coders.VarLongCoder; -import org.apache.beam.sdk.transforms.Aggregator; +import org.apache.beam.sdk.metrics.Counter; +import org.apache.beam.sdk.metrics.Metrics; 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; import org.apache.beam.sdk.transforms.join.CoGbkResult; import org.apache.beam.sdk.transforms.join.CoGroupByKey; import org.apache.beam.sdk.transforms.join.KeyedPCollectionTuple; @@ -323,56 +323,52 @@ public PCollection expand(PCollection events) { // Find the highest price valid bid for each closed auction. return - // Join auctions and bids. - KeyedPCollectionTuple.of(NexmarkQuery.AUCTION_TAG, auctionsById) - .and(NexmarkQuery.BID_TAG, bidsByAuctionId) - .apply(CoGroupByKey.create()) - - // Filter and select. - .apply(name + ".Join", - ParDo.of(new DoFn, AuctionBid>() { - final Aggregator noAuctionCounter = - createAggregator("noAuction", Sum.ofLongs()); - final Aggregator underReserveCounter = - createAggregator("underReserve", Sum.ofLongs()); - final Aggregator noValidBidsCounter = - createAggregator("noValidBids", Sum.ofLongs()); - - - @ProcessElement - public void processElement(ProcessContext c) { - Auction auction = - c.element().getValue().getOnly(NexmarkQuery.AUCTION_TAG, null); - if (auction == null) { - // We have bids without a matching auction. Give up. - noAuctionCounter.addValue(1L); - return; - } - // Find the current winning bid for auction. - // The earliest bid with the maximum price above the reserve wins. - Bid bestBid = null; - for (Bid bid : c.element().getValue().getAll(NexmarkQuery.BID_TAG)) { - // Bids too late for their auction will have been - // filtered out by the window merge function. - checkState(bid.dateTime < auction.expires); - if (bid.price < auction.reserve) { - // Bid price is below auction reserve. - underReserveCounter.addValue(1L); - continue; - } - - if (bestBid == null - || Bid.PRICE_THEN_DESCENDING_TIME.compare(bid, bestBid) > 0) { - bestBid = bid; - } - } - if (bestBid == null) { - // We don't have any valid bids for auction. - noValidBidsCounter.addValue(1L); - return; - } - c.output(new AuctionBid(auction, bestBid)); - } - })); + // Join auctions and bids. + KeyedPCollectionTuple.of(NexmarkQuery.AUCTION_TAG, auctionsById) + .and(NexmarkQuery.BID_TAG, bidsByAuctionId) + .apply(CoGroupByKey.create()) + // Filter and select. + .apply(name + ".Join", + ParDo.of(new DoFn, AuctionBid>() { + private final Counter noAuctionCounter = Metrics.counter(name, "noAuction"); + private final Counter underReserveCounter = Metrics.counter(name, "underReserve"); + private final Counter noValidBidsCounter = Metrics.counter(name, "noValidBids"); + + @ProcessElement + public void processElement(ProcessContext c) { + Auction auction = + c.element().getValue().getOnly(NexmarkQuery.AUCTION_TAG, null); + if (auction == null) { + // We have bids without a matching auction. Give up. + noAuctionCounter.inc(); + return; + } + // Find the current winning bid for auction. + // The earliest bid with the maximum price above the reserve wins. + Bid bestBid = null; + for (Bid bid : c.element().getValue().getAll(NexmarkQuery.BID_TAG)) { + // Bids too late for their auction will have been + // filtered out by the window merge function. + checkState(bid.dateTime < auction.expires); + if (bid.price < auction.reserve) { + // Bid price is below auction reserve. + underReserveCounter.inc(); + continue; + } + + if (bestBid == null + || Bid.PRICE_THEN_DESCENDING_TIME.compare(bid, bestBid) > 0) { + bestBid = bid; + } + } + if (bestBid == null) { + // We don't have any valid bids for auction. + noValidBidsCounter.inc(); + return; + } + c.output(new AuctionBid(auction, bestBid)); + } + } + )); } } diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/drivers/NexmarkGoogleRunner.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/drivers/NexmarkGoogleRunner.java index 7ffd47aa7578f..935bf0d2735cb 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/drivers/NexmarkGoogleRunner.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/drivers/NexmarkGoogleRunner.java @@ -130,7 +130,9 @@ protected void waitForPublisherPreload() { NexmarkUtils.console("%s publisher (%d events)", state, numEvents); return; case RUNNING: - numEvents = getLong(job, publisherMonitor.getElementCounter()); + //TODO Ismael Validate that this counter is ok + numEvents = + getCounterMetric(job, publisherMonitor.name, publisherMonitor.prefix + ".elements", -1); if (startMsSinceEpoch < 0 && numEvents > 0) { startMsSinceEpoch = System.currentTimeMillis(); endMsSinceEpoch = startMsSinceEpoch diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query0.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query0.java index f60d5dee5a3d9..84696c49f9777 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query0.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query0.java @@ -28,10 +28,10 @@ import org.apache.beam.integration.nexmark.model.KnownSize; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.CoderException; -import org.apache.beam.sdk.transforms.Aggregator; +import org.apache.beam.sdk.metrics.Counter; +import org.apache.beam.sdk.metrics.Metrics; 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.PCollection; /** @@ -49,15 +49,15 @@ private PCollection applyTyped(PCollection events) { // Force round trip through coder. .apply(name + ".Serialize", ParDo.of(new DoFn() { - private final Aggregator bytes = - createAggregator("bytes", Sum.ofLongs()); + private final Counter bytesMetric = + Metrics.counter(name , "bytes"); @ProcessElement public void processElement(ProcessContext c) throws CoderException, IOException { ByteArrayOutputStream outStream = new ByteArrayOutputStream(); coder.encode(c.element(), outStream, Coder.Context.OUTER); byte[] byteArray = outStream.toByteArray(); - bytes.addValue((long) byteArray.length); + bytesMetric.inc((long) byteArray.length); ByteArrayInputStream inStream = new ByteArrayInputStream(byteArray); Event event = coder.decode(inStream, Coder.Context.OUTER); c.output(event); diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query10.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query10.java index 5246427c5643e..d9b3557af6359 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query10.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query10.java @@ -34,12 +34,12 @@ import org.apache.beam.integration.nexmark.model.Event; import org.apache.beam.integration.nexmark.model.KnownSize; import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.metrics.Counter; +import org.apache.beam.sdk.metrics.Metrics; import org.apache.beam.sdk.options.GcsOptions; -import org.apache.beam.sdk.transforms.Aggregator; 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.Sum; import org.apache.beam.sdk.transforms.windowing.AfterEach; import org.apache.beam.sdk.transforms.windowing.AfterFirst; import org.apache.beam.sdk.transforms.windowing.AfterPane; @@ -184,196 +184,189 @@ private String indexPathFor(BoundedWindow window) { private PCollection applyTyped(PCollection events) { final int numLogShards = maxNumWorkers * NUM_SHARDS_PER_WORKER; - return events.apply(name + ".ShardEvents", - ParDo.of(new DoFn>() { - final Aggregator lateCounter = - createAggregator("actuallyLateEvent", Sum.ofLongs()); - final Aggregator onTimeCounter = - createAggregator("actuallyOnTimeEvent", Sum.ofLongs()); + return events + .apply(name + ".ShardEvents", + ParDo.of(new DoFn>() { + private final Counter lateCounter = Metrics.counter(name , "actuallyLateEvent"); + private final Counter onTimeCounter = Metrics.counter(name , "onTimeCounter"); - @ProcessElement - public void processElement(ProcessContext c) { - if (c.element().hasAnnotation("LATE")) { - lateCounter.addValue(1L); - LOG.error("Observed late: %s", c.element()); - } else { - onTimeCounter.addValue(1L); - } - int shardNum = (int) Math.abs((long) c.element().hashCode() % numLogShards); - String shard = String.format("shard-%05d-of-%05d", shardNum, numLogShards); - c.output(KV.of(shard, c.element())); - } - })) - .apply(name + ".WindowEvents", - Window.>into( - FixedWindows.of(Duration.standardSeconds(configuration.windowSizeSec))) - .triggering(AfterEach.inOrder( - Repeatedly - .forever(AfterPane.elementCountAtLeast(configuration.maxLogEvents)) - .orFinally(AfterWatermark.pastEndOfWindow()), - Repeatedly.forever( - AfterFirst.of(AfterPane.elementCountAtLeast(configuration.maxLogEvents), - AfterProcessingTime.pastFirstElementInPane() - .plusDelayOf(LATE_BATCHING_PERIOD))))) - .discardingFiredPanes() - // Use a 1 day allowed lateness so that any forgotten hold will stall the - // pipeline for that period and be very noticeable. - .withAllowedLateness(Duration.standardDays(1))) - .apply(name + ".GroupByKey", GroupByKey.create()) - .apply(name + ".CheckForLateEvents", - ParDo.of(new DoFn>, - KV>>() { - final Aggregator earlyCounter = - createAggregator("earlyShard", Sum.ofLongs()); - final Aggregator onTimeCounter = - createAggregator("onTimeShard", Sum.ofLongs()); - final Aggregator lateCounter = - createAggregator("lateShard", Sum.ofLongs()); - final Aggregator unexpectedLatePaneCounter = - createAggregator("ERROR_unexpectedLatePane", Sum.ofLongs()); - final Aggregator unexpectedOnTimeElementCounter = - createAggregator("ERROR_unexpectedOnTimeElement", Sum.ofLongs()); + @ProcessElement + public void processElement(ProcessContext c) { + if (c.element().hasAnnotation("LATE")) { + lateCounter.inc(); + LOG.error("Observed late: %s", c.element()); + } else { + onTimeCounter.inc(); + } + int shardNum = (int) Math.abs((long) c.element().hashCode() % numLogShards); + String shard = String.format("shard-%05d-of-%05d", shardNum, numLogShards); + c.output(KV.of(shard, c.element())); + } + })) + .apply(name + ".WindowEvents", + Window.>into( + FixedWindows.of(Duration.standardSeconds(configuration.windowSizeSec))) + .triggering(AfterEach.inOrder( + Repeatedly + .forever(AfterPane.elementCountAtLeast(configuration.maxLogEvents)) + .orFinally(AfterWatermark.pastEndOfWindow()), + Repeatedly.forever( + AfterFirst.of(AfterPane.elementCountAtLeast(configuration.maxLogEvents), + AfterProcessingTime.pastFirstElementInPane() + .plusDelayOf(LATE_BATCHING_PERIOD))))) + .discardingFiredPanes() + // Use a 1 day allowed lateness so that any forgotten hold will stall the + // pipeline for that period and be very noticeable. + .withAllowedLateness(Duration.standardDays(1))) + .apply(name + ".GroupByKey", GroupByKey.create()) + .apply(name + ".CheckForLateEvents", + ParDo.of(new DoFn>, + KV>>() { + private final Counter earlyCounter = Metrics.counter(name , "earlyShard"); + private final Counter onTimeCounter = Metrics.counter(name , "onTimeShard"); + private final Counter lateCounter = Metrics.counter(name , "lateShard"); + private final Counter unexpectedLatePaneCounter = + Metrics.counter(name , "ERROR_unexpectedLatePane"); + private final Counter unexpectedOnTimeElementCounter = + Metrics.counter(name , "ERROR_unexpectedOnTimeElement"); - @ProcessElement - public void processElement(ProcessContext c, BoundedWindow window) { - int numLate = 0; - int numOnTime = 0; - for (Event event : c.element().getValue()) { - if (event.hasAnnotation("LATE")) { - numLate++; - } else { - numOnTime++; - } - } - String shard = c.element().getKey(); - LOG.error( - "%s with timestamp %s has %d actually late and %d on-time " - + "elements in pane %s for window %s", - shard, c.timestamp(), numLate, numOnTime, c.pane(), - window.maxTimestamp()); - if (c.pane().getTiming() == PaneInfo.Timing.LATE) { - if (numLate == 0) { - LOG.error( - "ERROR! No late events in late pane for %s", shard); - unexpectedLatePaneCounter.addValue(1L); - } - if (numOnTime > 0) { - LOG.error( - "ERROR! Have %d on-time events in late pane for %s", - numOnTime, shard); - unexpectedOnTimeElementCounter.addValue(1L); - } - lateCounter.addValue(1L); - } else if (c.pane().getTiming() == PaneInfo.Timing.EARLY) { - if (numOnTime + numLate < configuration.maxLogEvents) { - LOG.error( - "ERROR! Only have %d events in early pane for %s", - numOnTime + numLate, shard); - } - earlyCounter.addValue(1L); - } else { - onTimeCounter.addValue(1L); - } - c.output(c.element()); - } - })) - .apply(name + ".UploadEvents", - ParDo.of(new DoFn>, - KV>() { - final Aggregator savedFileCounter = - createAggregator("savedFile", Sum.ofLongs()); - final Aggregator writtenRecordsCounter = - createAggregator("writtenRecords", Sum.ofLongs()); + @ProcessElement + public void processElement(ProcessContext c, BoundedWindow window) { + int numLate = 0; + int numOnTime = 0; + for (Event event : c.element().getValue()) { + if (event.hasAnnotation("LATE")) { + numLate++; + } else { + numOnTime++; + } + } + String shard = c.element().getKey(); + LOG.error( + "%s with timestamp %s has %d actually late and %d on-time " + + "elements in pane %s for window %s", + shard, c.timestamp(), numLate, numOnTime, c.pane(), + window.maxTimestamp()); + if (c.pane().getTiming() == PaneInfo.Timing.LATE) { + if (numLate == 0) { + LOG.error( + "ERROR! No late events in late pane for %s", shard); + unexpectedLatePaneCounter.inc(); + } + if (numOnTime > 0) { + LOG.error( + "ERROR! Have %d on-time events in late pane for %s", + numOnTime, shard); + unexpectedOnTimeElementCounter.inc(); + } + lateCounter.inc(); + } else if (c.pane().getTiming() == PaneInfo.Timing.EARLY) { + if (numOnTime + numLate < configuration.maxLogEvents) { + LOG.error( + "ERROR! Only have %d events in early pane for %s", + numOnTime + numLate, shard); + } + earlyCounter.inc(); + } else { + onTimeCounter.inc(); + } + c.output(c.element()); + } + })) + .apply(name + ".UploadEvents", + ParDo.of(new DoFn>, + KV>() { + private final Counter savedFileCounter = Metrics.counter(name , "savedFile"); + private final Counter writtenRecordsCounter = Metrics.counter(name , "writtenRecords"); - @ProcessElement - public void processElement(ProcessContext c, BoundedWindow window) - throws IOException { - String shard = c.element().getKey(); - GcsOptions options = c.getPipelineOptions().as(GcsOptions.class); - OutputFile outputFile = outputFileFor(window, shard, c.pane()); - LOG.error( - "Writing %s with record timestamp %s, window timestamp %s, pane %s", - shard, c.timestamp(), window.maxTimestamp(), c.pane()); - if (outputFile.filename != null) { - LOG.error("Beginning write to '%s'", outputFile.filename); - int n = 0; - try (OutputStream output = - Channels.newOutputStream(openWritableGcsFile(options, outputFile - .filename))) { - for (Event event : c.element().getValue()) { - Event.CODER.encode(event, output, Coder.Context.OUTER); - writtenRecordsCounter.addValue(1L); - if (++n % 10000 == 0) { - LOG.error("So far written %d records to '%s'", n, - outputFile.filename); - } - } - } - LOG.error("Written all %d records to '%s'", n, outputFile.filename); - } - savedFileCounter.addValue(1L); - c.output(KV.of(null, outputFile)); - } - })) - // Clear fancy triggering from above. - .apply(name + ".WindowLogFiles", Window.>into( - FixedWindows.of(Duration.standardSeconds(configuration.windowSizeSec))) - .triggering(AfterWatermark.pastEndOfWindow()) - // We expect no late data here, but we'll assume the worst so we can detect any. - .withAllowedLateness(Duration.standardDays(1)) - .discardingFiredPanes()) + @ProcessElement + public void processElement(ProcessContext c, BoundedWindow window) + throws IOException { + String shard = c.element().getKey(); + GcsOptions options = c.getPipelineOptions().as(GcsOptions.class); + OutputFile outputFile = outputFileFor(window, shard, c.pane()); + LOG.error( + "Writing %s with record timestamp %s, window timestamp %s, pane %s", + shard, c.timestamp(), window.maxTimestamp(), c.pane()); + if (outputFile.filename != null) { + LOG.error("Beginning write to '%s'", outputFile.filename); + int n = 0; + try (OutputStream output = + Channels.newOutputStream(openWritableGcsFile(options, outputFile + .filename))) { + for (Event event : c.element().getValue()) { + Event.CODER.encode(event, output, Coder.Context.OUTER); + writtenRecordsCounter.inc(); + if (++n % 10000 == 0) { + LOG.error("So far written %d records to '%s'", n, + outputFile.filename); + } + } + } + LOG.error("Written all %d records to '%s'", n, outputFile.filename); + } + savedFileCounter.inc(); + c.output(KV.of(null, outputFile)); + } + })) + // Clear fancy triggering from above. + .apply(name + ".WindowLogFiles", Window.>into( + FixedWindows.of(Duration.standardSeconds(configuration.windowSizeSec))) + .triggering(AfterWatermark.pastEndOfWindow()) + // We expect no late data here, but we'll assume the worst so we can detect any. + .withAllowedLateness(Duration.standardDays(1)) + .discardingFiredPanes()) // this GroupByKey allows to have one file per window .apply(name + ".GroupByKey2", GroupByKey.create()) - .apply(name + ".Index", - ParDo.of(new DoFn>, Done>() { - final Aggregator unexpectedLateCounter = - createAggregator("ERROR_unexpectedLate", Sum.ofLongs()); - final Aggregator unexpectedEarlyCounter = - createAggregator("ERROR_unexpectedEarly", Sum.ofLongs()); - final Aggregator unexpectedIndexCounter = - createAggregator("ERROR_unexpectedIndex", Sum.ofLongs()); - final Aggregator finalizedCounter = - createAggregator("indexed", Sum.ofLongs()); + .apply(name + ".Index", + ParDo.of(new DoFn>, Done>() { + private final Counter unexpectedLateCounter = + Metrics.counter(name , "ERROR_unexpectedLate"); + private final Counter unexpectedEarlyCounter = + Metrics.counter(name , "ERROR_unexpectedEarly"); + private final Counter unexpectedIndexCounter = + Metrics.counter(name , "ERROR_unexpectedIndex"); + private final Counter finalizedCounter = Metrics.counter(name , "indexed"); - @ProcessElement - public void processElement(ProcessContext c, BoundedWindow window) - throws IOException { - if (c.pane().getTiming() == Timing.LATE) { - unexpectedLateCounter.addValue(1L); - LOG.error("ERROR! Unexpected LATE pane: %s", c.pane()); - } else if (c.pane().getTiming() == Timing.EARLY) { - unexpectedEarlyCounter.addValue(1L); - LOG.error("ERROR! Unexpected EARLY pane: %s", c.pane()); - } else if (c.pane().getTiming() == Timing.ON_TIME - && c.pane().getIndex() != 0) { - unexpectedIndexCounter.addValue(1L); - LOG.error("ERROR! Unexpected ON_TIME pane index: %s", c.pane()); - } else { - GcsOptions options = c.getPipelineOptions().as(GcsOptions.class); - LOG.error( - "Index with record timestamp %s, window timestamp %s, pane %s", - c.timestamp(), window.maxTimestamp(), c.pane()); + @ProcessElement + public void processElement(ProcessContext c, BoundedWindow window) + throws IOException { + if (c.pane().getTiming() == Timing.LATE) { + unexpectedLateCounter.inc(); + LOG.error("ERROR! Unexpected LATE pane: %s", c.pane()); + } else if (c.pane().getTiming() == Timing.EARLY) { + unexpectedEarlyCounter.inc(); + LOG.error("ERROR! Unexpected EARLY pane: %s", c.pane()); + } else if (c.pane().getTiming() == Timing.ON_TIME + && c.pane().getIndex() != 0) { + unexpectedIndexCounter.inc(); + LOG.error("ERROR! Unexpected ON_TIME pane index: %s", c.pane()); + } else { + GcsOptions options = c.getPipelineOptions().as(GcsOptions.class); + LOG.error( + "Index with record timestamp %s, window timestamp %s, pane %s", + c.timestamp(), window.maxTimestamp(), c.pane()); - @Nullable String filename = indexPathFor(window); - if (filename != null) { - LOG.error("Beginning write to '%s'", filename); - int n = 0; - try (OutputStream output = - Channels.newOutputStream( - openWritableGcsFile(options, filename))) { - for (OutputFile outputFile : c.element().getValue()) { - output.write(outputFile.toString().getBytes()); - n++; - } - } - LOG.error("Written all %d lines to '%s'", n, filename); - } - c.output( - new Done("written for timestamp " + window.maxTimestamp())); - finalizedCounter.addValue(1L); - } - } - })); + @Nullable String filename = indexPathFor(window); + if (filename != null) { + LOG.error("Beginning write to '%s'", filename); + int n = 0; + try (OutputStream output = + Channels.newOutputStream( + openWritableGcsFile(options, filename))) { + for (OutputFile outputFile : c.element().getValue()) { + output.write(outputFile.toString().getBytes()); + n++; + } + } + LOG.error("Written all %d lines to '%s'", n, filename); + } + c.output( + new Done("written for timestamp " + window.maxTimestamp())); + finalizedCounter.inc(); + } + } + })); } @Override diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query3.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query3.java index ba31e9ff5aba3..12b16f1ee8c56 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query3.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query3.java @@ -20,7 +20,6 @@ import java.io.IOException; import java.util.ArrayList; import java.util.List; -import javax.annotation.Nullable; import org.apache.beam.integration.nexmark.NexmarkConfiguration; import org.apache.beam.integration.nexmark.NexmarkQuery; import org.apache.beam.integration.nexmark.NexmarkUtils; @@ -30,12 +29,12 @@ import org.apache.beam.integration.nexmark.model.NameCityStateId; import org.apache.beam.integration.nexmark.model.Person; import org.apache.beam.sdk.coders.ListCoder; -import org.apache.beam.sdk.transforms.Aggregator; +import org.apache.beam.sdk.metrics.Counter; +import org.apache.beam.sdk.metrics.Metrics; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.Filter; import org.apache.beam.sdk.transforms.ParDo; import org.apache.beam.sdk.transforms.SerializableFunction; -import org.apache.beam.sdk.transforms.Sum; import org.apache.beam.sdk.transforms.join.CoGbkResult; import org.apache.beam.sdk.transforms.join.CoGroupByKey; import org.apache.beam.sdk.transforms.join.KeyedPCollectionTuple; @@ -81,14 +80,7 @@ public class Query3 extends NexmarkQuery { public Query3(NexmarkConfiguration configuration) { super(configuration, "Query3"); - joinDoFn = new JoinDoFn(configuration.maxAuctionsWaitingTime); - - } - - @Override - @Nullable - public Aggregator getFatalCount() { - return joinDoFn.fatalCounter; + joinDoFn = new JoinDoFn(name, configuration.maxAuctionsWaitingTime); } private PCollection applyTyped(PCollection events) { @@ -195,8 +187,6 @@ private static class JoinDoFn extends DoFn, KV fatalCounter = createAggregator("fatal", Sum.ofLongs()); - @StateId(AUCTIONS) private final StateSpec>> auctionsSpec = StateSpecs.value(ListCoder.of(Auction.CODER)); @@ -204,19 +194,25 @@ private static class JoinDoFn extends DoFn, KV newAuctionCounter = - createAggregator("newAuction", Sum.ofLongs()); - private final Aggregator newPersonCounter = - createAggregator("newPerson", Sum.ofLongs()); - private final Aggregator newNewOutputCounter = - createAggregator("newNewOutput", Sum.ofLongs()); - private final Aggregator newOldOutputCounter = - createAggregator("newOldOutput", Sum.ofLongs()); - private final Aggregator oldNewOutputCounter = - createAggregator("oldNewOutput", Sum.ofLongs()); + // Used to refer the metrics namespace + private final String name; - private JoinDoFn(int maxAuctionsWaitingTime) { + private final Counter newAuctionCounter; + private final Counter newPersonCounter; + private final Counter newNewOutputCounter; + private final Counter newOldOutputCounter; + private final Counter oldNewOutputCounter; + private final Counter fatalCounter; + + private JoinDoFn(String name, int maxAuctionsWaitingTime) { + this.name = name; this.maxAuctionsWaitingTime = maxAuctionsWaitingTime; + newAuctionCounter = Metrics.counter(name, "newAuction"); + newPersonCounter = Metrics.counter(name, "newPerson"); + newNewOutputCounter = Metrics.counter(name, "newNewOutput"); + newOldOutputCounter = Metrics.counter(name, "newOldOutput"); + oldNewOutputCounter = Metrics.counter(name, "oldNewOutput"); + fatalCounter = Metrics.counter(name , "fatal"); } @ProcessElement @@ -232,14 +228,13 @@ public void processElement( // we need to wait for the pending ReduceFn API. Person existingPerson = personState.read(); - if (existingPerson != null) { // We've already seen the new person event for this person id. // We can join with any new auctions on-the-fly without needing any // additional persistent state. for (Auction newAuction : c.element().getValue().getAll(AUCTION_TAG)) { - newAuctionCounter.addValue(1L); - newOldOutputCounter.addValue(1L); + newAuctionCounter.inc(); + newOldOutputCounter.inc(); c.output(KV.of(newAuction, existingPerson)); } return; @@ -255,24 +250,24 @@ public void processElement( } else { LOG.error("**** conflicting persons {} and {} ****", theNewPerson, newPerson); } - fatalCounter.addValue(1L); + fatalCounter.inc(); continue; } - newPersonCounter.addValue(1L); + newPersonCounter.inc(); // We've now seen the person for this person id so can flush any // pending auctions for the same seller id (an auction is done by only one seller). List pendingAuctions = auctionsState.read(); if (pendingAuctions != null) { for (Auction pendingAuction : pendingAuctions) { - oldNewOutputCounter.addValue(1L); + oldNewOutputCounter.inc(); c.output(KV.of(pendingAuction, newPerson)); } auctionsState.clear(); } // Also deal with any new auctions. for (Auction newAuction : c.element().getValue().getAll(AUCTION_TAG)) { - newAuctionCounter.addValue(1L); - newNewOutputCounter.addValue(1L); + newAuctionCounter.inc(); + newNewOutputCounter.inc(); c.output(KV.of(newAuction, newPerson)); } // Remember this person for any future auctions. @@ -293,17 +288,17 @@ public void processElement( pendingAuctions = new ArrayList<>(); } for (Auction newAuction : c.element().getValue().getAll(AUCTION_TAG)) { - newAuctionCounter.addValue(1L); + newAuctionCounter.inc(); pendingAuctions.add(newAuction); } auctionsState.write(pendingAuctions); } - @OnTimer(PERSON_STATE_EXPIRING) - public void onTimerCallback( - OnTimerContext context, - @StateId(PERSON) ValueState personState) { - personState.clear(); - } + @OnTimer(PERSON_STATE_EXPIRING) + public void onTimerCallback( + OnTimerContext context, + @StateId(PERSON) ValueState personState) { + personState.clear(); + } } } diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/sources/BoundedEventSource.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/sources/BoundedEventSource.java index be741516d79c9..43d6690848a9f 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/sources/BoundedEventSource.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/sources/BoundedEventSource.java @@ -156,7 +156,7 @@ public synchronized BoundedEventSource splitAtFraction(double fraction) { } @Override - public List splitIntoBundles( + public List split( long desiredBundleSizeBytes, PipelineOptions options) { NexmarkUtils.info("slitting bounded source %s into %d sub-sources", config, numEventGenerators); List results = new ArrayList<>(); diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/sources/UnboundedEventSource.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/sources/UnboundedEventSource.java index 286c5768f2ea5..c3c6eb031e875 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/sources/UnboundedEventSource.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/sources/UnboundedEventSource.java @@ -289,7 +289,7 @@ public Coder getCheckpointMarkCoder() { } @Override - public List generateInitialSplits( + public List split( int desiredNumSplits, PipelineOptions options) { LOG.trace("splitting unbounded source into {} sub-sources", numEventGenerators); List results = new ArrayList<>(); diff --git a/integration/java/nexmark/src/test/java/org/apache/beam/integration/nexmark/sources/BoundedEventSourceTest.java b/integration/java/nexmark/src/test/java/org/apache/beam/integration/nexmark/sources/BoundedEventSourceTest.java index 3f85bab14f07a..c5d77253f6ba8 100644 --- a/integration/java/nexmark/src/test/java/org/apache/beam/integration/nexmark/sources/BoundedEventSourceTest.java +++ b/integration/java/nexmark/src/test/java/org/apache/beam/integration/nexmark/sources/BoundedEventSourceTest.java @@ -66,6 +66,6 @@ public void splitIntoBundlesRespectsContract() throws Exception { long n = 200L; BoundedEventSource source = new BoundedEventSource(makeConfig(n), 1); SourceTestUtils.assertSourcesEqualReferenceSource( - source, source.splitIntoBundles(10, options), options); + source, source.split(10, options), options); } } From 8b96949b934be1df7102aeb24ef4b23d9dd28812 Mon Sep 17 00:00:00 2001 From: Etienne Chauchot Date: Fri, 28 Apr 2017 10:29:38 +0200 Subject: [PATCH 302/346] Fix compile after Coders and Pubsub refactor --- .../integration/nexmark/NexmarkOptions.java | 3 +-- .../beam/integration/nexmark/WinningBids.java | 23 +++++++++++-------- .../integration/nexmark/io/PubsubClient.java | 2 +- .../integration/nexmark/io/PubsubHelper.java | 2 +- .../nexmark/io/PubsubJsonClient.java | 2 +- .../nexmark/io/PubsubTestClient.java | 2 +- .../integration/nexmark/model/Auction.java | 6 ++--- .../integration/nexmark/model/AuctionBid.java | 6 ++--- .../nexmark/model/AuctionCount.java | 6 ++--- .../nexmark/model/AuctionPrice.java | 6 ++--- .../beam/integration/nexmark/model/Bid.java | 8 +++---- .../nexmark/model/BidsPerSession.java | 7 +++--- .../nexmark/model/CategoryPrice.java | 7 +++--- .../beam/integration/nexmark/model/Done.java | 7 +++--- .../beam/integration/nexmark/model/Event.java | 6 ++--- .../nexmark/model/IdNameReserve.java | 7 +++--- .../nexmark/model/NameCityStateId.java | 7 +++--- .../integration/nexmark/model/Person.java | 7 +++--- .../nexmark/model/SellerPrice.java | 7 +++--- .../nexmark/sources/Generator.java | 6 ++--- 20 files changed, 57 insertions(+), 70 deletions(-) diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkOptions.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkOptions.java index 5d093ae64bc43..e1c1af2d9a46c 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkOptions.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkOptions.java @@ -18,10 +18,9 @@ package org.apache.beam.integration.nexmark; import javax.annotation.Nullable; - +import org.apache.beam.sdk.io.gcp.pubsub.PubsubOptions; import org.apache.beam.sdk.options.Default; import org.apache.beam.sdk.options.Description; -import org.apache.beam.sdk.options.PubsubOptions; /** * Command line flags. diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/WinningBids.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/WinningBids.java index f2566b805cc9a..3815b9d39cfc1 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/WinningBids.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/WinningBids.java @@ -35,9 +35,9 @@ import org.apache.beam.integration.nexmark.model.Bid; import org.apache.beam.integration.nexmark.model.Event; import org.apache.beam.integration.nexmark.sources.GeneratorConfig; -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.CustomCoder; import org.apache.beam.sdk.coders.VarIntCoder; import org.apache.beam.sdk.coders.VarLongCoder; import org.apache.beam.sdk.metrics.Counter; @@ -145,7 +145,7 @@ public String toString() { /** * Encodes an {@link AuctionOrBidWindow} as an {@link IntervalWindow} and an auction id long. */ - private static class AuctionOrBidWindowCoder extends AtomicCoder { + private static class AuctionOrBidWindowCoder extends CustomCoder { private static final AuctionOrBidWindowCoder INSTANCE = new AuctionOrBidWindowCoder(); private static final Coder SUPER_CODER = IntervalWindow.getCoder(); private static final Coder ID_CODER = VarLongCoder.of(); @@ -157,22 +157,25 @@ public static AuctionOrBidWindowCoder of() { } @Override - public void encode(AuctionOrBidWindow window, OutputStream outStream, Context context) + public void encode(AuctionOrBidWindow window, OutputStream outStream, Coder.Context context) throws IOException, CoderException { - SUPER_CODER.encode(window, outStream, Context.NESTED); - ID_CODER.encode(window.auction, outStream, Context.NESTED); - INT_CODER.encode(window.isAuctionWindow ? 1 : 0, outStream, Context.NESTED); + SUPER_CODER.encode(window, outStream, Coder.Context.NESTED); + ID_CODER.encode(window.auction, outStream, Coder.Context.NESTED); + INT_CODER.encode(window.isAuctionWindow ? 1 : 0, outStream, Coder.Context.NESTED); } @Override - public AuctionOrBidWindow decode(InputStream inStream, Context context) + public AuctionOrBidWindow decode(InputStream inStream, Coder.Context context) throws IOException, CoderException { - IntervalWindow superWindow = SUPER_CODER.decode(inStream, Context.NESTED); - long auction = ID_CODER.decode(inStream, Context.NESTED); - boolean isAuctionWindow = INT_CODER.decode(inStream, Context.NESTED) == 0 ? false : true; + IntervalWindow superWindow = SUPER_CODER.decode(inStream, Coder.Context.NESTED); + long auction = ID_CODER.decode(inStream, Coder.Context.NESTED); + boolean isAuctionWindow = + INT_CODER.decode(inStream, Coder.Context.NESTED) == 0 ? false : true; return new AuctionOrBidWindow( superWindow.start(), superWindow.end(), auction, isAuctionWindow); } + + @Override public void verifyDeterministic() throws NonDeterministicException {} } /** Assign events to auction windows and merges them intelligently. */ diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/io/PubsubClient.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/io/PubsubClient.java index 687aa35d9cc6c..931fe6e69da4f 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/io/PubsubClient.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/io/PubsubClient.java @@ -32,7 +32,7 @@ import java.util.Map; import java.util.concurrent.ThreadLocalRandom; import javax.annotation.Nullable; -import org.apache.beam.sdk.options.PubsubOptions; +import org.apache.beam.sdk.io.gcp.pubsub.PubsubOptions; /** * An (abstract) helper class for talking to Pubsub via an underlying transport. diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/io/PubsubHelper.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/io/PubsubHelper.java index 15401b72a54b0..bcc5b1ce1e034 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/io/PubsubHelper.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/io/PubsubHelper.java @@ -23,7 +23,7 @@ import java.util.List; import org.apache.beam.integration.nexmark.NexmarkUtils; -import org.apache.beam.sdk.options.PubsubOptions; +import org.apache.beam.sdk.io.gcp.pubsub.PubsubOptions; /** * Helper for working with pubsub. diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/io/PubsubJsonClient.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/io/PubsubJsonClient.java index b778a094022e7..afddbd80fe124 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/io/PubsubJsonClient.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/io/PubsubJsonClient.java @@ -47,7 +47,7 @@ import java.util.Map; import java.util.TreeMap; import javax.annotation.Nullable; -import org.apache.beam.sdk.options.PubsubOptions; +import org.apache.beam.sdk.io.gcp.pubsub.PubsubOptions; import org.apache.beam.sdk.util.RetryHttpRequestInitializer; import org.apache.beam.sdk.util.Transport; diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/io/PubsubTestClient.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/io/PubsubTestClient.java index 125a8d69c4ae7..69ba2b0cbcac3 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/io/PubsubTestClient.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/io/PubsubTestClient.java @@ -33,7 +33,7 @@ import java.util.Map; import java.util.Set; import javax.annotation.Nullable; -import org.apache.beam.sdk.options.PubsubOptions; +import org.apache.beam.sdk.io.gcp.pubsub.PubsubOptions; /** * A (partial) implementation of {@link PubsubClient} for use by unit tests. Only suitable for diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/Auction.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/Auction.java index ac3056887d41a..4b1a8480cd952 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/Auction.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/Auction.java @@ -19,16 +19,14 @@ import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.core.JsonProcessingException; - import java.io.IOException; import java.io.InputStream; import java.io.OutputStream; import java.io.Serializable; - import org.apache.beam.integration.nexmark.NexmarkUtils; -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.CustomCoder; import org.apache.beam.sdk.coders.StringUtf8Coder; import org.apache.beam.sdk.coders.VarLongCoder; @@ -39,7 +37,7 @@ public class Auction implements KnownSize, Serializable { private static final Coder LONG_CODER = VarLongCoder.of(); private static final Coder STRING_CODER = StringUtf8Coder.of(); - public static final Coder CODER = new AtomicCoder() { + public static final Coder CODER = new CustomCoder() { @Override public void encode(Auction value, OutputStream outStream, Coder.Context context) diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/AuctionBid.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/AuctionBid.java index c014257fc4194..7f6b7c9272178 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/AuctionBid.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/AuctionBid.java @@ -19,23 +19,21 @@ import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.core.JsonProcessingException; - import java.io.IOException; import java.io.InputStream; import java.io.OutputStream; import java.io.Serializable; - import org.apache.beam.integration.nexmark.NexmarkUtils; import org.apache.beam.integration.nexmark.WinningBids; -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.CustomCoder; /** * Result of {@link WinningBids} transform. */ public class AuctionBid implements KnownSize, Serializable { - public static final Coder CODER = new AtomicCoder() { + public static final Coder CODER = new CustomCoder() { @Override public void encode(AuctionBid value, OutputStream outStream, Coder.Context context) diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/AuctionCount.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/AuctionCount.java index aa166297be8bc..e6d34504e1972 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/AuctionCount.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/AuctionCount.java @@ -19,16 +19,14 @@ import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.core.JsonProcessingException; - import java.io.IOException; import java.io.InputStream; import java.io.OutputStream; import java.io.Serializable; - import org.apache.beam.integration.nexmark.NexmarkUtils; -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.CustomCoder; import org.apache.beam.sdk.coders.VarLongCoder; /** @@ -37,7 +35,7 @@ public class AuctionCount implements KnownSize, Serializable { private static final Coder LONG_CODER = VarLongCoder.of(); - public static final Coder CODER = new AtomicCoder() { + public static final Coder CODER = new CustomCoder() { @Override public void encode(AuctionCount value, OutputStream outStream, Coder.Context context) diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/AuctionPrice.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/AuctionPrice.java index f365cc84964ab..cb971e29e1234 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/AuctionPrice.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/AuctionPrice.java @@ -19,16 +19,14 @@ import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.core.JsonProcessingException; - import java.io.IOException; import java.io.InputStream; import java.io.OutputStream; import java.io.Serializable; - import org.apache.beam.integration.nexmark.NexmarkUtils; -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.CustomCoder; import org.apache.beam.sdk.coders.VarLongCoder; /** @@ -37,7 +35,7 @@ public class AuctionPrice implements KnownSize, Serializable { private static final Coder LONG_CODER = VarLongCoder.of(); - public static final Coder CODER = new AtomicCoder() { + public static final Coder CODER = new CustomCoder() { @Override public void encode(AuctionPrice value, OutputStream outStream, Coder.Context context) diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/Bid.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/Bid.java index 59a33c1cf10ac..faeb928307cc0 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/Bid.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/Bid.java @@ -19,17 +19,15 @@ import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.core.JsonProcessingException; - import java.io.IOException; import java.io.InputStream; import java.io.OutputStream; import java.io.Serializable; import java.util.Comparator; - import org.apache.beam.integration.nexmark.NexmarkUtils; -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.CustomCoder; import org.apache.beam.sdk.coders.StringUtf8Coder; import org.apache.beam.sdk.coders.VarLongCoder; @@ -40,7 +38,7 @@ public class Bid implements KnownSize, Serializable { private static final Coder LONG_CODER = VarLongCoder.of(); private static final Coder STRING_CODER = StringUtf8Coder.of(); - public static final Coder CODER = new AtomicCoder() { + public static final Coder CODER = new CustomCoder() { @Override public void encode(Bid value, OutputStream outStream, Coder.Context context) @@ -63,6 +61,8 @@ public Bid decode( String extra = STRING_CODER.decode(inStream, Context.NESTED); return new Bid(auction, bidder, price, dateTime, extra); } + + @Override public void verifyDeterministic() throws NonDeterministicException {} }; /** diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/BidsPerSession.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/BidsPerSession.java index 7c4dfae56c9df..26b6a414d39fb 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/BidsPerSession.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/BidsPerSession.java @@ -19,16 +19,14 @@ import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.core.JsonProcessingException; - import java.io.IOException; import java.io.InputStream; import java.io.OutputStream; import java.io.Serializable; - import org.apache.beam.integration.nexmark.NexmarkUtils; -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.CustomCoder; import org.apache.beam.sdk.coders.VarLongCoder; /** @@ -37,7 +35,7 @@ public class BidsPerSession implements KnownSize, Serializable { private static final Coder LONG_CODER = VarLongCoder.of(); - public static final Coder CODER = new AtomicCoder() { + public static final Coder CODER = new CustomCoder() { @Override public void encode(BidsPerSession value, OutputStream outStream, Coder.Context context) @@ -54,6 +52,7 @@ public BidsPerSession decode( long bidsPerSession = LONG_CODER.decode(inStream, Context.NESTED); return new BidsPerSession(personId, bidsPerSession); } + @Override public void verifyDeterministic() throws NonDeterministicException {} }; @JsonProperty diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/CategoryPrice.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/CategoryPrice.java index 6512cc1301d6d..ccb2bc7ed4228 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/CategoryPrice.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/CategoryPrice.java @@ -19,16 +19,14 @@ import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.core.JsonProcessingException; - import java.io.IOException; import java.io.InputStream; import java.io.OutputStream; import java.io.Serializable; - import org.apache.beam.integration.nexmark.NexmarkUtils; -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.CustomCoder; import org.apache.beam.sdk.coders.VarIntCoder; import org.apache.beam.sdk.coders.VarLongCoder; @@ -39,7 +37,7 @@ public class CategoryPrice implements KnownSize, Serializable { private static final Coder LONG_CODER = VarLongCoder.of(); private static final Coder INT_CODER = VarIntCoder.of(); - public static final Coder CODER = new AtomicCoder() { + public static final Coder CODER = new CustomCoder() { @Override public void encode(CategoryPrice value, OutputStream outStream, Coder.Context context) @@ -58,6 +56,7 @@ public CategoryPrice decode( boolean isLast = INT_CODER.decode(inStream, context) != 0; return new CategoryPrice(category, price, isLast); } + @Override public void verifyDeterministic() throws NonDeterministicException {} }; @JsonProperty diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/Done.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/Done.java index 6009463970feb..42999cd0a3c91 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/Done.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/Done.java @@ -19,16 +19,14 @@ import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.core.JsonProcessingException; - import java.io.IOException; import java.io.InputStream; import java.io.OutputStream; import java.io.Serializable; - import org.apache.beam.integration.nexmark.NexmarkUtils; -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.CustomCoder; import org.apache.beam.sdk.coders.StringUtf8Coder; /** @@ -37,7 +35,7 @@ public class Done implements KnownSize, Serializable { private static final Coder STRING_CODER = StringUtf8Coder.of(); - public static final Coder CODER = new AtomicCoder() { + public static final Coder CODER = new CustomCoder() { @Override public void encode(Done value, OutputStream outStream, Coder.Context context) @@ -52,6 +50,7 @@ public Done decode( String message = STRING_CODER.decode(inStream, Context.NESTED); return new Done(message); } + @Override public void verifyDeterministic() throws NonDeterministicException {} }; @JsonProperty diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/Event.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/Event.java index 8a278bfe3c32d..e2130c99fb4ae 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/Event.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/Event.java @@ -22,10 +22,9 @@ import java.io.OutputStream; import java.io.Serializable; import javax.annotation.Nullable; - -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.CustomCoder; import org.apache.beam.sdk.coders.VarIntCoder; /** @@ -35,7 +34,7 @@ public class Event implements KnownSize, Serializable { private static final Coder INT_CODER = VarIntCoder.of(); - public static final Coder CODER = new AtomicCoder() { + public static final Coder CODER = new CustomCoder() { @Override public void encode(Event value, OutputStream outStream, Coder.Context context) throws CoderException, IOException { @@ -71,6 +70,7 @@ public Event decode( throw new RuntimeException("invalid event encoding"); } } + @Override public void verifyDeterministic() throws NonDeterministicException {} }; @Nullable diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/IdNameReserve.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/IdNameReserve.java index 5d22651500f59..cf1e571f5a139 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/IdNameReserve.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/IdNameReserve.java @@ -19,16 +19,14 @@ import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.core.JsonProcessingException; - import java.io.IOException; import java.io.InputStream; import java.io.OutputStream; import java.io.Serializable; - import org.apache.beam.integration.nexmark.NexmarkUtils; -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.CustomCoder; import org.apache.beam.sdk.coders.StringUtf8Coder; import org.apache.beam.sdk.coders.VarLongCoder; @@ -39,7 +37,7 @@ public class IdNameReserve implements KnownSize, Serializable { private static final Coder LONG_CODER = VarLongCoder.of(); private static final Coder STRING_CODER = StringUtf8Coder.of(); - public static final Coder CODER = new AtomicCoder() { + public static final Coder CODER = new CustomCoder() { @Override public void encode(IdNameReserve value, OutputStream outStream, Coder.Context context) @@ -58,6 +56,7 @@ public IdNameReserve decode( long reserve = LONG_CODER.decode(inStream, Context.NESTED); return new IdNameReserve(id, name, reserve); } + @Override public void verifyDeterministic() throws NonDeterministicException {} }; @JsonProperty diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/NameCityStateId.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/NameCityStateId.java index ac22879d5ceec..86d1738c3ee4d 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/NameCityStateId.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/NameCityStateId.java @@ -19,16 +19,14 @@ import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.core.JsonProcessingException; - import java.io.IOException; import java.io.InputStream; import java.io.OutputStream; import java.io.Serializable; - import org.apache.beam.integration.nexmark.NexmarkUtils; -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.CustomCoder; import org.apache.beam.sdk.coders.StringUtf8Coder; import org.apache.beam.sdk.coders.VarLongCoder; @@ -39,7 +37,7 @@ public class NameCityStateId implements KnownSize, Serializable { private static final Coder LONG_CODER = VarLongCoder.of(); private static final Coder STRING_CODER = StringUtf8Coder.of(); - public static final Coder CODER = new AtomicCoder() { + public static final Coder CODER = new CustomCoder() { @Override public void encode(NameCityStateId value, OutputStream outStream, Coder.Context context) @@ -60,6 +58,7 @@ public NameCityStateId decode( long id = LONG_CODER.decode(inStream, Context.NESTED); return new NameCityStateId(name, city, state, id); } + @Override public void verifyDeterministic() throws NonDeterministicException {} }; @JsonProperty diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/Person.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/Person.java index 85c71839b32dc..906df941798b5 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/Person.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/Person.java @@ -19,16 +19,14 @@ import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.core.JsonProcessingException; - import java.io.IOException; import java.io.InputStream; import java.io.OutputStream; import java.io.Serializable; - import org.apache.beam.integration.nexmark.NexmarkUtils; -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.CustomCoder; import org.apache.beam.sdk.coders.StringUtf8Coder; import org.apache.beam.sdk.coders.VarLongCoder; @@ -38,7 +36,7 @@ public class Person implements KnownSize, Serializable { private static final Coder LONG_CODER = VarLongCoder.of(); private static final Coder STRING_CODER = StringUtf8Coder.of(); - public static final Coder CODER = new AtomicCoder() { + public static final Coder CODER = new CustomCoder() { @Override public void encode(Person value, OutputStream outStream, Coder.Context context) @@ -67,6 +65,7 @@ public Person decode( String extra = STRING_CODER.decode(inStream, Context.NESTED); return new Person(id, name, emailAddress, creditCard, city, state, dateTime, extra); } + @Override public void verifyDeterministic() throws NonDeterministicException {} }; /** Id of person. */ diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/SellerPrice.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/SellerPrice.java index b7c2b1414a539..68f2697c8162b 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/SellerPrice.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/SellerPrice.java @@ -19,16 +19,14 @@ import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.core.JsonProcessingException; - import java.io.IOException; import java.io.InputStream; import java.io.OutputStream; import java.io.Serializable; - import org.apache.beam.integration.nexmark.NexmarkUtils; -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.CustomCoder; import org.apache.beam.sdk.coders.VarLongCoder; /** @@ -37,7 +35,7 @@ public class SellerPrice implements KnownSize, Serializable { private static final Coder LONG_CODER = VarLongCoder.of(); - public static final Coder CODER = new AtomicCoder() { + public static final Coder CODER = new CustomCoder() { @Override public void encode(SellerPrice value, OutputStream outStream, Coder.Context context) @@ -54,6 +52,7 @@ public SellerPrice decode( long price = LONG_CODER.decode(inStream, Context.NESTED); return new SellerPrice(seller, price); } + @Override public void verifyDeterministic() throws NonDeterministicException {} }; @JsonProperty diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/sources/Generator.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/sources/Generator.java index cffc7a5c97717..012d4e65007b2 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/sources/Generator.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/sources/Generator.java @@ -27,14 +27,13 @@ import java.util.Iterator; import java.util.List; import java.util.Random; - import org.apache.beam.integration.nexmark.model.Auction; import org.apache.beam.integration.nexmark.model.Bid; import org.apache.beam.integration.nexmark.model.Event; import org.apache.beam.integration.nexmark.model.Person; -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.CustomCoder; import org.apache.beam.sdk.coders.VarLongCoder; import org.apache.beam.sdk.io.UnboundedSource; import org.apache.beam.sdk.values.TimestampedValue; @@ -102,7 +101,7 @@ public static class Checkpoint implements UnboundedSource.CheckpointMark { /** Coder for this class. */ public static final Coder CODER_INSTANCE = - new AtomicCoder() { + new CustomCoder() { @Override public void encode( Checkpoint value, @@ -121,6 +120,7 @@ public Checkpoint decode( long wallclockBaseTime = LONG_CODER.decode(inStream, Context.NESTED); return new Checkpoint(numEvents, wallclockBaseTime); } + @Override public void verifyDeterministic() throws NonDeterministicException {} }; private long numEvents; From a6dbdfa5457344191ebba383174063270239d9fa Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Isma=C3=ABl=20Mej=C3=ADa?= Date: Sat, 29 Apr 2017 16:50:22 +0200 Subject: [PATCH 303/346] Remove NexmarkDrivers and make execution runner-agnostic This configuration should be external to the benchmark to avoid unexpected dependencies and to have a more Beam like (runner-independent) apprach. Add maven profiles to execute NexMark in the different runners Fix compile after extra PubSubIO refactor and remove PubsubClient. PubsubClient was used to create and reuse topics, this logic should not be part of Nexmark because this add extra complexity/dependencies. A simple script should do this, or the user should provide the corresponding topics. --- integration/java/nexmark/pom.xml | 294 +++++----- .../integration/nexmark/NexmarkDriver.java | 9 + .../integration/nexmark/NexmarkRunner.java | 106 +--- .../nexmark/drivers/NexmarkApexDriver.java | 50 -- .../nexmark/drivers/NexmarkApexRunner.java | 65 --- .../nexmark/drivers/NexmarkDirectDriver.java | 49 -- .../nexmark/drivers/NexmarkDirectRunner.java | 60 -- .../nexmark/drivers/NexmarkFlinkDriver.java | 50 -- .../nexmark/drivers/NexmarkFlinkRunner.java | 55 -- .../nexmark/drivers/NexmarkGoogleDriver.java | 67 --- .../nexmark/drivers/NexmarkGoogleRunner.java | 165 ------ .../nexmark/drivers/NexmarkSparkDriver.java | 48 -- .../nexmark/drivers/NexmarkSparkRunner.java | 56 -- .../nexmark/drivers/package-info.java | 22 - .../integration/nexmark/io/PubsubClient.java | 543 ------------------ .../integration/nexmark/io/PubsubHelper.java | 215 ------- .../nexmark/io/PubsubJsonClient.java | 318 ---------- .../nexmark/io/PubsubTestClient.java | 436 -------------- .../integration/nexmark/io/package-info.java | 22 - .../sources/BoundedEventSourceTest.java | 8 +- 20 files changed, 172 insertions(+), 2466 deletions(-) delete mode 100644 integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/drivers/NexmarkApexDriver.java delete mode 100644 integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/drivers/NexmarkApexRunner.java delete mode 100644 integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/drivers/NexmarkDirectDriver.java delete mode 100644 integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/drivers/NexmarkDirectRunner.java delete mode 100644 integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/drivers/NexmarkFlinkDriver.java delete mode 100644 integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/drivers/NexmarkFlinkRunner.java delete mode 100644 integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/drivers/NexmarkGoogleDriver.java delete mode 100644 integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/drivers/NexmarkGoogleRunner.java delete mode 100644 integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/drivers/NexmarkSparkDriver.java delete mode 100644 integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/drivers/NexmarkSparkRunner.java delete mode 100644 integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/drivers/package-info.java delete mode 100644 integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/io/PubsubClient.java delete mode 100644 integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/io/PubsubHelper.java delete mode 100644 integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/io/PubsubJsonClient.java delete mode 100644 integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/io/PubsubTestClient.java delete mode 100644 integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/io/package-info.java diff --git a/integration/java/nexmark/pom.xml b/integration/java/nexmark/pom.xml index 103c18f28f703..fb213e9c9b809 100644 --- a/integration/java/nexmark/pom.xml +++ b/integration/java/nexmark/pom.xml @@ -31,42 +31,112 @@ Apache Beam :: Integration Tests :: Java :: Nexmark jar - - 1.2.0 - 1.6.3 - 1.9.3 - true - + + + + + direct-runner + + true + + + + org.apache.beam + beam-runners-direct-java + runtime + + + + + + + apex-runner + + + org.apache.beam + beam-runners-apex + runtime + + + + org.apache.httpcomponents + httpclient + 4.3.5 + runtime + + + commons-codec + commons-codec + + + + + + + + + flink-runner + + + org.apache.beam + beam-runners-flink_2.10 + runtime + + + + + + + spark-runner + + + org.apache.beam + beam-runners-spark + runtime + + + org.apache.spark + spark-streaming_2.10 + ${spark.version} + runtime + + + org.apache.spark + spark-core_2.10 + ${spark.version} + runtime + + + org.slf4j + jul-to-slf4j + + + + + + + + + dataflow-runner + + + org.apache.beam + beam-runners-google-cloud-dataflow-java + runtime + + + + - - org.apache.maven.plugins - maven-compiler-plugin - - - - org.apache.maven.plugins - maven-surefire-plugin - - - - - - - - - - org.apache.maven.plugins - maven-checkstyle-plugin - - - - - org.apache.maven.plugins - maven-source-plugin - - org.apache.maven.plugins maven-shade-plugin @@ -98,11 +168,6 @@ - - org.apache.maven.plugins - maven-jar-plugin - - org.apache.avro @@ -127,22 +192,6 @@ org.jacoco jacoco-maven-plugin - - - org.apache.maven.plugins - maven-dependency-plugin - - - analyze-only - - - true - false - - - - - @@ -153,73 +202,6 @@ beam-sdks-java-core - - - org.apache.beam - beam-runners-google-cloud-dataflow-java - - - - - org.apache.beam - beam-runners-direct-java - - - - - org.apache.beam - beam-runners-flink_2.10 - - - - org.apache.flink - flink-shaded-hadoop2 - ${flink.version} - provided - - - - - org.apache.beam - beam-runners-spark - - - org.apache.spark - spark-core_2.10 - ${spark.version} - runtime - - - org.apache.spark - spark-streaming_2.10 - ${spark.version} - runtime - - - - - org.apache.beam - beam-runners-apex - - - com.esotericsoftware.kryo - kryo - ${apex.kryo.version} - runtime - - - org.codehaus.jackson - jackson-mapper-asl - ${apex.codehaus.jackson.version} - runtime - - - org.codehaus.jackson - jackson-core-asl - ${apex.codehaus.jackson.version} - runtime - - org.apache.beam @@ -231,57 +213,20 @@ beam-sdks-java-extensions-gcp-core - - - com.google.apis - google-api-services-dataflow - ${dataflow.version} - runtime - - - - joda-time - joda-time - - - - com.fasterxml.jackson.core - jackson-core - - com.google.apis google-api-services-bigquery - - com.google.apis - google-api-services-pubsub - - - - com.google.auth - google-auth-library-credentials - - - - com.google.auth - google-auth-library-oauth2-http - - com.google.cloud.bigdataoss gcsio + - com.google.cloud.bigdataoss - util - - - - com.google.http-client - google-http-client + com.fasterxml.jackson.core + jackson-core @@ -300,19 +245,18 @@ - com.google.guava - guava + joda-time + joda-time - org.hamcrest - hamcrest-all + com.google.guava + guava org.slf4j slf4j-api - compile @@ -325,5 +269,23 @@ junit compile + + + org.hamcrest + hamcrest-core + compile + + + + org.hamcrest + hamcrest-all + + + + + org.apache.beam + beam-runners-direct-java + test + diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkDriver.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkDriver.java index 4714124a7b041..7d532ccbc70ec 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkDriver.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkDriver.java @@ -31,6 +31,7 @@ import org.apache.beam.integration.nexmark.model.Auction; import org.apache.beam.integration.nexmark.model.Bid; import org.apache.beam.integration.nexmark.model.Person; +import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.joda.time.Duration; import org.joda.time.Instant; @@ -294,4 +295,12 @@ private static void saveJavascript( } NexmarkUtils.console("saved javascript to file %s.", javascriptFilename); } + + public static void main(String[] args) { + NexmarkOptions options = PipelineOptionsFactory.fromArgs(args) + .withValidation() + .as(NexmarkOptions.class); + NexmarkRunner runner = new NexmarkRunner(options); + new NexmarkDriver().runAll(options, runner); + } } diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkRunner.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkRunner.java index 87314cebddce8..ebfd196d814ce 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkRunner.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkRunner.java @@ -33,7 +33,6 @@ import java.util.NoSuchElementException; import java.util.concurrent.ThreadLocalRandom; import javax.annotation.Nullable; -import org.apache.beam.integration.nexmark.io.PubsubHelper; import org.apache.beam.integration.nexmark.model.Auction; import org.apache.beam.integration.nexmark.model.Bid; import org.apache.beam.integration.nexmark.model.Event; @@ -86,7 +85,7 @@ /** * Run a single Nexmark query using a given configuration. */ -public abstract class NexmarkRunner { +public class NexmarkRunner { /** * Minimum number of samples needed for 'stead-state' rate calculation. */ @@ -124,12 +123,6 @@ public abstract class NexmarkRunner { @Nullable protected NexmarkConfiguration configuration; - /** - * Accumulate the pub/sub subscriptions etc which should be cleaned up on end of run. - */ - @Nullable - protected PubsubHelper pubsub; - /** * If in --pubsubMode=COMBINED, the event monitor for the publisher pipeline. Otherwise null. */ @@ -158,16 +151,6 @@ public NexmarkRunner(OptionT options) { this.options = options; } - /** - * Return a Pubsub helper. - */ - private PubsubHelper getPubsub() { - if (pubsub == null) { - pubsub = PubsubHelper.create(options); - } - return pubsub; - } - // ================================================================================ // Overridden by each runner. // ================================================================================ @@ -175,17 +158,23 @@ private PubsubHelper getPubsub() { /** * Is this query running in streaming mode? */ - protected abstract boolean isStreaming(); + protected boolean isStreaming() { + return options.isStreaming(); + } /** * Return number of cores per worker. */ - protected abstract int coresPerWorker(); + protected int coresPerWorker() { + return 4; + } /** * Return maximum number of workers. */ - protected abstract int maxNumWorkers(); + protected int maxNumWorkers() { + return 5; + } /** * Return the current value for a long counter, or a default value if can't be retrieved. @@ -544,13 +533,20 @@ protected interface PipelineBuilder { /** * Invoke the builder with options suitable for running a publish-only child pipeline. */ - protected abstract void invokeBuilderForPublishOnlyPipeline(PipelineBuilder builder); + protected void invokeBuilderForPublishOnlyPipeline( + PipelineBuilder builder) { + builder.build(options); +// throw new UnsupportedOperationException( +// "Cannot use --pubSubMode=COMBINED with DirectRunner"); + } /** * If monitoring, wait until the publisher pipeline has run long enough to establish * a backlog on the Pubsub topic. Otherwise, return immediately. */ - protected abstract void waitForPublisherPreload(); + protected void waitForPublisherPreload() { + throw new UnsupportedOperationException(); + } /** * Monitor the performance and progress of a running job. Return final performance if @@ -841,24 +837,14 @@ private PCollection sourceEventsFromSynthetic(Pipeline p) { * Return source of events from Pubsub. */ private PCollection sourceEventsFromPubsub(Pipeline p, long now) { - String shortTopic = shortTopic(now); String shortSubscription = shortSubscription(now); - - // Create/confirm the subscription. - String subscription = null; - if (!options.getManageResources()) { - // The subscription should already have been created by the user. - subscription = getPubsub().reuseSubscription(shortTopic, shortSubscription).getPath(); - } else { - subscription = getPubsub().createSubscription(shortTopic, shortSubscription).getPath(); - } - NexmarkUtils.console("Reading events from Pubsub %s", subscription); + NexmarkUtils.console("Reading events from Pubsub %s", shortSubscription); PubsubIO.Read io = - PubsubIO.read().subscription(subscription) - .idLabel(NexmarkUtils.PUBSUB_ID) + PubsubIO.read().fromSubscription(shortSubscription) + .withIdAttribute(NexmarkUtils.PUBSUB_ID) .withCoder(Event.CODER); if (!configuration.usePubsubPublishTime) { - io = io.timestampLabel(NexmarkUtils.PUBSUB_TIMESTAMP); + io = io.withTimestampAttribute(NexmarkUtils.PUBSUB_TIMESTAMP); } return p.apply(queryName + ".ReadPubsubEvents", io); } @@ -884,26 +870,13 @@ private PCollection sourceEventsFromAvro(Pipeline p) { */ private void sinkEventsToPubsub(PCollection events, long now) { String shortTopic = shortTopic(now); - - // Create/confirm the topic. - String topic; - if (!options.getManageResources() - || configuration.pubSubMode == NexmarkUtils.PubSubMode.SUBSCRIBE_ONLY) { - // The topic should already have been created by the user or - // a companion 'PUBLISH_ONLY' process. - topic = getPubsub().reuseTopic(shortTopic).getPath(); - } else { - // Create a fresh topic to loopback via. It will be destroyed when the - // (necessarily blocking) job is done. - topic = getPubsub().createTopic(shortTopic).getPath(); - } - NexmarkUtils.console("Writing events to Pubsub %s", topic); + NexmarkUtils.console("Writing events to Pubsub %s", shortTopic); PubsubIO.Write io = - PubsubIO.write().topic(topic) - .idLabel(NexmarkUtils.PUBSUB_ID) + PubsubIO.write().to(shortTopic) + .withIdAttribute(NexmarkUtils.PUBSUB_ID) .withCoder(Event.CODER); if (!configuration.usePubsubPublishTime) { - io = io.timestampLabel(NexmarkUtils.PUBSUB_TIMESTAMP); + io = io.withTimestampAttribute(NexmarkUtils.PUBSUB_TIMESTAMP); } events.apply(queryName + ".WritePubsubEvents", io); } @@ -913,18 +886,12 @@ private void sinkEventsToPubsub(PCollection events, long now) { */ private void sinkResultsToPubsub(PCollection formattedResults, long now) { String shortTopic = shortTopic(now); - String topic; - if (!options.getManageResources()) { - topic = getPubsub().reuseTopic(shortTopic).getPath(); - } else { - topic = getPubsub().createTopic(shortTopic).getPath(); - } - NexmarkUtils.console("Writing results to Pubsub %s", topic); + NexmarkUtils.console("Writing results to Pubsub %s", shortTopic); PubsubIO.Write io = - PubsubIO.write().topic(topic) - .idLabel(NexmarkUtils.PUBSUB_ID); + PubsubIO.write().to(shortTopic) + .withIdAttribute(NexmarkUtils.PUBSUB_ID); if (!configuration.usePubsubPublishTime) { - io = io.timestampLabel(NexmarkUtils.PUBSUB_TIMESTAMP); + io = io.withTimestampAttribute(NexmarkUtils.PUBSUB_TIMESTAMP); } formattedResults.apply(queryName + ".WritePubsubResults", io); } @@ -1168,7 +1135,6 @@ public NexmarkPerf run(NexmarkConfiguration runConfiguration) { // Setup per-run state. // checkState(configuration == null); - checkState(pubsub == null); checkState(queryName == null); configuration = runConfiguration; @@ -1282,19 +1248,9 @@ public NexmarkPerf run(NexmarkConfiguration runConfiguration) { mainResult.waitUntilFinish(Duration.standardSeconds(configuration.streamTimeout)); return monitor(query); } finally { - // - // Cleanup per-run state. - // - if (pubsub != null) { - // Delete any subscriptions and topics we created. - pubsub.close(); - pubsub = null; - } configuration = null; queryName = null; // TODO: Cleanup pathsToDelete } - } - } diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/drivers/NexmarkApexDriver.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/drivers/NexmarkApexDriver.java deleted file mode 100644 index 265ccf747460c..0000000000000 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/drivers/NexmarkApexDriver.java +++ /dev/null @@ -1,50 +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.integration.nexmark.drivers; - -import org.apache.beam.integration.nexmark.NexmarkDriver; -import org.apache.beam.integration.nexmark.NexmarkOptions; -import org.apache.beam.runners.apex.ApexPipelineOptions; -import org.apache.beam.runners.apex.ApexRunner; -import org.apache.beam.sdk.options.PipelineOptionsFactory; - -/** - * Run NexMark queries using the Apex runner. - */ -public class NexmarkApexDriver extends NexmarkDriver { - /** - * Command line flags. - */ - public interface NexmarkApexOptions extends NexmarkOptions, ApexPipelineOptions { - } - - /** - * Entry point. - */ - public static void main(String[] args) { - // Gather command line args, baseline, configurations, etc. - NexmarkApexOptions options = PipelineOptionsFactory.fromArgs(args) - .withValidation() - .as(NexmarkApexOptions.class); - options.setRunner(ApexRunner.class); - NexmarkApexRunner runner = new NexmarkApexRunner(options); - new NexmarkApexDriver().runAll(options, runner); - } -} - - diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/drivers/NexmarkApexRunner.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/drivers/NexmarkApexRunner.java deleted file mode 100644 index 2bcf82d66c1ab..0000000000000 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/drivers/NexmarkApexRunner.java +++ /dev/null @@ -1,65 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.beam.integration.nexmark.drivers; - -import javax.annotation.Nullable; - -import org.apache.beam.integration.nexmark.NexmarkPerf; -import org.apache.beam.integration.nexmark.NexmarkQuery; -import org.apache.beam.integration.nexmark.NexmarkRunner; - -/** - * Run a query using the Apex runner. - */ -public class NexmarkApexRunner extends NexmarkRunner { - @Override - protected boolean isStreaming() { - return options.isStreaming(); - } - - @Override - protected int coresPerWorker() { - return 4; - } - - @Override - protected int maxNumWorkers() { - return 5; - } - - @Override - protected void invokeBuilderForPublishOnlyPipeline( - PipelineBuilder builder) { - builder.build(options); - } - - @Override - protected void waitForPublisherPreload() { - throw new UnsupportedOperationException(); - } - - @Override - @Nullable - protected NexmarkPerf monitor(NexmarkQuery query) { - return null; - } - - public NexmarkApexRunner(NexmarkApexDriver.NexmarkApexOptions options) { - super(options); - } -} diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/drivers/NexmarkDirectDriver.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/drivers/NexmarkDirectDriver.java deleted file mode 100644 index 2b825f33ba7ff..0000000000000 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/drivers/NexmarkDirectDriver.java +++ /dev/null @@ -1,49 +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.integration.nexmark.drivers; - -import org.apache.beam.integration.nexmark.NexmarkDriver; -import org.apache.beam.integration.nexmark.NexmarkOptions; -import org.apache.beam.runners.direct.DirectOptions; -import org.apache.beam.runners.direct.DirectRunner; -import org.apache.beam.sdk.options.PipelineOptionsFactory; - -/** - * An implementation of the 'NEXMark queries' using the Direct Runner. - */ -class NexmarkDirectDriver extends NexmarkDriver { - /** - * Command line flags. - */ - public interface NexmarkDirectOptions extends NexmarkOptions, DirectOptions { - } - - /** - * Entry point. - */ - public static void main(String[] args) { - NexmarkDirectOptions options = - PipelineOptionsFactory.fromArgs(args) - .withValidation() - .as(NexmarkDirectOptions.class); - options.setRunner(DirectRunner.class); - NexmarkDirectRunner runner = new NexmarkDirectRunner(options); - new NexmarkDirectDriver().runAll(options, runner); - } -} - diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/drivers/NexmarkDirectRunner.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/drivers/NexmarkDirectRunner.java deleted file mode 100644 index 139104061ca48..0000000000000 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/drivers/NexmarkDirectRunner.java +++ /dev/null @@ -1,60 +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.integration.nexmark.drivers; - -import org.apache.beam.integration.nexmark.NexmarkRunner; - -/** - * Run a single query using the Direct Runner. - */ -class NexmarkDirectRunner extends NexmarkRunner { - public NexmarkDirectRunner(NexmarkDirectDriver.NexmarkDirectOptions options) { - super(options); - } - - @Override - protected boolean isStreaming() { - return options.isStreaming(); - } - - @Override - protected int coresPerWorker() { - return 4; - } - - @Override - protected int maxNumWorkers() { - return 1; - } - - @Override - protected void invokeBuilderForPublishOnlyPipeline(PipelineBuilder builder) { - throw new UnsupportedOperationException( - "Cannot use --pubSubMode=COMBINED with DirectRunner"); - } - - /** - * Monitor the progress of the publisher job. Return when it has been generating events for - * at least {@code configuration.preloadSeconds}. - */ - @Override - protected void waitForPublisherPreload() { - throw new UnsupportedOperationException( - "Cannot use --pubSubMode=COMBINED with DirectRunner"); - } -} diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/drivers/NexmarkFlinkDriver.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/drivers/NexmarkFlinkDriver.java deleted file mode 100644 index bf0b115e19808..0000000000000 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/drivers/NexmarkFlinkDriver.java +++ /dev/null @@ -1,50 +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.integration.nexmark.drivers; - -import org.apache.beam.integration.nexmark.NexmarkDriver; -import org.apache.beam.integration.nexmark.NexmarkOptions; -import org.apache.beam.runners.flink.FlinkPipelineOptions; -import org.apache.beam.runners.flink.FlinkRunner; -import org.apache.beam.sdk.options.PipelineOptionsFactory; - -/** - * Run NexMark queries using the Flink runner. - */ -public class NexmarkFlinkDriver extends NexmarkDriver { - /** - * Command line flags. - */ - public interface NexmarkFlinkOptions extends NexmarkOptions, FlinkPipelineOptions { - } - - /** - * Entry point. - */ - public static void main(String[] args) { - // Gather command line args, baseline, configurations, etc. - NexmarkFlinkOptions options = PipelineOptionsFactory.fromArgs(args) - .withValidation() - .as(NexmarkFlinkOptions.class); - options.setRunner(FlinkRunner.class); - NexmarkFlinkRunner runner = new NexmarkFlinkRunner(options); - new NexmarkFlinkDriver().runAll(options, runner); - } -} - - diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/drivers/NexmarkFlinkRunner.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/drivers/NexmarkFlinkRunner.java deleted file mode 100644 index 9d547ef825ed3..0000000000000 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/drivers/NexmarkFlinkRunner.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.integration.nexmark.drivers; - -import org.apache.beam.integration.nexmark.NexmarkRunner; - -/** - * Run a query using the Flink runner. - */ -public class NexmarkFlinkRunner extends NexmarkRunner { - @Override - protected boolean isStreaming() { - return options.isStreaming(); - } - - @Override - protected int coresPerWorker() { - return 4; - } - - @Override - protected int maxNumWorkers() { - return 5; - } - - @Override - protected void invokeBuilderForPublishOnlyPipeline( - PipelineBuilder builder) { - builder.build(options); - } - - @Override - protected void waitForPublisherPreload() { - throw new UnsupportedOperationException(); - } - - public NexmarkFlinkRunner(NexmarkFlinkDriver.NexmarkFlinkOptions options) { - super(options); - } -} diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/drivers/NexmarkGoogleDriver.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/drivers/NexmarkGoogleDriver.java deleted file mode 100644 index f5a9751cbd6a2..0000000000000 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/drivers/NexmarkGoogleDriver.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.integration.nexmark.drivers; - -import org.apache.beam.integration.nexmark.NexmarkDriver; -import org.apache.beam.integration.nexmark.NexmarkOptions; -import org.apache.beam.integration.nexmark.model.Auction; -import org.apache.beam.integration.nexmark.model.Bid; -import org.apache.beam.integration.nexmark.model.Person; -import org.apache.beam.runners.dataflow.DataflowRunner; -import org.apache.beam.runners.dataflow.options.DataflowPipelineOptions; -import org.apache.beam.sdk.options.PipelineOptionsFactory; - -/** - * An implementation of the 'NEXMark queries' for Google Dataflow. - * These are multiple queries over a three table schema representing an online auction system: - *

                - *
              • {@link Person} represents a person submitting an item for auction and/or making a bid - * on an auction. - *
              • {@link Auction} represents an item under auction. - *
              • {@link Bid} represents a bid for an item under auction. - *
              - * The queries exercise many aspects of streaming dataflow. - * - *

              We synthesize the creation of people, auctions and bids in real-time. The data is not - * particularly sensible. - * - *

              See - * http://datalab.cs.pdx.edu/niagaraST/NEXMark/ - */ -class NexmarkGoogleDriver extends NexmarkDriver { - /** - * Command line flags. - */ - public interface NexmarkGoogleOptions extends NexmarkOptions, DataflowPipelineOptions { - - } - - /** - * Entry point. - */ - public static void main(String[] args) { - // Gather command line args, baseline, configurations, etc. - NexmarkGoogleOptions options = PipelineOptionsFactory.fromArgs(args) - .withValidation() - .as(NexmarkGoogleOptions.class); - options.setRunner(DataflowRunner.class); - NexmarkGoogleRunner runner = new NexmarkGoogleRunner(options); - new NexmarkGoogleDriver().runAll(options, runner); - } -} - diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/drivers/NexmarkGoogleRunner.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/drivers/NexmarkGoogleRunner.java deleted file mode 100644 index 935bf0d2735cb..0000000000000 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/drivers/NexmarkGoogleRunner.java +++ /dev/null @@ -1,165 +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.integration.nexmark.drivers; - -import static com.google.common.base.Preconditions.checkNotNull; - -import org.apache.beam.integration.nexmark.Monitor; -import org.apache.beam.integration.nexmark.NexmarkRunner; -import org.apache.beam.integration.nexmark.NexmarkUtils; -import org.apache.beam.integration.nexmark.model.Event; -import org.apache.beam.runners.dataflow.DataflowPipelineJob; -import org.apache.beam.sdk.PipelineResult; -import org.joda.time.Duration; - -/** - * Run a singe Nexmark query using a given configuration on Google Dataflow. - */ -class NexmarkGoogleRunner extends NexmarkRunner { - - public NexmarkGoogleRunner(NexmarkGoogleDriver.NexmarkGoogleOptions options) { - super(options); - } - - @Override - protected boolean isStreaming() { - return options.isStreaming(); - } - - @Override - protected int coresPerWorker() { - String machineType = options.getWorkerMachineType(); - if (machineType == null || machineType.isEmpty()) { - return 1; - } - String[] split = machineType.split("-"); - if (split.length != 3) { - return 1; - } - try { - return Integer.parseInt(split[2]); - } catch (NumberFormatException ex) { - return 1; - } - } - - @Override - protected int maxNumWorkers() { - return Math.max(options.getNumWorkers(), options.getMaxNumWorkers()); - } - - @Override - protected String getJobId(PipelineResult job) { - return ((DataflowPipelineJob) job).getJobId(); - } - - @Override - protected void invokeBuilderForPublishOnlyPipeline(PipelineBuilder builder) { - String jobName = options.getJobName(); - String appName = options.getAppName(); - options.setJobName("p-" + jobName); - options.setAppName("p-" + appName); - int coresPerWorker = coresPerWorker(); - int eventGeneratorWorkers = (configuration.numEventGenerators + coresPerWorker - 1) - / coresPerWorker; - options.setMaxNumWorkers(Math.min(options.getMaxNumWorkers(), eventGeneratorWorkers)); - options.setNumWorkers(Math.min(options.getNumWorkers(), eventGeneratorWorkers)); - publisherMonitor = new Monitor(queryName, "publisher"); - try { - builder.build(options); - } finally { - options.setJobName(jobName); - options.setAppName(appName); - options.setMaxNumWorkers(options.getMaxNumWorkers()); - options.setNumWorkers(options.getNumWorkers()); - } - } - - /** - * Monitor the progress of the publisher job. Return when it has been generating events for - * at least {@code configuration.preloadSeconds}. - */ - @Override - protected void waitForPublisherPreload() { - checkNotNull(publisherMonitor); - checkNotNull(publisherResult); - if (!options.getMonitorJobs()) { - return; - } - if (!(publisherResult instanceof DataflowPipelineJob)) { - return; - } - if (configuration.preloadSeconds <= 0) { - return; - } - - NexmarkUtils.console("waiting for publisher to pre-load"); - - DataflowPipelineJob job = (DataflowPipelineJob) publisherResult; - - long numEvents = 0; - long startMsSinceEpoch = -1; - long endMsSinceEpoch = -1; - while (true) { - PipelineResult.State state = job.getState(); - switch (state) { - case UNKNOWN: - // Keep waiting. - NexmarkUtils.console("%s publisher (%d events)", state, numEvents); - break; - case STOPPED: - case DONE: - case CANCELLED: - case FAILED: - case UPDATED: - NexmarkUtils.console("%s publisher (%d events)", state, numEvents); - return; - case RUNNING: - //TODO Ismael Validate that this counter is ok - numEvents = - getCounterMetric(job, publisherMonitor.name, publisherMonitor.prefix + ".elements", -1); - if (startMsSinceEpoch < 0 && numEvents > 0) { - startMsSinceEpoch = System.currentTimeMillis(); - endMsSinceEpoch = startMsSinceEpoch - + Duration.standardSeconds(configuration.preloadSeconds).getMillis(); - } - if (endMsSinceEpoch < 0) { - NexmarkUtils.console("%s publisher (%d events)", state, numEvents); - } else { - long remainMs = endMsSinceEpoch - System.currentTimeMillis(); - if (remainMs > 0) { - NexmarkUtils.console("%s publisher (%d events, waiting for %ds)", state, numEvents, - remainMs / 1000); - } else { - NexmarkUtils.console("publisher preloaded %d events", numEvents); - return; - } - } - break; - } - - try { - Thread.sleep(PERF_DELAY.getMillis()); - } catch (InterruptedException e) { - Thread.interrupted(); - throw new RuntimeException("Interrupted: publisher still running."); - } - } - } - -} diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/drivers/NexmarkSparkDriver.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/drivers/NexmarkSparkDriver.java deleted file mode 100644 index c7c32c2eb1c67..0000000000000 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/drivers/NexmarkSparkDriver.java +++ /dev/null @@ -1,48 +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.integration.nexmark.drivers; - -import org.apache.beam.integration.nexmark.NexmarkDriver; -import org.apache.beam.integration.nexmark.NexmarkOptions; -import org.apache.beam.runners.spark.SparkPipelineOptions; -import org.apache.beam.runners.spark.SparkRunner; -import org.apache.beam.sdk.options.PipelineOptionsFactory; - -/** - * Run NexMark queries using the Spark runner. - */ -class NexmarkSparkDriver extends NexmarkDriver { - /** - * Command line flags. - */ - public interface NexmarkSparkOptions extends NexmarkOptions, SparkPipelineOptions { - } - - /** - * Entry point. - */ - public static void main(String[] args) { - NexmarkSparkOptions options = - PipelineOptionsFactory.fromArgs(args) - .withValidation() - .as(NexmarkSparkOptions.class); - options.setRunner(SparkRunner.class); - NexmarkSparkRunner runner = new NexmarkSparkRunner(options); - new NexmarkSparkDriver().runAll(options, runner); - } -} diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/drivers/NexmarkSparkRunner.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/drivers/NexmarkSparkRunner.java deleted file mode 100644 index 1d49a3a7c4975..0000000000000 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/drivers/NexmarkSparkRunner.java +++ /dev/null @@ -1,56 +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.integration.nexmark.drivers; - -import org.apache.beam.integration.nexmark.NexmarkRunner; - -/** - * Run a query using the Spark runner. - */ -public class NexmarkSparkRunner extends NexmarkRunner { - @Override - protected boolean isStreaming() { - return options.isStreaming(); - } - - @Override - protected int coresPerWorker() { - return 4; - } - - @Override - protected int maxNumWorkers() { - return 5; - } - - @Override - protected void invokeBuilderForPublishOnlyPipeline( - PipelineBuilder builder) { - builder.build(options); - } - - @Override - protected void waitForPublisherPreload() { - throw new UnsupportedOperationException(); - } - - - public NexmarkSparkRunner(NexmarkSparkDriver.NexmarkSparkOptions options) { - super(options); - } -} diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/drivers/package-info.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/drivers/package-info.java deleted file mode 100644 index c8aa144c4fcf8..0000000000000 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/drivers/package-info.java +++ /dev/null @@ -1,22 +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. - */ - -/** - * Nexmark Benchmark Execution Drivers. - */ -package org.apache.beam.integration.nexmark.drivers; diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/io/PubsubClient.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/io/PubsubClient.java deleted file mode 100644 index 931fe6e69da4f..0000000000000 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/io/PubsubClient.java +++ /dev/null @@ -1,543 +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.integration.nexmark.io; - -import static com.google.common.base.Preconditions.checkArgument; -import static com.google.common.base.Preconditions.checkState; - -import com.google.api.client.util.DateTime; -import com.google.common.base.Objects; -import com.google.common.base.Strings; -import java.io.Closeable; -import java.io.IOException; -import java.io.Serializable; -import java.util.Arrays; -import java.util.List; -import java.util.Map; -import java.util.concurrent.ThreadLocalRandom; -import javax.annotation.Nullable; -import org.apache.beam.sdk.io.gcp.pubsub.PubsubOptions; - -/** - * An (abstract) helper class for talking to Pubsub via an underlying transport. - */ -abstract class PubsubClient implements Closeable { - /** - * Factory for creating clients. - */ - public interface PubsubClientFactory extends Serializable { - /** - * Construct a new Pubsub client. It should be closed via {@link #close} in order - * to ensure tidy cleanup of underlying netty resources (or use the try-with-resources - * construct). Uses {@code options} to derive pubsub endpoints and application credentials. - * If non-{@literal null}, use {@code timestampLabel} and {@code idLabel} to store custom - * timestamps/ids within message metadata. - */ - PubsubClient newClient(@Nullable String timestampLabel, - @Nullable String idLabel, PubsubOptions options) throws IOException; - - /** - * Return the display name for this factory. Eg "Json", "gRPC". - */ - String getKind(); - } - - /** - * Return timestamp as ms-since-unix-epoch corresponding to {@code timestamp}. - * Return {@literal null} if no timestamp could be found. Throw {@link IllegalArgumentException} - * if timestamp cannot be recognized. - */ - @Nullable - private static Long asMsSinceEpoch(@Nullable String timestamp) { - if (Strings.isNullOrEmpty(timestamp)) { - return null; - } - try { - // Try parsing as milliseconds since epoch. Note there is no way to parse a - // string in RFC 3339 format here. - // Expected IllegalArgumentException if parsing fails; we use that to fall back - // to RFC 3339. - return Long.parseLong(timestamp); - } catch (IllegalArgumentException e1) { - // Try parsing as RFC3339 string. DateTime.parseRfc3339 will throw an - // IllegalArgumentException if parsing fails, and the caller should handle. - return DateTime.parseRfc3339(timestamp).getValue(); - } - } - - /** - * Return the timestamp (in ms since unix epoch) to use for a Pubsub message with {@code - * attributes} and {@code pubsubTimestamp}. - * - *

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

              NOTE: This class is {@link Serializable} only to support the {@link PubsubTestClient}. - * Java serialization is never used for non-test clients. - */ - static class OutgoingMessage implements Serializable { - /** - * Underlying (encoded) element. - */ - public final byte[] elementBytes; - - public final Map attributes; - - /** - * Timestamp for element (ms since epoch). - */ - public final long timestampMsSinceEpoch; - - /** - * If using an id label, the record id to associate with this record's metadata so the receiver - * can reject duplicates. Otherwise {@literal null}. - */ - @Nullable - public final String recordId; - - public OutgoingMessage(byte[] elementBytes, Map attributes, - long timestampMsSinceEpoch, @Nullable String recordId) { - this.elementBytes = elementBytes; - this.attributes = attributes; - this.timestampMsSinceEpoch = timestampMsSinceEpoch; - this.recordId = recordId; - } - - @Override - public String toString() { - return String.format("OutgoingMessage(%db, %dms)", - elementBytes.length, timestampMsSinceEpoch); - } - - @Override - public boolean equals(Object o) { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass()) { - return false; - } - - OutgoingMessage that = (OutgoingMessage) o; - - return timestampMsSinceEpoch == that.timestampMsSinceEpoch - && Arrays.equals(elementBytes, that.elementBytes) - && Objects.equal(attributes, that.attributes) - && Objects.equal(recordId, that.recordId); - } - - @Override - public int hashCode() { - return Objects.hashCode(Arrays.hashCode(elementBytes), attributes, timestampMsSinceEpoch, - recordId); - } - } - - /** - * A message received from Pubsub. - * - *

              NOTE: This class is {@link Serializable} only to support the {@link PubsubTestClient}. - * Java serialization is never used for non-test clients. - */ - static class IncomingMessage implements Serializable { - /** - * Underlying (encoded) element. - */ - public final byte[] elementBytes; - - public Map attributes; - - /** - * Timestamp for element (ms since epoch). Either Pubsub's processing time, - * or the custom timestamp associated with the message. - */ - public final long timestampMsSinceEpoch; - - /** - * Timestamp (in system time) at which we requested the message (ms since epoch). - */ - public final long requestTimeMsSinceEpoch; - - /** - * Id to pass back to Pubsub to acknowledge receipt of this message. - */ - public final String ackId; - - /** - * Id to pass to the runner to distinguish this message from all others. - */ - public final String recordId; - - public IncomingMessage( - byte[] elementBytes, - Map attributes, - long timestampMsSinceEpoch, - long requestTimeMsSinceEpoch, - String ackId, - String recordId) { - this.elementBytes = elementBytes; - this.attributes = attributes; - this.timestampMsSinceEpoch = timestampMsSinceEpoch; - this.requestTimeMsSinceEpoch = requestTimeMsSinceEpoch; - this.ackId = ackId; - this.recordId = recordId; - } - - public IncomingMessage withRequestTime(long requestTimeMsSinceEpoch) { - return new IncomingMessage(elementBytes, attributes, timestampMsSinceEpoch, - requestTimeMsSinceEpoch, ackId, recordId); - } - - @Override - public String toString() { - return String.format("IncomingMessage(%db, %dms)", - elementBytes.length, timestampMsSinceEpoch); - } - - @Override - public boolean equals(Object o) { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass()) { - return false; - } - - IncomingMessage that = (IncomingMessage) o; - - return timestampMsSinceEpoch == that.timestampMsSinceEpoch - && requestTimeMsSinceEpoch == that.requestTimeMsSinceEpoch - && ackId.equals(that.ackId) - && recordId.equals(that.recordId) - && Arrays.equals(elementBytes, that.elementBytes) - && Objects.equal(attributes, that.attributes); - } - - @Override - public int hashCode() { - return Objects.hashCode(Arrays.hashCode(elementBytes), attributes, timestampMsSinceEpoch, - requestTimeMsSinceEpoch, - ackId, recordId); - } - } - - /** - * Publish {@code outgoingMessages} to Pubsub {@code topic}. Return number of messages - * published. - */ - public abstract int publish(TopicPath topic, List outgoingMessages) - throws IOException; - - /** - * Request the next batch of up to {@code batchSize} messages from {@code subscription}. - * Return the received messages, or empty collection if none were available. Does not - * wait for messages to arrive if {@code returnImmediately} is {@literal true}. - * Returned messages will record their request time as {@code requestTimeMsSinceEpoch}. - */ - public abstract List pull( - long requestTimeMsSinceEpoch, - SubscriptionPath subscription, - int batchSize, - boolean returnImmediately) - throws IOException; - - /** - * Acknowldege messages from {@code subscription} with {@code ackIds}. - */ - public abstract void acknowledge(SubscriptionPath subscription, List ackIds) - throws IOException; - - /** - * Modify the ack deadline for messages from {@code subscription} with {@code ackIds} to - * be {@code deadlineSeconds} from now. - */ - public abstract void modifyAckDeadline( - SubscriptionPath subscription, List ackIds, - int deadlineSeconds) throws IOException; - - /** - * Create {@code topic}. - */ - public abstract void createTopic(TopicPath topic) throws IOException; - - /* - * Delete {@code topic}. - */ - public abstract void deleteTopic(TopicPath topic) throws IOException; - - /** - * Return a list of topics for {@code project}. - */ - public abstract List listTopics(ProjectPath project) throws IOException; - - /** - * Create {@code subscription} to {@code topic}. - */ - public abstract void createSubscription( - TopicPath topic, SubscriptionPath subscription, int ackDeadlineSeconds) throws IOException; - - /** - * Create a random subscription for {@code topic}. Return the {@link SubscriptionPath}. It - * is the responsibility of the caller to later delete the subscription. - */ - public SubscriptionPath createRandomSubscription( - ProjectPath project, TopicPath topic, int ackDeadlineSeconds) throws IOException { - // Create a randomized subscription derived from the topic name. - String subscriptionName = topic.getName() + "_beam_" + ThreadLocalRandom.current().nextLong(); - SubscriptionPath subscription = - PubsubClient - .subscriptionPathFromName(project.getId(), subscriptionName); - createSubscription(topic, subscription, ackDeadlineSeconds); - return subscription; - } - - /** - * Delete {@code subscription}. - */ - public abstract void deleteSubscription(SubscriptionPath subscription) throws IOException; - - /** - * Return a list of subscriptions for {@code topic} in {@code project}. - */ - public abstract List listSubscriptions(ProjectPath project, TopicPath topic) - throws IOException; - - /** - * Return the ack deadline, in seconds, for {@code subscription}. - */ - public abstract int ackDeadlineSeconds(SubscriptionPath subscription) throws IOException; - - /** - * Return {@literal true} if {@link #pull} will always return empty list. Actual clients - * will return {@literal false}. Test clients may return {@literal true} to signal that all - * expected messages have been pulled and the test may complete. - */ - public abstract boolean isEOF(); -} diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/io/PubsubHelper.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/io/PubsubHelper.java deleted file mode 100644 index bcc5b1ce1e034..0000000000000 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/io/PubsubHelper.java +++ /dev/null @@ -1,215 +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.integration.nexmark.io; - -import java.io.IOException; -import java.util.ArrayList; -import java.util.Collection; -import java.util.List; - -import org.apache.beam.integration.nexmark.NexmarkUtils; -import org.apache.beam.sdk.io.gcp.pubsub.PubsubOptions; - -/** - * Helper for working with pubsub. - */ -public class PubsubHelper implements AutoCloseable { - /** - * Underlying pub/sub client. - */ - private final PubsubClient pubsubClient; - - /** - * Project id. - */ - private final String projectId; - - /** - * Topics we should delete on close. - */ - private final List createdTopics; - - /** - * Subscriptions we should delete on close. - */ - private final List createdSubscriptions; - - private PubsubHelper(PubsubClient pubsubClient, String projectId) { - this.pubsubClient = pubsubClient; - this.projectId = projectId; - createdTopics = new ArrayList<>(); - createdSubscriptions = new ArrayList<>(); - } - - /** - * Create a helper. - */ - public static PubsubHelper create(PubsubOptions options) { - try { - return new PubsubHelper( - PubsubJsonClient.FACTORY.newClient(null, null, options), - options.getProject()); - } catch (IOException e) { - throw new RuntimeException("Unable to create Pubsub client: ", e); - } - } - - /** - * Create a topic from short name. Delete it if it already exists. Ensure the topic will be - * deleted on cleanup. Return full topic name. - */ - public PubsubClient.TopicPath createTopic(String shortTopic) { - PubsubClient.TopicPath topic = PubsubClient.topicPathFromName(projectId, shortTopic); - try { - if (topicExists(shortTopic)) { - NexmarkUtils.console("attempting to cleanup topic %s", topic); - pubsubClient.deleteTopic(topic); - } - NexmarkUtils.console("create topic %s", topic); - pubsubClient.createTopic(topic); - createdTopics.add(topic); - return topic; - } catch (IOException e) { - throw new RuntimeException("Unable to create Pubsub topic " + topic + ": ", e); - } - } - - /** - * Create a topic from short name if it does not already exist. The topic will not be - * deleted on cleanup. Return full topic name. - */ - public PubsubClient.TopicPath createOrReuseTopic(String shortTopic) { - PubsubClient.TopicPath topic = PubsubClient.topicPathFromName(projectId, shortTopic); - try { - if (topicExists(shortTopic)) { - NexmarkUtils.console("topic %s already exists", topic); - return topic; - } - NexmarkUtils.console("create topic %s", topic); - pubsubClient.createTopic(topic); - return topic; - } catch (IOException e) { - throw new RuntimeException("Unable to create or reuse Pubsub topic " + topic + ": ", e); - } - } - - /** - * Check a topic corresponding to short name exists, and throw exception if not. The - * topic will not be deleted on cleanup. Return full topic name. - */ - public PubsubClient.TopicPath reuseTopic(String shortTopic) { - PubsubClient.TopicPath topic = PubsubClient.topicPathFromName(projectId, shortTopic); - if (topicExists(shortTopic)) { - NexmarkUtils.console("reusing existing topic %s", topic); - return topic; - } - throw new RuntimeException("topic '" + topic + "' does not already exist"); - } - - /** - * Does topic corresponding to short name exist? - */ - public boolean topicExists(String shortTopic) { - PubsubClient.ProjectPath project = PubsubClient.projectPathFromId(projectId); - PubsubClient.TopicPath topic = PubsubClient.topicPathFromName(projectId, shortTopic); - try { - Collection existingTopics = pubsubClient.listTopics(project); - return existingTopics.contains(topic); - } catch (IOException e) { - throw new RuntimeException("Unable to check Pubsub topic " + topic + ": ", e); - } - } - - /** - * Create subscription from short name. Delete subscription if it already exists. Ensure the - * subscription will be deleted on cleanup. Return full subscription name. - */ - public PubsubClient.SubscriptionPath createSubscription( - String shortTopic, String shortSubscription) { - PubsubClient.TopicPath topic = PubsubClient.topicPathFromName(projectId, shortTopic); - PubsubClient.SubscriptionPath subscription = - PubsubClient.subscriptionPathFromName(projectId, shortSubscription); - try { - if (subscriptionExists(shortTopic, shortSubscription)) { - NexmarkUtils.console("attempting to cleanup subscription %s", subscription); - pubsubClient.deleteSubscription(subscription); - } - NexmarkUtils.console("create subscription %s", subscription); - pubsubClient.createSubscription(topic, subscription, 60); - createdSubscriptions.add(subscription); - } catch (IOException e) { - throw new RuntimeException("Unable to create Pubsub subscription " + subscription + ": ", e); - } - return subscription; - } - - /** - * Check a subscription corresponding to short name exists, and throw exception if not. The - * subscription will not be deleted on cleanup. Return full topic name. - */ - public PubsubClient.SubscriptionPath reuseSubscription( - String shortTopic, String shortSubscription) { - PubsubClient.SubscriptionPath subscription = - PubsubClient.subscriptionPathFromName(projectId, shortSubscription); - if (subscriptionExists(shortTopic, shortSubscription)) { - NexmarkUtils.console("reusing existing subscription %s", subscription); - return subscription; - } - throw new RuntimeException("subscription'" + subscription + "' does not already exist"); - } - - /** - * Does subscription corresponding to short name exist? - */ - public boolean subscriptionExists(String shortTopic, String shortSubscription) { - PubsubClient.ProjectPath project = PubsubClient.projectPathFromId(projectId); - PubsubClient.TopicPath topic = PubsubClient.topicPathFromName(projectId, shortTopic); - PubsubClient.SubscriptionPath subscription = - PubsubClient.subscriptionPathFromName(projectId, shortSubscription); - try { - Collection existingSubscriptions = - pubsubClient.listSubscriptions(project, topic); - return existingSubscriptions.contains(subscription); - } catch (IOException e) { - throw new RuntimeException("Unable to check Pubsub subscription" + subscription + ": ", e); - } - } - - /** - * Delete all the subscriptions and topics we created. - */ - @Override - public void close() { - for (PubsubClient.SubscriptionPath subscription : createdSubscriptions) { - try { - NexmarkUtils.console("delete subscription %s", subscription); - pubsubClient.deleteSubscription(subscription); - } catch (IOException ex) { - NexmarkUtils.console("could not delete subscription %s", subscription); - } - } - for (PubsubClient.TopicPath topic : createdTopics) { - try { - NexmarkUtils.console("delete topic %s", topic); - pubsubClient.deleteTopic(topic); - } catch (IOException ex) { - NexmarkUtils.console("could not delete topic %s", topic); - } - } - } -} diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/io/PubsubJsonClient.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/io/PubsubJsonClient.java deleted file mode 100644 index afddbd80fe124..0000000000000 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/io/PubsubJsonClient.java +++ /dev/null @@ -1,318 +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.integration.nexmark.io; - -import static com.google.common.base.Preconditions.checkState; - -import com.google.api.client.http.HttpRequestInitializer; -import com.google.api.services.pubsub.Pubsub; -import com.google.api.services.pubsub.Pubsub.Builder; -import com.google.api.services.pubsub.model.AcknowledgeRequest; -import com.google.api.services.pubsub.model.ListSubscriptionsResponse; -import com.google.api.services.pubsub.model.ListTopicsResponse; -import com.google.api.services.pubsub.model.ModifyAckDeadlineRequest; -import com.google.api.services.pubsub.model.PublishRequest; -import com.google.api.services.pubsub.model.PublishResponse; -import com.google.api.services.pubsub.model.PubsubMessage; -import com.google.api.services.pubsub.model.PullRequest; -import com.google.api.services.pubsub.model.PullResponse; -import com.google.api.services.pubsub.model.ReceivedMessage; -import com.google.api.services.pubsub.model.Subscription; -import com.google.api.services.pubsub.model.Topic; -import com.google.auth.Credentials; -import com.google.auth.http.HttpCredentialsAdapter; -import com.google.cloud.hadoop.util.ChainingHttpRequestInitializer; -import com.google.common.annotations.VisibleForTesting; -import com.google.common.base.Strings; -import com.google.common.collect.ImmutableList; -import java.io.IOException; -import java.util.ArrayList; -import java.util.List; -import java.util.Map; -import java.util.TreeMap; -import javax.annotation.Nullable; -import org.apache.beam.sdk.io.gcp.pubsub.PubsubOptions; -import org.apache.beam.sdk.util.RetryHttpRequestInitializer; -import org.apache.beam.sdk.util.Transport; - -/** - * A Pubsub client using JSON transport. - */ -class PubsubJsonClient extends PubsubClient { - - private static class PubsubJsonClientFactory implements PubsubClientFactory { - private static HttpRequestInitializer chainHttpRequestInitializer( - Credentials credential, HttpRequestInitializer httpRequestInitializer) { - if (credential == null) { - return httpRequestInitializer; - } else { - return new ChainingHttpRequestInitializer( - new HttpCredentialsAdapter(credential), - httpRequestInitializer); - } - } - - @Override - public PubsubClient newClient( - @Nullable String timestampLabel, @Nullable String idLabel, PubsubOptions options) - throws IOException { - Pubsub pubsub = new Builder( - Transport.getTransport(), - Transport.getJsonFactory(), - chainHttpRequestInitializer( - options.getGcpCredential(), - // Do not log 404. It clutters the output and is possibly even required by the caller. - new RetryHttpRequestInitializer(ImmutableList.of(404)))) - .setRootUrl(options.getPubsubRootUrl()) - .setApplicationName(options.getAppName()) - .setGoogleClientRequestInitializer(options.getGoogleApiTrace()) - .build(); - return new PubsubJsonClient(timestampLabel, idLabel, pubsub); - } - - @Override - public String getKind() { - return "Json"; - } - } - - /** - * Factory for creating Pubsub clients using Json transport. - */ - public static final PubsubClientFactory FACTORY = new PubsubJsonClientFactory(); - - /** - * Label to use for custom timestamps, or {@literal null} if should use Pubsub publish time - * instead. - */ - @Nullable - private final String timestampLabel; - - /** - * Label to use for custom ids, or {@literal null} if should use Pubsub provided ids. - */ - @Nullable - private final String idLabel; - - /** - * Underlying JSON transport. - */ - private Pubsub pubsub; - - @VisibleForTesting PubsubJsonClient( - @Nullable String timestampLabel, - @Nullable String idLabel, - Pubsub pubsub) { - this.timestampLabel = timestampLabel; - this.idLabel = idLabel; - this.pubsub = pubsub; - } - - @Override - public void close() { - // Nothing to close. - } - - @Override - public int publish(TopicPath topic, List outgoingMessages) - throws IOException { - List pubsubMessages = new ArrayList<>(outgoingMessages.size()); - for (OutgoingMessage outgoingMessage : outgoingMessages) { - PubsubMessage pubsubMessage = new PubsubMessage().encodeData(outgoingMessage.elementBytes); - - Map attributes = outgoingMessage.attributes; - if ((timestampLabel != null || idLabel != null) && attributes == null) { - attributes = new TreeMap<>(); - } - if (attributes != null) { - pubsubMessage.setAttributes(attributes); - } - - if (timestampLabel != null) { - attributes.put(timestampLabel, String.valueOf(outgoingMessage.timestampMsSinceEpoch)); - } - - if (idLabel != null && !Strings.isNullOrEmpty(outgoingMessage.recordId)) { - attributes.put(idLabel, outgoingMessage.recordId); - } - - pubsubMessages.add(pubsubMessage); - } - PublishRequest request = new PublishRequest().setMessages(pubsubMessages); - PublishResponse response = pubsub.projects() - .topics() - .publish(topic.getPath(), request) - .execute(); - return response.getMessageIds().size(); - } - - @Override - public List pull( - long requestTimeMsSinceEpoch, - SubscriptionPath subscription, - int batchSize, - boolean returnImmediately) throws IOException { - PullRequest request = new PullRequest() - .setReturnImmediately(returnImmediately) - .setMaxMessages(batchSize); - PullResponse response = pubsub.projects() - .subscriptions() - .pull(subscription.getPath(), request) - .execute(); - if (response.getReceivedMessages() == null || response.getReceivedMessages().size() == 0) { - return ImmutableList.of(); - } - List incomingMessages = new ArrayList<>(response.getReceivedMessages().size()); - for (ReceivedMessage message : response.getReceivedMessages()) { - PubsubMessage pubsubMessage = message.getMessage(); - @Nullable Map attributes = pubsubMessage.getAttributes(); - - // Payload. - byte[] elementBytes = pubsubMessage.decodeData(); - - // Timestamp. - long timestampMsSinceEpoch = - extractTimestamp(timestampLabel, message.getMessage().getPublishTime(), attributes); - - // Ack id. - String ackId = message.getAckId(); - checkState(!Strings.isNullOrEmpty(ackId)); - - // Record id, if any. - @Nullable String recordId = null; - if (idLabel != null && attributes != null) { - recordId = attributes.get(idLabel); - } - if (Strings.isNullOrEmpty(recordId)) { - // Fall back to the Pubsub provided message id. - recordId = pubsubMessage.getMessageId(); - } - - incomingMessages.add(new IncomingMessage(elementBytes, attributes, timestampMsSinceEpoch, - requestTimeMsSinceEpoch, ackId, recordId)); - } - - return incomingMessages; - } - - @Override - public void acknowledge(SubscriptionPath subscription, List ackIds) throws IOException { - AcknowledgeRequest request = new AcknowledgeRequest().setAckIds(ackIds); - pubsub.projects() - .subscriptions() - .acknowledge(subscription.getPath(), request) - .execute(); // ignore Empty result. - } - - @Override - public void modifyAckDeadline( - SubscriptionPath subscription, List ackIds, int deadlineSeconds) - throws IOException { - ModifyAckDeadlineRequest request = - new ModifyAckDeadlineRequest().setAckIds(ackIds) - .setAckDeadlineSeconds(deadlineSeconds); - pubsub.projects() - .subscriptions() - .modifyAckDeadline(subscription.getPath(), request) - .execute(); // ignore Empty result. - } - - @Override - public void createTopic(TopicPath topic) throws IOException { - pubsub.projects() - .topics() - .create(topic.getPath(), new Topic()) - .execute(); // ignore Topic result. - } - - @Override - public void deleteTopic(TopicPath topic) throws IOException { - pubsub.projects() - .topics() - .delete(topic.getPath()) - .execute(); // ignore Empty result. - } - - @Override - public List listTopics(ProjectPath project) throws IOException { - ListTopicsResponse response = pubsub.projects() - .topics() - .list(project.getPath()) - .execute(); - if (response.getTopics() == null || response.getTopics().isEmpty()) { - return ImmutableList.of(); - } - List topics = new ArrayList<>(response.getTopics().size()); - for (Topic topic : response.getTopics()) { - topics.add(topicPathFromPath(topic.getName())); - } - return topics; - } - - @Override - public void createSubscription( - TopicPath topic, SubscriptionPath subscription, - int ackDeadlineSeconds) throws IOException { - Subscription request = new Subscription() - .setTopic(topic.getPath()) - .setAckDeadlineSeconds(ackDeadlineSeconds); - pubsub.projects() - .subscriptions() - .create(subscription.getPath(), request) - .execute(); // ignore Subscription result. - } - - @Override - public void deleteSubscription(SubscriptionPath subscription) throws IOException { - pubsub.projects() - .subscriptions() - .delete(subscription.getPath()) - .execute(); // ignore Empty result. - } - - @Override - public List listSubscriptions(ProjectPath project, TopicPath topic) - throws IOException { - ListSubscriptionsResponse response = pubsub.projects() - .subscriptions() - .list(project.getPath()) - .execute(); - if (response.getSubscriptions() == null || response.getSubscriptions().isEmpty()) { - return ImmutableList.of(); - } - List subscriptions = new ArrayList<>(response.getSubscriptions().size()); - for (Subscription subscription : response.getSubscriptions()) { - if (subscription.getTopic().equals(topic.getPath())) { - subscriptions.add(subscriptionPathFromPath(subscription.getName())); - } - } - return subscriptions; - } - - @Override - public int ackDeadlineSeconds(SubscriptionPath subscription) throws IOException { - Subscription response = pubsub.projects().subscriptions().get(subscription.getPath()).execute(); - return response.getAckDeadlineSeconds(); - } - - @Override - public boolean isEOF() { - return false; - } -} diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/io/PubsubTestClient.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/io/PubsubTestClient.java deleted file mode 100644 index 69ba2b0cbcac3..0000000000000 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/io/PubsubTestClient.java +++ /dev/null @@ -1,436 +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.integration.nexmark.io; - -import static com.google.common.base.Preconditions.checkState; - -import com.google.api.client.util.Clock; -import com.google.common.collect.Lists; -import com.google.common.collect.Sets; -import java.io.Closeable; -import java.io.IOException; -import java.io.Serializable; -import java.util.ArrayList; -import java.util.HashMap; -import java.util.Iterator; -import java.util.List; -import java.util.Map; -import java.util.Set; -import javax.annotation.Nullable; -import org.apache.beam.sdk.io.gcp.pubsub.PubsubOptions; - -/** - * A (partial) implementation of {@link PubsubClient} for use by unit tests. Only suitable for - * testing {@link #publish}, {@link #pull}, {@link #acknowledge} and {@link #modifyAckDeadline} - * methods. Relies on statics to mimic the Pubsub service, though we try to hide that. - */ -class PubsubTestClient extends PubsubClient implements Serializable { - /** - * Mimic the state of the simulated Pubsub 'service'. - * - *

              Note that the {@link PubsubTestClientFactory} is serialized/deserialized even when running - * test pipelines. Meanwhile it is valid for multiple {@link PubsubTestClient}s to be created - * from the same client factory and run in parallel. Thus we can't enforce aliasing of the - * following data structures over all clients and must resort to a static. - */ - private static class State { - /** - * True if has been primed for a test but not yet validated. - */ - boolean isActive; - - /** - * Publish mode only: Only publish calls for this topic are allowed. - */ - @Nullable - TopicPath expectedTopic; - - /** - * Publish mode only: Messages yet to seen in a {@link #publish} call. - */ - @Nullable - Set remainingExpectedOutgoingMessages; - - /** - * Publish mode only: Messages which should throw when first sent to simulate transient publish - * failure. - */ - @Nullable - Set remainingFailingOutgoingMessages; - - /** - * Pull mode only: Clock from which to get current time. - */ - @Nullable - Clock clock; - - /** - * Pull mode only: Only pull calls for this subscription are allowed. - */ - @Nullable - SubscriptionPath expectedSubscription; - - /** - * Pull mode only: Timeout to simulate. - */ - int ackTimeoutSec; - - /** - * Pull mode only: Messages waiting to be received by a {@link #pull} call. - */ - @Nullable - List remainingPendingIncomingMessages; - - /** - * Pull mode only: Messages which have been returned from a {@link #pull} call and - * not yet ACKed by an {@link #acknowledge} call. - */ - @Nullable - Map pendingAckIncomingMessages; - - /** - * Pull mode only: When above messages are due to have their ACK deadlines expire. - */ - @Nullable - Map ackDeadline; - } - - private static final State STATE = new State(); - - /** Closing the factory will validate all expected messages were processed. */ - public interface PubsubTestClientFactory - extends PubsubClientFactory, Closeable, Serializable { - } - - /** - * Return a factory for testing publishers. Only one factory may be in-flight at a time. - * The factory must be closed when the test is complete, at which point final validation will - * occur. - */ - static PubsubTestClientFactory createFactoryForPublish( - final TopicPath expectedTopic, - final Iterable expectedOutgoingMessages, - final Iterable failingOutgoingMessages) { - synchronized (STATE) { - checkState(!STATE.isActive, "Test still in flight"); - STATE.expectedTopic = expectedTopic; - STATE.remainingExpectedOutgoingMessages = Sets.newHashSet(expectedOutgoingMessages); - STATE.remainingFailingOutgoingMessages = Sets.newHashSet(failingOutgoingMessages); - STATE.isActive = true; - } - return new PubsubTestClientFactory() { - @Override - public PubsubClient newClient( - @Nullable String timestampLabel, @Nullable String idLabel, PubsubOptions options) - throws IOException { - return new PubsubTestClient(); - } - - @Override - public String getKind() { - return "PublishTest"; - } - - @Override - public void close() { - synchronized (STATE) { - checkState(STATE.isActive, "No test still in flight"); - checkState(STATE.remainingExpectedOutgoingMessages.isEmpty(), - "Still waiting for %s messages to be published", - STATE.remainingExpectedOutgoingMessages.size()); - STATE.isActive = false; - STATE.remainingExpectedOutgoingMessages = null; - } - } - }; - } - - /** - * Return a factory for testing subscribers. Only one factory may be in-flight at a time. - * The factory must be closed when the test in complete - */ - public static PubsubTestClientFactory createFactoryForPull( - final Clock clock, - final SubscriptionPath expectedSubscription, - final int ackTimeoutSec, - final Iterable expectedIncomingMessages) { - synchronized (STATE) { - checkState(!STATE.isActive, "Test still in flight"); - STATE.clock = clock; - STATE.expectedSubscription = expectedSubscription; - STATE.ackTimeoutSec = ackTimeoutSec; - STATE.remainingPendingIncomingMessages = Lists.newArrayList(expectedIncomingMessages); - STATE.pendingAckIncomingMessages = new HashMap<>(); - STATE.ackDeadline = new HashMap<>(); - STATE.isActive = true; - } - return new PubsubTestClientFactory() { - @Override - public PubsubClient newClient( - @Nullable String timestampLabel, @Nullable String idLabel, PubsubOptions options) - throws IOException { - return new PubsubTestClient(); - } - - @Override - public String getKind() { - return "PullTest"; - } - - @Override - public void close() { - synchronized (STATE) { - checkState(STATE.isActive, "No test still in flight"); - checkState(STATE.remainingPendingIncomingMessages.isEmpty(), - "Still waiting for %s messages to be pulled", - STATE.remainingPendingIncomingMessages.size()); - checkState(STATE.pendingAckIncomingMessages.isEmpty(), - "Still waiting for %s messages to be ACKed", - STATE.pendingAckIncomingMessages.size()); - checkState(STATE.ackDeadline.isEmpty(), - "Still waiting for %s messages to be ACKed", - STATE.ackDeadline.size()); - STATE.isActive = false; - STATE.remainingPendingIncomingMessages = null; - STATE.pendingAckIncomingMessages = null; - STATE.ackDeadline = null; - } - } - }; - } - - public static PubsubTestClientFactory createFactoryForCreateSubscription() { - return new PubsubTestClientFactory() { - int numCalls = 0; - - @Override - public void close() throws IOException { - checkState( - numCalls == 1, "Expected exactly one subscription to be created, got %s", numCalls); - } - - @Override - public PubsubClient newClient( - @Nullable String timestampLabel, @Nullable String idLabel, PubsubOptions options) - throws IOException { - return new PubsubTestClient() { - @Override - public void createSubscription( - TopicPath topic, SubscriptionPath subscription, int ackDeadlineSeconds) - throws IOException { - checkState(numCalls == 0, "Expected at most one subscription to be created"); - numCalls++; - } - }; - } - - @Override - public String getKind() { - return "CreateSubscriptionTest"; - } - }; - } - - /** - * Return true if in pull mode. - */ - private boolean inPullMode() { - checkState(STATE.isActive, "No test is active"); - return STATE.expectedSubscription != null; - } - - /** - * Return true if in publish mode. - */ - private boolean inPublishMode() { - checkState(STATE.isActive, "No test is active"); - return STATE.expectedTopic != null; - } - - /** - * For subscription mode only: - * Track progression of time according to the {@link Clock} passed . This will simulate Pubsub - * expiring - * outstanding ACKs. - */ - public void advance() { - synchronized (STATE) { - checkState(inPullMode(), "Can only advance in pull mode"); - // Any messages who's ACKs timed out are available for re-pulling. - Iterator> deadlineItr = STATE.ackDeadline.entrySet().iterator(); - while (deadlineItr.hasNext()) { - Map.Entry entry = deadlineItr.next(); - if (entry.getValue() <= STATE.clock.currentTimeMillis()) { - STATE.remainingPendingIncomingMessages.add( - STATE.pendingAckIncomingMessages.remove(entry.getKey())); - deadlineItr.remove(); - } - } - } - } - - @Override - public void close() { - } - - @Override - public int publish( - TopicPath topic, List outgoingMessages) throws IOException { - synchronized (STATE) { - checkState(inPublishMode(), "Can only publish in publish mode"); - checkState(topic.equals(STATE.expectedTopic), "Topic %s does not match expected %s", topic, - STATE.expectedTopic); - for (OutgoingMessage outgoingMessage : outgoingMessages) { - if (STATE.remainingFailingOutgoingMessages.remove(outgoingMessage)) { - throw new RuntimeException("Simulating failure for " + outgoingMessage); - } - checkState(STATE.remainingExpectedOutgoingMessages.remove(outgoingMessage), - "Unexpected outgoing message %s", outgoingMessage); - } - return outgoingMessages.size(); - } - } - - @Override - public List pull( - long requestTimeMsSinceEpoch, SubscriptionPath subscription, int batchSize, - boolean returnImmediately) throws IOException { - synchronized (STATE) { - checkState(inPullMode(), "Can only pull in pull mode"); - long now = STATE.clock.currentTimeMillis(); - checkState(requestTimeMsSinceEpoch == now, - "Simulated time %s does not match request time %s", now, requestTimeMsSinceEpoch); - checkState(subscription.equals(STATE.expectedSubscription), - "Subscription %s does not match expected %s", subscription, - STATE.expectedSubscription); - checkState(returnImmediately, "Pull only supported if returning immediately"); - - List incomingMessages = new ArrayList<>(); - Iterator pendItr = STATE.remainingPendingIncomingMessages.iterator(); - while (pendItr.hasNext()) { - IncomingMessage incomingMessage = pendItr.next(); - pendItr.remove(); - IncomingMessage incomingMessageWithRequestTime = - incomingMessage.withRequestTime(requestTimeMsSinceEpoch); - incomingMessages.add(incomingMessageWithRequestTime); - STATE.pendingAckIncomingMessages.put(incomingMessageWithRequestTime.ackId, - incomingMessageWithRequestTime); - STATE.ackDeadline.put(incomingMessageWithRequestTime.ackId, - requestTimeMsSinceEpoch + STATE.ackTimeoutSec * 1000); - if (incomingMessages.size() >= batchSize) { - break; - } - } - return incomingMessages; - } - } - - @Override - public void acknowledge( - SubscriptionPath subscription, - List ackIds) throws IOException { - synchronized (STATE) { - checkState(inPullMode(), "Can only acknowledge in pull mode"); - checkState(subscription.equals(STATE.expectedSubscription), - "Subscription %s does not match expected %s", subscription, - STATE.expectedSubscription); - - for (String ackId : ackIds) { - checkState(STATE.ackDeadline.remove(ackId) != null, - "No message with ACK id %s is waiting for an ACK", ackId); - checkState(STATE.pendingAckIncomingMessages.remove(ackId) != null, - "No message with ACK id %s is waiting for an ACK", ackId); - } - } - } - - @Override - public void modifyAckDeadline( - SubscriptionPath subscription, List ackIds, int deadlineSeconds) throws IOException { - synchronized (STATE) { - checkState(inPullMode(), "Can only modify ack deadline in pull mode"); - checkState(subscription.equals(STATE.expectedSubscription), - "Subscription %s does not match expected %s", subscription, - STATE.expectedSubscription); - - for (String ackId : ackIds) { - if (deadlineSeconds > 0) { - checkState(STATE.ackDeadline.remove(ackId) != null, - "No message with ACK id %s is waiting for an ACK", ackId); - checkState(STATE.pendingAckIncomingMessages.containsKey(ackId), - "No message with ACK id %s is waiting for an ACK", ackId); - STATE.ackDeadline.put(ackId, STATE.clock.currentTimeMillis() + deadlineSeconds * 1000); - } else { - checkState(STATE.ackDeadline.remove(ackId) != null, - "No message with ACK id %s is waiting for an ACK", ackId); - IncomingMessage message = STATE.pendingAckIncomingMessages.remove(ackId); - checkState(message != null, "No message with ACK id %s is waiting for an ACK", ackId); - STATE.remainingPendingIncomingMessages.add(message); - } - } - } - } - - @Override - public void createTopic(TopicPath topic) throws IOException { - throw new UnsupportedOperationException(); - } - - @Override - public void deleteTopic(TopicPath topic) throws IOException { - throw new UnsupportedOperationException(); - } - - @Override - public List listTopics(ProjectPath project) throws IOException { - throw new UnsupportedOperationException(); - } - - @Override - public void createSubscription( - TopicPath topic, SubscriptionPath subscription, int ackDeadlineSeconds) throws IOException { - throw new UnsupportedOperationException(); - } - - @Override - public void deleteSubscription(SubscriptionPath subscription) throws IOException { - throw new UnsupportedOperationException(); - } - - @Override - public List listSubscriptions( - ProjectPath project, TopicPath topic) throws IOException { - throw new UnsupportedOperationException(); - } - - @Override - public int ackDeadlineSeconds(SubscriptionPath subscription) throws IOException { - synchronized (STATE) { - return STATE.ackTimeoutSec; - } - } - - @Override - public boolean isEOF() { - synchronized (STATE) { - checkState(inPullMode(), "Can only check EOF in pull mode"); - return STATE.remainingPendingIncomingMessages.isEmpty(); - } - } -} diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/io/package-info.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/io/package-info.java deleted file mode 100644 index 1161f3e399e64..0000000000000 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/io/package-info.java +++ /dev/null @@ -1,22 +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. - */ - -/** - * Nexmark Beam IO related utilities. - */ -package org.apache.beam.integration.nexmark.io; diff --git a/integration/java/nexmark/src/test/java/org/apache/beam/integration/nexmark/sources/BoundedEventSourceTest.java b/integration/java/nexmark/src/test/java/org/apache/beam/integration/nexmark/sources/BoundedEventSourceTest.java index c5d77253f6ba8..d95461a09ed6e 100644 --- a/integration/java/nexmark/src/test/java/org/apache/beam/integration/nexmark/sources/BoundedEventSourceTest.java +++ b/integration/java/nexmark/src/test/java/org/apache/beam/integration/nexmark/sources/BoundedEventSourceTest.java @@ -18,7 +18,7 @@ package org.apache.beam.integration.nexmark.sources; import org.apache.beam.integration.nexmark.NexmarkConfiguration; -import org.apache.beam.runners.dataflow.options.DataflowPipelineOptions; +import org.apache.beam.integration.nexmark.NexmarkOptions; import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.testing.SourceTestUtils; @@ -38,7 +38,7 @@ private GeneratorConfig makeConfig(long n) { @Test public void sourceAndReadersWork() throws Exception { - DataflowPipelineOptions options = PipelineOptionsFactory.as(DataflowPipelineOptions.class); + NexmarkOptions options = PipelineOptionsFactory.as(NexmarkOptions.class); long n = 200L; BoundedEventSource source = new BoundedEventSource(makeConfig(n), 1); @@ -48,7 +48,7 @@ public void sourceAndReadersWork() throws Exception { @Test public void splitAtFractionRespectsContract() throws Exception { - DataflowPipelineOptions options = PipelineOptionsFactory.as(DataflowPipelineOptions.class); + NexmarkOptions options = PipelineOptionsFactory.as(NexmarkOptions.class); long n = 20L; BoundedEventSource source = new BoundedEventSource(makeConfig(n), 1); @@ -62,7 +62,7 @@ public void splitAtFractionRespectsContract() throws Exception { @Test public void splitIntoBundlesRespectsContract() throws Exception { - DataflowPipelineOptions options = PipelineOptionsFactory.as(DataflowPipelineOptions.class); + NexmarkOptions options = PipelineOptionsFactory.as(NexmarkOptions.class); long n = 200L; BoundedEventSource source = new BoundedEventSource(makeConfig(n), 1); SourceTestUtils.assertSourcesEqualReferenceSource( From a39cb80009f569e1c8ba82ee9c67a7c5dbe3d16f Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Isma=C3=ABl=20Mej=C3=ADa?= Date: Sun, 30 Apr 2017 17:44:07 +0200 Subject: [PATCH 304/346] Move WinningBids into the queries package --- .../beam/integration/nexmark/NexmarkRunner.java | 2 ++ .../integration/nexmark/model/AuctionBid.java | 3 +-- .../nexmark/{ => queries}/AbstractSimulator.java | 3 ++- .../nexmark/{ => queries}/NexmarkQuery.java | 15 +++++++++------ .../nexmark/{ => queries}/NexmarkQueryModel.java | 7 ++++--- .../beam/integration/nexmark/queries/Query0.java | 1 - .../integration/nexmark/queries/Query0Model.java | 4 +--- .../beam/integration/nexmark/queries/Query1.java | 1 - .../beam/integration/nexmark/queries/Query10.java | 1 - .../beam/integration/nexmark/queries/Query11.java | 1 - .../beam/integration/nexmark/queries/Query12.java | 1 - .../integration/nexmark/queries/Query1Model.java | 2 -- .../beam/integration/nexmark/queries/Query2.java | 1 - .../integration/nexmark/queries/Query2Model.java | 2 -- .../beam/integration/nexmark/queries/Query3.java | 1 - .../integration/nexmark/queries/Query3Model.java | 2 -- .../beam/integration/nexmark/queries/Query4.java | 2 -- .../integration/nexmark/queries/Query4Model.java | 3 --- .../beam/integration/nexmark/queries/Query5.java | 1 - .../integration/nexmark/queries/Query5Model.java | 2 -- .../beam/integration/nexmark/queries/Query6.java | 2 -- .../integration/nexmark/queries/Query6Model.java | 3 --- .../beam/integration/nexmark/queries/Query7.java | 1 - .../integration/nexmark/queries/Query7Model.java | 2 -- .../beam/integration/nexmark/queries/Query8.java | 1 - .../integration/nexmark/queries/Query8Model.java | 2 -- .../beam/integration/nexmark/queries/Query9.java | 2 -- .../integration/nexmark/queries/Query9Model.java | 3 --- .../nexmark/{ => queries}/WinningBids.java | 4 +++- .../{ => queries}/WinningBidsSimulator.java | 4 +++- .../integration/nexmark/queries/QueryTest.java | 2 -- 31 files changed, 25 insertions(+), 56 deletions(-) rename integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/{ => queries}/AbstractSimulator.java (98%) rename integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/{ => queries}/NexmarkQuery.java (93%) rename integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/{ => queries}/NexmarkQueryModel.java (95%) rename integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/{ => queries}/WinningBids.java (98%) rename integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/{ => queries}/WinningBidsSimulator.java (97%) diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkRunner.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkRunner.java index ebfd196d814ce..a3c4d338766f9 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkRunner.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkRunner.java @@ -38,6 +38,8 @@ import org.apache.beam.integration.nexmark.model.Event; import org.apache.beam.integration.nexmark.model.KnownSize; import org.apache.beam.integration.nexmark.model.Person; +import org.apache.beam.integration.nexmark.queries.NexmarkQuery; +import org.apache.beam.integration.nexmark.queries.NexmarkQueryModel; import org.apache.beam.integration.nexmark.queries.Query0; import org.apache.beam.integration.nexmark.queries.Query0Model; import org.apache.beam.integration.nexmark.queries.Query1; diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/AuctionBid.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/AuctionBid.java index 7f6b7c9272178..b1d9ec2c5055d 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/AuctionBid.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/AuctionBid.java @@ -24,13 +24,12 @@ import java.io.OutputStream; import java.io.Serializable; import org.apache.beam.integration.nexmark.NexmarkUtils; -import org.apache.beam.integration.nexmark.WinningBids; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.CoderException; import org.apache.beam.sdk.coders.CustomCoder; /** - * Result of {@link WinningBids} transform. + * Result of {@link org.apache.beam.integration.nexmark.queries.WinningBids} transform. */ public class AuctionBid implements KnownSize, Serializable { public static final Coder CODER = new CustomCoder() { diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/AbstractSimulator.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/AbstractSimulator.java similarity index 98% rename from integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/AbstractSimulator.java rename to integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/AbstractSimulator.java index b01284267abcb..270b5c3c9f017 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/AbstractSimulator.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/AbstractSimulator.java @@ -16,13 +16,14 @@ * limitations under the License. */ -package org.apache.beam.integration.nexmark; +package org.apache.beam.integration.nexmark.queries; import java.util.ArrayList; import java.util.Iterator; import java.util.List; import javax.annotation.Nullable; +import org.apache.beam.integration.nexmark.NexmarkUtils; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.values.TimestampedValue; import org.joda.time.Duration; diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkQuery.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/NexmarkQuery.java similarity index 93% rename from integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkQuery.java rename to integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/NexmarkQuery.java index ab1c3052d2b06..0796ce5677a3a 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkQuery.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/NexmarkQuery.java @@ -15,8 +15,11 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.integration.nexmark; +package org.apache.beam.integration.nexmark.queries; +import org.apache.beam.integration.nexmark.Monitor; +import org.apache.beam.integration.nexmark.NexmarkConfiguration; +import org.apache.beam.integration.nexmark.NexmarkUtils; import org.apache.beam.integration.nexmark.model.Auction; import org.apache.beam.integration.nexmark.model.Bid; import org.apache.beam.integration.nexmark.model.Event; @@ -41,8 +44,8 @@ */ public abstract class NexmarkQuery extends PTransform, PCollection>> { - protected static final TupleTag AUCTION_TAG = new TupleTag<>("auctions"); - protected static final TupleTag BID_TAG = new TupleTag<>("bids"); + public static final TupleTag AUCTION_TAG = new TupleTag<>("auctions"); + public static final TupleTag BID_TAG = new TupleTag<>("bids"); protected static final TupleTag PERSON_TAG = new TupleTag<>("person"); /** Predicate to detect a new person event. */ @@ -169,7 +172,7 @@ public void processElement(ProcessContext c) { /** * Transform to filter for just the new auction events. */ - protected static final PTransform, PCollection> JUST_NEW_AUCTIONS = + public static final PTransform, PCollection> JUST_NEW_AUCTIONS = new PTransform, PCollection>("justNewAuctions") { @Override public PCollection expand(PCollection input) { @@ -181,7 +184,7 @@ public PCollection expand(PCollection input) { /** * Transform to filter for just the new person events. */ - protected static final PTransform, PCollection> JUST_NEW_PERSONS = + public static final PTransform, PCollection> JUST_NEW_PERSONS = new PTransform, PCollection>("justNewPersons") { @Override public PCollection expand(PCollection input) { @@ -193,7 +196,7 @@ public PCollection expand(PCollection input) { /** * Transform to filter for just the bid events. */ - protected static final PTransform, PCollection> JUST_BIDS = + public static final PTransform, PCollection> JUST_BIDS = new PTransform, PCollection>("justBids") { @Override public PCollection expand(PCollection input) { diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkQueryModel.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/NexmarkQueryModel.java similarity index 95% rename from integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkQueryModel.java rename to integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/NexmarkQueryModel.java index b2b1826a8106c..1ad909988f021 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkQueryModel.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/NexmarkQueryModel.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.integration.nexmark; +package org.apache.beam.integration.nexmark.queries; import java.io.Serializable; import java.util.ArrayList; @@ -25,6 +25,7 @@ import java.util.List; import java.util.Set; +import org.apache.beam.integration.nexmark.NexmarkConfiguration; import org.apache.beam.integration.nexmark.model.KnownSize; import org.apache.beam.sdk.transforms.SerializableFunction; import org.apache.beam.sdk.values.TimestampedValue; @@ -40,7 +41,7 @@ * applied against the actual query results to check their consistency with the model. */ public abstract class NexmarkQueryModel implements Serializable { - protected final NexmarkConfiguration configuration; + public final NexmarkConfiguration configuration; public NexmarkQueryModel(NexmarkConfiguration configuration) { this.configuration = configuration; @@ -86,7 +87,7 @@ protected static Set toValue(Iterator> itr) { } /** Return simulator for query. */ - protected abstract AbstractSimulator simulator(); + public abstract AbstractSimulator simulator(); /** Return sub-sequence of results which are significant for model. */ protected Iterable> relevantResults( diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query0.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query0.java index 84696c49f9777..00a49a8865023 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query0.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query0.java @@ -22,7 +22,6 @@ import java.io.IOException; import org.apache.beam.integration.nexmark.NexmarkConfiguration; -import org.apache.beam.integration.nexmark.NexmarkQuery; import org.apache.beam.integration.nexmark.NexmarkUtils; import org.apache.beam.integration.nexmark.model.Event; import org.apache.beam.integration.nexmark.model.KnownSize; diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query0Model.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query0Model.java index 991b1d42ee862..6fb6613735d34 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query0Model.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query0Model.java @@ -20,9 +20,7 @@ import java.util.Collection; import java.util.Iterator; -import org.apache.beam.integration.nexmark.AbstractSimulator; import org.apache.beam.integration.nexmark.NexmarkConfiguration; -import org.apache.beam.integration.nexmark.NexmarkQueryModel; import org.apache.beam.integration.nexmark.NexmarkUtils; import org.apache.beam.integration.nexmark.model.Event; import org.apache.beam.sdk.values.TimestampedValue; @@ -56,7 +54,7 @@ public Query0Model(NexmarkConfiguration configuration) { } @Override - protected AbstractSimulator simulator() { + public AbstractSimulator simulator() { return new Simulator(configuration); } diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query1.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query1.java index 0be77ce2d1426..8d90b701a9090 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query1.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query1.java @@ -18,7 +18,6 @@ package org.apache.beam.integration.nexmark.queries; import org.apache.beam.integration.nexmark.NexmarkConfiguration; -import org.apache.beam.integration.nexmark.NexmarkQuery; import org.apache.beam.integration.nexmark.NexmarkUtils; import org.apache.beam.integration.nexmark.model.Bid; import org.apache.beam.integration.nexmark.model.Event; diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query10.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query10.java index d9b3557af6359..c919691e7aba8 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query10.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query10.java @@ -28,7 +28,6 @@ import java.util.concurrent.ThreadLocalRandom; import javax.annotation.Nullable; import org.apache.beam.integration.nexmark.NexmarkConfiguration; -import org.apache.beam.integration.nexmark.NexmarkQuery; import org.apache.beam.integration.nexmark.NexmarkUtils; import org.apache.beam.integration.nexmark.model.Done; import org.apache.beam.integration.nexmark.model.Event; diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query11.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query11.java index a8a61aebec1a5..fd936a9d4e097 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query11.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query11.java @@ -18,7 +18,6 @@ package org.apache.beam.integration.nexmark.queries; import org.apache.beam.integration.nexmark.NexmarkConfiguration; -import org.apache.beam.integration.nexmark.NexmarkQuery; import org.apache.beam.integration.nexmark.NexmarkUtils; import org.apache.beam.integration.nexmark.model.Bid; import org.apache.beam.integration.nexmark.model.BidsPerSession; diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query12.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query12.java index a5db5047b5e63..20f45fb1d60a1 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query12.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query12.java @@ -18,7 +18,6 @@ package org.apache.beam.integration.nexmark.queries; import org.apache.beam.integration.nexmark.NexmarkConfiguration; -import org.apache.beam.integration.nexmark.NexmarkQuery; import org.apache.beam.integration.nexmark.NexmarkUtils; import org.apache.beam.integration.nexmark.model.Bid; import org.apache.beam.integration.nexmark.model.BidsPerSession; diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query1Model.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query1Model.java index 58037d33a49b2..03886874c78c8 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query1Model.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query1Model.java @@ -21,9 +21,7 @@ import java.util.Collection; import java.util.Iterator; -import org.apache.beam.integration.nexmark.AbstractSimulator; import org.apache.beam.integration.nexmark.NexmarkConfiguration; -import org.apache.beam.integration.nexmark.NexmarkQueryModel; import org.apache.beam.integration.nexmark.NexmarkUtils; import org.apache.beam.integration.nexmark.model.Bid; import org.apache.beam.integration.nexmark.model.Event; diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query2.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query2.java index 4c8f878c30648..a365b973b7b8c 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query2.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query2.java @@ -18,7 +18,6 @@ package org.apache.beam.integration.nexmark.queries; import org.apache.beam.integration.nexmark.NexmarkConfiguration; -import org.apache.beam.integration.nexmark.NexmarkQuery; import org.apache.beam.integration.nexmark.NexmarkUtils; import org.apache.beam.integration.nexmark.model.AuctionPrice; import org.apache.beam.integration.nexmark.model.Bid; diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query2Model.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query2Model.java index f578e4c571178..e00992f39d5bd 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query2Model.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query2Model.java @@ -21,9 +21,7 @@ import java.util.Collection; import java.util.Iterator; -import org.apache.beam.integration.nexmark.AbstractSimulator; import org.apache.beam.integration.nexmark.NexmarkConfiguration; -import org.apache.beam.integration.nexmark.NexmarkQueryModel; import org.apache.beam.integration.nexmark.NexmarkUtils; import org.apache.beam.integration.nexmark.model.AuctionPrice; import org.apache.beam.integration.nexmark.model.Bid; diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query3.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query3.java index 12b16f1ee8c56..71364ba6285ff 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query3.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query3.java @@ -21,7 +21,6 @@ import java.util.ArrayList; import java.util.List; import org.apache.beam.integration.nexmark.NexmarkConfiguration; -import org.apache.beam.integration.nexmark.NexmarkQuery; import org.apache.beam.integration.nexmark.NexmarkUtils; import org.apache.beam.integration.nexmark.model.Auction; import org.apache.beam.integration.nexmark.model.Event; diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query3Model.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query3Model.java index e4b72d277ff03..6b98e2a449fa7 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query3Model.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query3Model.java @@ -26,9 +26,7 @@ import java.util.Iterator; import java.util.Map; -import org.apache.beam.integration.nexmark.AbstractSimulator; import org.apache.beam.integration.nexmark.NexmarkConfiguration; -import org.apache.beam.integration.nexmark.NexmarkQueryModel; import org.apache.beam.integration.nexmark.NexmarkUtils; import org.apache.beam.integration.nexmark.model.Auction; import org.apache.beam.integration.nexmark.model.Event; diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query4.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query4.java index 61991c87f40b0..9c0fe6d0609a7 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query4.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query4.java @@ -19,9 +19,7 @@ import org.apache.beam.integration.nexmark.Monitor; import org.apache.beam.integration.nexmark.NexmarkConfiguration; -import org.apache.beam.integration.nexmark.NexmarkQuery; import org.apache.beam.integration.nexmark.NexmarkUtils; -import org.apache.beam.integration.nexmark.WinningBids; import org.apache.beam.integration.nexmark.model.Auction; import org.apache.beam.integration.nexmark.model.AuctionBid; import org.apache.beam.integration.nexmark.model.Bid; diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query4Model.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query4Model.java index 9405ac8e40e61..634a58e44f034 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query4Model.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query4Model.java @@ -24,11 +24,8 @@ import java.util.List; import java.util.Map; import java.util.TreeMap; -import org.apache.beam.integration.nexmark.AbstractSimulator; import org.apache.beam.integration.nexmark.NexmarkConfiguration; -import org.apache.beam.integration.nexmark.NexmarkQueryModel; import org.apache.beam.integration.nexmark.NexmarkUtils; -import org.apache.beam.integration.nexmark.WinningBidsSimulator; import org.apache.beam.integration.nexmark.model.Auction; import org.apache.beam.integration.nexmark.model.AuctionBid; import org.apache.beam.integration.nexmark.model.Bid; diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query5.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query5.java index 34b7b50e40660..18ce5789eaf4c 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query5.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query5.java @@ -21,7 +21,6 @@ import java.util.Arrays; import java.util.List; import org.apache.beam.integration.nexmark.NexmarkConfiguration; -import org.apache.beam.integration.nexmark.NexmarkQuery; import org.apache.beam.integration.nexmark.NexmarkUtils; import org.apache.beam.integration.nexmark.model.AuctionCount; import org.apache.beam.integration.nexmark.model.Bid; diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query5Model.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query5Model.java index 6bf65dc2d5fa4..24d9a00339593 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query5Model.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query5Model.java @@ -24,9 +24,7 @@ import java.util.List; import java.util.Map; import java.util.TreeMap; -import org.apache.beam.integration.nexmark.AbstractSimulator; import org.apache.beam.integration.nexmark.NexmarkConfiguration; -import org.apache.beam.integration.nexmark.NexmarkQueryModel; import org.apache.beam.integration.nexmark.NexmarkUtils; import org.apache.beam.integration.nexmark.model.AuctionCount; import org.apache.beam.integration.nexmark.model.Bid; diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query6.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query6.java index 2a5ab7029bfdb..65789abd132a9 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query6.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query6.java @@ -22,9 +22,7 @@ import java.util.Collections; import java.util.List; import org.apache.beam.integration.nexmark.NexmarkConfiguration; -import org.apache.beam.integration.nexmark.NexmarkQuery; import org.apache.beam.integration.nexmark.NexmarkUtils; -import org.apache.beam.integration.nexmark.WinningBids; import org.apache.beam.integration.nexmark.model.Auction; import org.apache.beam.integration.nexmark.model.AuctionBid; import org.apache.beam.integration.nexmark.model.Bid; diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query6Model.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query6Model.java index 432533702199e..0691714a7158c 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query6Model.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query6Model.java @@ -22,11 +22,8 @@ import java.util.Iterator; import java.util.Map; import java.util.TreeMap; -import org.apache.beam.integration.nexmark.AbstractSimulator; import org.apache.beam.integration.nexmark.NexmarkConfiguration; -import org.apache.beam.integration.nexmark.NexmarkQueryModel; import org.apache.beam.integration.nexmark.NexmarkUtils; -import org.apache.beam.integration.nexmark.WinningBidsSimulator; import org.apache.beam.integration.nexmark.model.Auction; import org.apache.beam.integration.nexmark.model.AuctionBid; import org.apache.beam.integration.nexmark.model.Bid; diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query7.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query7.java index f3d1ba4f3a2e4..2a94ca9950f58 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query7.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query7.java @@ -18,7 +18,6 @@ package org.apache.beam.integration.nexmark.queries; import org.apache.beam.integration.nexmark.NexmarkConfiguration; -import org.apache.beam.integration.nexmark.NexmarkQuery; import org.apache.beam.integration.nexmark.NexmarkUtils; import org.apache.beam.integration.nexmark.model.Bid; import org.apache.beam.integration.nexmark.model.Event; diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query7Model.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query7Model.java index 0a80e590d7991..5c039f9580b96 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query7Model.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query7Model.java @@ -23,9 +23,7 @@ import java.util.Iterator; import java.util.List; -import org.apache.beam.integration.nexmark.AbstractSimulator; import org.apache.beam.integration.nexmark.NexmarkConfiguration; -import org.apache.beam.integration.nexmark.NexmarkQueryModel; import org.apache.beam.integration.nexmark.NexmarkUtils; import org.apache.beam.integration.nexmark.model.Bid; import org.apache.beam.integration.nexmark.model.Event; diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query8.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query8.java index e7daccdea947e..603841be6d7df 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query8.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query8.java @@ -18,7 +18,6 @@ package org.apache.beam.integration.nexmark.queries; import org.apache.beam.integration.nexmark.NexmarkConfiguration; -import org.apache.beam.integration.nexmark.NexmarkQuery; import org.apache.beam.integration.nexmark.NexmarkUtils; import org.apache.beam.integration.nexmark.model.Auction; import org.apache.beam.integration.nexmark.model.Event; diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query8Model.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query8Model.java index 11619942990fb..8c76bc662ced4 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query8Model.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query8Model.java @@ -24,9 +24,7 @@ import java.util.HashMap; import java.util.Iterator; import java.util.Map; -import org.apache.beam.integration.nexmark.AbstractSimulator; import org.apache.beam.integration.nexmark.NexmarkConfiguration; -import org.apache.beam.integration.nexmark.NexmarkQueryModel; import org.apache.beam.integration.nexmark.NexmarkUtils; import org.apache.beam.integration.nexmark.model.Auction; import org.apache.beam.integration.nexmark.model.Event; diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query9.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query9.java index aed827b8a1853..6dd189d11fac5 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query9.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query9.java @@ -18,9 +18,7 @@ package org.apache.beam.integration.nexmark.queries; import org.apache.beam.integration.nexmark.NexmarkConfiguration; -import org.apache.beam.integration.nexmark.NexmarkQuery; import org.apache.beam.integration.nexmark.NexmarkUtils; -import org.apache.beam.integration.nexmark.WinningBids; import org.apache.beam.integration.nexmark.model.AuctionBid; import org.apache.beam.integration.nexmark.model.Event; import org.apache.beam.integration.nexmark.model.KnownSize; diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query9Model.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query9Model.java index b88d60a829b0e..d117e2dfabb6e 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query9Model.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query9Model.java @@ -21,10 +21,7 @@ import java.util.Collection; import java.util.Iterator; -import org.apache.beam.integration.nexmark.AbstractSimulator; import org.apache.beam.integration.nexmark.NexmarkConfiguration; -import org.apache.beam.integration.nexmark.NexmarkQueryModel; -import org.apache.beam.integration.nexmark.WinningBidsSimulator; import org.apache.beam.sdk.values.TimestampedValue; /** diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/WinningBids.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/WinningBids.java similarity index 98% rename from integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/WinningBids.java rename to integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/WinningBids.java index 3815b9d39cfc1..11a4d38c91872 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/WinningBids.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/WinningBids.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.integration.nexmark; +package org.apache.beam.integration.nexmark.queries; import static com.google.common.base.Preconditions.checkState; @@ -30,6 +30,8 @@ import java.util.List; import java.util.Map; import java.util.TreeMap; +import org.apache.beam.integration.nexmark.NexmarkConfiguration; +import org.apache.beam.integration.nexmark.NexmarkUtils; import org.apache.beam.integration.nexmark.model.Auction; import org.apache.beam.integration.nexmark.model.AuctionBid; import org.apache.beam.integration.nexmark.model.Bid; diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/WinningBidsSimulator.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/WinningBidsSimulator.java similarity index 97% rename from integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/WinningBidsSimulator.java rename to integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/WinningBidsSimulator.java index e7f51b776590f..7d74f8f8e2a07 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/WinningBidsSimulator.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/WinningBidsSimulator.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.integration.nexmark; +package org.apache.beam.integration.nexmark.queries; import java.util.ArrayList; import java.util.Iterator; @@ -26,6 +26,8 @@ import java.util.TreeSet; import javax.annotation.Nullable; +import org.apache.beam.integration.nexmark.NexmarkConfiguration; +import org.apache.beam.integration.nexmark.NexmarkUtils; import org.apache.beam.integration.nexmark.model.Auction; import org.apache.beam.integration.nexmark.model.AuctionBid; import org.apache.beam.integration.nexmark.model.Bid; diff --git a/integration/java/nexmark/src/test/java/org/apache/beam/integration/nexmark/queries/QueryTest.java b/integration/java/nexmark/src/test/java/org/apache/beam/integration/nexmark/queries/QueryTest.java index 284aa7e4f2c9b..b005d652b9ab8 100644 --- a/integration/java/nexmark/src/test/java/org/apache/beam/integration/nexmark/queries/QueryTest.java +++ b/integration/java/nexmark/src/test/java/org/apache/beam/integration/nexmark/queries/QueryTest.java @@ -18,8 +18,6 @@ package org.apache.beam.integration.nexmark.queries; import org.apache.beam.integration.nexmark.NexmarkConfiguration; -import org.apache.beam.integration.nexmark.NexmarkQuery; -import org.apache.beam.integration.nexmark.NexmarkQueryModel; import org.apache.beam.integration.nexmark.NexmarkUtils; import org.apache.beam.integration.nexmark.model.KnownSize; import org.apache.beam.sdk.PipelineResult; From 1541fad077e47df1d47636fd186a72aa827bbc42 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Isma=C3=ABl=20Mej=C3=ADa?= Date: Mon, 1 May 2017 00:54:08 +0200 Subject: [PATCH 305/346] Fix static analysis issues Restrict access level on classes + other static analysis fixes Fix findbugs issues (issue #33) Fix compile after AvroIO, TextIO, PubsubIO and State refactor --- integration/java/nexmark/pom.xml | 2 +- .../beam/integration/nexmark/Monitor.java | 4 +- .../integration/nexmark/NexmarkDriver.java | 12 +- .../integration/nexmark/NexmarkRunner.java | 124 ++++++++++-------- .../integration/nexmark/NexmarkUtils.java | 34 +++-- .../integration/nexmark/model/Auction.java | 8 +- .../nexmark/model/AuctionCount.java | 6 +- .../nexmark/model/AuctionPrice.java | 4 +- .../nexmark/model/BidsPerSession.java | 4 +- .../beam/integration/nexmark/model/Done.java | 2 +- .../beam/integration/nexmark/model/Event.java | 13 -- .../nexmark/model/IdNameReserve.java | 6 +- .../nexmark/model/NameCityStateId.java | 8 +- .../integration/nexmark/model/Person.java | 6 +- .../nexmark/model/SellerPrice.java | 2 +- .../nexmark/queries/AbstractSimulator.java | 10 +- .../nexmark/queries/NexmarkQuery.java | 34 ++--- .../nexmark/queries/NexmarkQueryModel.java | 17 +-- .../nexmark/queries/Query0Model.java | 2 +- .../integration/nexmark/queries/Query10.java | 6 +- .../integration/nexmark/queries/Query11.java | 3 +- .../nexmark/queries/Query1Model.java | 2 +- .../integration/nexmark/queries/Query3.java | 24 ++-- .../nexmark/queries/Query3Model.java | 2 +- .../nexmark/queries/Query4Model.java | 5 +- .../integration/nexmark/queries/Query5.java | 4 +- .../integration/nexmark/queries/Query6.java | 4 +- .../nexmark/queries/Query6Model.java | 5 +- .../nexmark/queries/WinningBids.java | 30 ++--- .../nexmark/sources/Generator.java | 11 +- .../nexmark/sources/GeneratorConfig.java | 26 ++-- .../nexmark/sources/UnboundedEventSource.java | 2 +- .../sources/UnboundedEventSourceTest.java | 5 +- integration/pom.xml | 14 ++ 34 files changed, 221 insertions(+), 220 deletions(-) diff --git a/integration/java/nexmark/pom.xml b/integration/java/nexmark/pom.xml index fb213e9c9b809..8a65c0fa7446f 100644 --- a/integration/java/nexmark/pom.xml +++ b/integration/java/nexmark/pom.xml @@ -210,7 +210,7 @@ org.apache.beam - beam-sdks-java-extensions-gcp-core + beam-sdks-java-extensions-google-cloud-platform-core diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Monitor.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Monitor.java index cb4d71c957c83..2f0c56a8e34b2 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Monitor.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Monitor.java @@ -63,8 +63,8 @@ public void processElement(ProcessContext c) { public final String name; public final String prefix; - final MonitorDoFn doFn; - final PTransform, PCollection> transform; + private final MonitorDoFn doFn; + private final PTransform, PCollection> transform; public Monitor(String name, String prefix) { this.name = name; diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkDriver.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkDriver.java index 7d532ccbc70ec..a982a8d7bf692 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkDriver.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkDriver.java @@ -57,7 +57,7 @@ public class NexmarkDriver { /** * Entry point. */ - public void runAll(OptionT options, NexmarkRunner runner) { + void runAll(OptionT options, NexmarkRunner runner) { Instant start = Instant.now(); Map baseline = loadBaseline(options.getBaselineFilename()); Map actual = new LinkedHashMap<>(); @@ -87,7 +87,7 @@ public void runAll(OptionT options, NexmarkRunner runner) { } if (!successful) { - System.exit(1); + throw new RuntimeException("Execution was not successful"); } } @@ -149,8 +149,6 @@ private static Map loadBaseline( /** * Print summary of {@code actual} vs (if non-null) {@code baseline}. - * - * @throws IOException */ private static void saveSummary( @Nullable String summaryFilename, @@ -227,7 +225,7 @@ private static void saveSummary( if (actualPerf != null) { List errors = actualPerf.errors; if (errors == null) { - errors = new ArrayList(); + errors = new ArrayList<>(); errors.add("NexmarkGoogleRunner returned null errors list"); } for (String error : errors) { @@ -300,7 +298,7 @@ public static void main(String[] args) { NexmarkOptions options = PipelineOptionsFactory.fromArgs(args) .withValidation() .as(NexmarkOptions.class); - NexmarkRunner runner = new NexmarkRunner(options); - new NexmarkDriver().runAll(options, runner); + NexmarkRunner runner = new NexmarkRunner<>(options); + new NexmarkDriver<>().runAll(options, runner); } } diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkRunner.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkRunner.java index a3c4d338766f9..6df76f092600c 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkRunner.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkRunner.java @@ -29,6 +29,7 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; +import java.util.HashMap; import java.util.List; import java.util.NoSuchElementException; import java.util.concurrent.ThreadLocalRandom; @@ -65,10 +66,12 @@ import org.apache.beam.integration.nexmark.queries.Query9Model; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.PipelineResult; +import org.apache.beam.sdk.coders.CoderException; import org.apache.beam.sdk.io.AvroIO; import org.apache.beam.sdk.io.TextIO; import org.apache.beam.sdk.io.gcp.bigquery.BigQueryIO; import org.apache.beam.sdk.io.gcp.pubsub.PubsubIO; +import org.apache.beam.sdk.io.gcp.pubsub.PubsubMessage; import org.apache.beam.sdk.metrics.DistributionResult; import org.apache.beam.sdk.metrics.MetricNameFilter; import org.apache.beam.sdk.metrics.MetricQueryResults; @@ -77,6 +80,7 @@ import org.apache.beam.sdk.testing.PAssert; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.ParDo; +import org.apache.beam.sdk.util.CoderUtils; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionTuple; import org.apache.beam.sdk.values.TimestampedValue; @@ -91,15 +95,15 @@ public class NexmarkRunner { /** * Minimum number of samples needed for 'stead-state' rate calculation. */ - protected static final int MIN_SAMPLES = 9; + private static final int MIN_SAMPLES = 9; /** * Minimum length of time over which to consider samples for 'steady-state' rate calculation. */ - protected static final Duration MIN_WINDOW = Duration.standardMinutes(2); + private static final Duration MIN_WINDOW = Duration.standardMinutes(2); /** * Delay between perf samples. */ - protected static final Duration PERF_DELAY = Duration.standardSeconds(15); + private static final Duration PERF_DELAY = Duration.standardSeconds(15); /** * How long to let streaming pipeline run after all events have been generated and we've * seen no activity. @@ -117,37 +121,37 @@ public class NexmarkRunner { /** * NexmarkOptions shared by all runs. */ - protected final OptionT options; + private final OptionT options; /** * Which configuration we are running. */ @Nullable - protected NexmarkConfiguration configuration; + private NexmarkConfiguration configuration; /** * If in --pubsubMode=COMBINED, the event monitor for the publisher pipeline. Otherwise null. */ @Nullable - protected Monitor publisherMonitor; + private Monitor publisherMonitor; /** * If in --pubsubMode=COMBINED, the pipeline result for the publisher pipeline. Otherwise null. */ @Nullable - protected PipelineResult publisherResult; + private PipelineResult publisherResult; /** * Result for the main pipeline. */ @Nullable - protected PipelineResult mainResult; + private PipelineResult mainResult; /** * Query name we are running. */ @Nullable - protected String queryName; + private String queryName; public NexmarkRunner(OptionT options) { this.options = options; @@ -160,7 +164,7 @@ public NexmarkRunner(OptionT options) { /** * Is this query running in streaming mode? */ - protected boolean isStreaming() { + private boolean isStreaming() { return options.isStreaming(); } @@ -174,7 +178,7 @@ protected int coresPerWorker() { /** * Return maximum number of workers. */ - protected int maxNumWorkers() { + private int maxNumWorkers() { return 5; } @@ -182,7 +186,7 @@ protected int maxNumWorkers() { * Return the current value for a long counter, or a default value if can't be retrieved. * Note this uses only attempted metrics because some runners don't support committed metrics. */ - protected long getCounterMetric(PipelineResult result, String namespace, String name, + private long getCounterMetric(PipelineResult result, String namespace, String name, long defaultValue) { //TODO Ismael calc this only once MetricQueryResults metrics = result.metrics().queryMetrics( @@ -201,7 +205,7 @@ protected long getCounterMetric(PipelineResult result, String namespace, String * Return the current value for a long counter, or a default value if can't be retrieved. * Note this uses only attempted metrics because some runners don't support committed metrics. */ - protected long getDistributionMetric(PipelineResult result, String namespace, String name, + private long getDistributionMetric(PipelineResult result, String namespace, String name, DistributionType distType, long defaultValue) { MetricQueryResults metrics = result.metrics().queryMetrics( MetricsFilter.builder().addNameFilter(MetricNameFilter.named(namespace, name)).build()); @@ -226,7 +230,7 @@ private enum DistributionType {MIN, MAX} /** * Return the current value for a time counter, or -1 if can't be retrieved. */ - protected long getTimestampMetric(long now, long value) { + private long getTimestampMetric(long now, long value) { //TODO Ismael improve doc if (Math.abs(value - now) > Duration.standardDays(10000).getMillis()) { return -1; @@ -238,8 +242,7 @@ protected long getTimestampMetric(long now, long value) { * Find a 'steady state' events/sec from {@code snapshots} and * store it in {@code perf} if found. */ - protected void captureSteadyState(NexmarkPerf perf, - List snapshots) { + private void captureSteadyState(NexmarkPerf perf, List snapshots) { if (!options.isStreaming()) { return; } @@ -426,7 +429,7 @@ private NexmarkPerf currentPerf( return perf; } - protected String getJobId(PipelineResult job) { + private String getJobId(PipelineResult job) { return ""; } @@ -528,15 +531,14 @@ enum MetricType { /** * Build and run a pipeline using specified options. */ - protected interface PipelineBuilder { + interface PipelineBuilder { void build(OptionT publishOnlyOptions); } /** * Invoke the builder with options suitable for running a publish-only child pipeline. */ - protected void invokeBuilderForPublishOnlyPipeline( - PipelineBuilder builder) { + private void invokeBuilderForPublishOnlyPipeline(PipelineBuilder builder) { builder.build(options); // throw new UnsupportedOperationException( // "Cannot use --pubSubMode=COMBINED with DirectRunner"); @@ -546,7 +548,7 @@ protected void invokeBuilderForPublishOnlyPipeline( * If monitoring, wait until the publisher pipeline has run long enough to establish * a backlog on the Pubsub topic. Otherwise, return immediately. */ - protected void waitForPublisherPreload() { + private void waitForPublisherPreload() { throw new UnsupportedOperationException(); } @@ -555,7 +557,7 @@ protected void waitForPublisherPreload() { * it was measured. */ @Nullable - protected NexmarkPerf monitor(NexmarkQuery query) { + private NexmarkPerf monitor(NexmarkQuery query) { if (!options.getMonitorJobs()) { return null; } @@ -841,14 +843,28 @@ private PCollection sourceEventsFromSynthetic(Pipeline p) { private PCollection sourceEventsFromPubsub(Pipeline p, long now) { String shortSubscription = shortSubscription(now); NexmarkUtils.console("Reading events from Pubsub %s", shortSubscription); - PubsubIO.Read io = - PubsubIO.read().fromSubscription(shortSubscription) - .withIdAttribute(NexmarkUtils.PUBSUB_ID) - .withCoder(Event.CODER); + + PubsubIO.Read io = + PubsubIO.readPubsubMessagesWithAttributes().fromSubscription(shortSubscription) + .withIdAttribute(NexmarkUtils.PUBSUB_ID); if (!configuration.usePubsubPublishTime) { io = io.withTimestampAttribute(NexmarkUtils.PUBSUB_TIMESTAMP); } - return p.apply(queryName + ".ReadPubsubEvents", io); + + return p + .apply(queryName + ".ReadPubsubEvents", io) + .apply(queryName + ".PubsubMessageToEvent", ParDo.of(new DoFn() { + @ProcessElement + public void processElement(ProcessContext c) { + byte[] payload = c.element().getPayload(); + try { + Event event = CoderUtils.decodeFromByteArray(Event.CODER, payload); + c.output(event); + } catch (CoderException e) { + // TODO Log decoding Event error + } + } + })); } /** @@ -861,9 +877,8 @@ private PCollection sourceEventsFromAvro(Pipeline p) { } NexmarkUtils.console("Reading events from Avro files at %s", filename); return p - .apply(queryName + ".ReadAvroEvents", AvroIO.Read - .from(filename + "*.avro") - .withSchema(Event.class)) + .apply(queryName + ".ReadAvroEvents", AvroIO.read(Event.class) + .from(filename + "*.avro")) .apply("OutputWithTimestamp", NexmarkQuery.EVENT_TIMESTAMP_FROM_DATA); } @@ -873,14 +888,28 @@ private PCollection sourceEventsFromAvro(Pipeline p) { private void sinkEventsToPubsub(PCollection events, long now) { String shortTopic = shortTopic(now); NexmarkUtils.console("Writing events to Pubsub %s", shortTopic); - PubsubIO.Write io = - PubsubIO.write().to(shortTopic) - .withIdAttribute(NexmarkUtils.PUBSUB_ID) - .withCoder(Event.CODER); + + PubsubIO.Write io = + PubsubIO.writePubsubMessages().to(shortTopic) + .withIdAttribute(NexmarkUtils.PUBSUB_ID); if (!configuration.usePubsubPublishTime) { io = io.withTimestampAttribute(NexmarkUtils.PUBSUB_TIMESTAMP); } - events.apply(queryName + ".WritePubsubEvents", io); + + events.apply(queryName + ".EventToPubsubMessage", + ParDo.of(new DoFn() { + @ProcessElement + public void processElement(ProcessContext c) { + try { + byte[] payload = CoderUtils.encodeToByteArray(Event.CODER, c.element()); + c.output(new PubsubMessage(payload, new HashMap())); + } catch (CoderException e1) { + // TODO Log encoding Event error + } + } + }) + ) + .apply(queryName + ".WritePubsubEvents", io); } /** @@ -890,7 +919,7 @@ private void sinkResultsToPubsub(PCollection formattedResults, long now) String shortTopic = shortTopic(now); NexmarkUtils.console("Writing results to Pubsub %s", shortTopic); PubsubIO.Write io = - PubsubIO.write().to(shortTopic) + PubsubIO.writeStrings().to(shortTopic) .withIdAttribute(NexmarkUtils.PUBSUB_ID); if (!configuration.usePubsubPublishTime) { io = io.withTimestampAttribute(NexmarkUtils.PUBSUB_TIMESTAMP); @@ -917,18 +946,16 @@ private void sinkEventsToAvro(PCollection source) { } NexmarkUtils.console("Writing events to Avro files at %s", filename); source.apply(queryName + ".WriteAvroEvents", - AvroIO.Write.to(filename + "/event").withSuffix(".avro").withSchema(Event.class)); + AvroIO.write(Event.class).to(filename + "/event").withSuffix(".avro")); source.apply(NexmarkQuery.JUST_BIDS) .apply(queryName + ".WriteAvroBids", - AvroIO.Write.to(filename + "/bid").withSuffix(".avro").withSchema(Bid.class)); + AvroIO.write(Bid.class).to(filename + "/bid").withSuffix(".avro")); source.apply(NexmarkQuery.JUST_NEW_AUCTIONS) .apply(queryName + ".WriteAvroAuctions", - AvroIO.Write.to(filename + "/auction").withSuffix(".avro") - .withSchema(Auction.class)); + AvroIO.write(Auction.class).to(filename + "/auction").withSuffix(".avro")); source.apply(NexmarkQuery.JUST_NEW_PERSONS) .apply(queryName + ".WriteAvroPeople", - AvroIO.Write.to(filename + "/person").withSuffix(".avro") - .withSchema(Person.class)); + AvroIO.write(Person.class).to(filename + "/person").withSuffix(".avro")); } /** @@ -938,7 +965,7 @@ private void sinkResultsToText(PCollection formattedResults, long now) { String filename = textFilename(now); NexmarkUtils.console("Writing results to text files at %s", filename); formattedResults.apply(queryName + ".WriteTextResults", - TextIO.Write.to(filename)); + TextIO.write().to(filename)); } private static class StringToTableRow extends DoFn { @@ -1010,12 +1037,12 @@ private PCollection createSource(Pipeline p, final long now) { // Send synthesized events to Pubsub in separate publisher job. // We won't start the main pipeline until the publisher has sent the pre-load events. // We'll shutdown the publisher job when we notice the main job has finished. - invokeBuilderForPublishOnlyPipeline(new PipelineBuilder() { + invokeBuilderForPublishOnlyPipeline(new PipelineBuilder() { @Override public void build(NexmarkOptions publishOnlyOptions) { Pipeline sp = Pipeline.create(options); NexmarkUtils.setupPipeline(configuration.coderStrategy, sp); - publisherMonitor = new Monitor(queryName, "publisher"); + publisherMonitor = new Monitor<>(queryName, "publisher"); sinkEventsToPubsub( sourceEventsFromSynthetic(sp) .apply(queryName + ".Monitor", publisherMonitor.getTransform()), @@ -1140,9 +1167,6 @@ public NexmarkPerf run(NexmarkConfiguration runConfiguration) { checkState(queryName == null); configuration = runConfiguration; - // GCS URI patterns to delete on exit. - List pathsToDelete = new ArrayList<>(); - try { NexmarkUtils.console("Running %s", configuration.toShortString()); @@ -1220,9 +1244,6 @@ public NexmarkPerf run(NexmarkConfiguration runConfiguration) { } ((Query10) query).setOutputPath(path); ((Query10) query).setMaxNumWorkers(maxNumWorkers()); - if (path != null && options.getManageResources()) { - pathsToDelete.add(path + "/**"); - } } // Apply query. @@ -1252,7 +1273,6 @@ public NexmarkPerf run(NexmarkConfiguration runConfiguration) { } finally { configuration = null; queryName = null; - // TODO: Cleanup pathsToDelete } } } diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkUtils.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkUtils.java index 18589c44e46de..f6215e9f15d6f 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkUtils.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkUtils.java @@ -55,6 +55,9 @@ import org.apache.beam.sdk.io.Read; import org.apache.beam.sdk.metrics.Counter; import org.apache.beam.sdk.metrics.Metrics; +import org.apache.beam.sdk.state.StateSpec; +import org.apache.beam.sdk.state.StateSpecs; +import org.apache.beam.sdk.state.ValueState; import org.apache.beam.sdk.transforms.Combine; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.PTransform; @@ -63,9 +66,6 @@ import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.GlobalWindows; import org.apache.beam.sdk.transforms.windowing.Window; -import org.apache.beam.sdk.util.state.StateSpec; -import org.apache.beam.sdk.util.state.StateSpecs; -import org.apache.beam.sdk.util.state.ValueState; import org.apache.beam.sdk.values.PBegin; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.TimestampedValue; @@ -178,7 +178,7 @@ public enum ResourceNameMode { /** Names are suffixed with the query being run. */ QUERY, /** Names are suffixed with the query being run and a random number. */ - QUERY_AND_SALT; + QUERY_AND_SALT } /** @@ -310,7 +310,7 @@ public static void info(String format, Object... args) { * Log message to console. For client side only. */ public static void console(String format, Object... args) { - System.out.printf("%s %s\n", Instant.now(), String.format(format, args)); + System.out.printf("%s %s%n", Instant.now(), String.format(format, args)); } /** @@ -326,7 +326,7 @@ public static void console(String format, Object... args) { /** * All events will be given a timestamp relative to this time (ms since epoch). */ - public static final long BASE_TIME = Instant.parse("2015-07-15T00:00:00.000Z").getMillis(); + private static final long BASE_TIME = Instant.parse("2015-07-15T00:00:00.000Z").getMillis(); /** * Instants guaranteed to be strictly before and after all event timestamps, and which won't @@ -377,7 +377,7 @@ public static void setupPipeline(CoderStrategy coderStrategy, Pipeline p) { /** * Return a generator config to match the given {@code options}. */ - public static GeneratorConfig standardGeneratorConfig(NexmarkConfiguration configuration) { + private static GeneratorConfig standardGeneratorConfig(NexmarkConfiguration configuration) { return new GeneratorConfig(configuration, configuration.useWallclockEventTime ? System.currentTimeMillis() : BASE_TIME, 0, @@ -558,15 +558,14 @@ public void processElement(ProcessContext c) { } p++; } - long next = System.currentTimeMillis(); - now = next; + now = System.currentTimeMillis(); } c.output(c.element()); } }); } - private static final StateSpec> DUMMY_TAG = + private static final StateSpec> DUMMY_TAG = StateSpecs.value(ByteArrayCoder.of()); private static final int MAX_BUFFER_SIZE = 1 << 24; @@ -578,20 +577,19 @@ public static ParDo.SingleOutput diskBusy(String name, final long byte @ProcessElement public void processElement(ProcessContext c) { long remain = bytes; - long start = System.currentTimeMillis(); - long now = start; +// long now = System.currentTimeMillis(); while (remain > 0) { + //TODO Ismael google on state long thisBytes = Math.min(remain, MAX_BUFFER_SIZE); remain -= thisBytes; - byte[] arr = new byte[(int) thisBytes]; - for (int i = 0; i < thisBytes; i++) { - arr[i] = (byte) now; - } - //TODO Ismael google on state +// byte[] arr = new byte[(int) thisBytes]; +// for (int i = 0; i < thisBytes; i++) { +// arr[i] = (byte) now; +// } // ValueState state = c.windowingInternals().stateInternals().state( // StateNamespaces.global(), DUMMY_TAG); // state.write(arr); - now = System.currentTimeMillis(); +// now = System.currentTimeMillis(); } c.output(c.element()); } diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/Auction.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/Auction.java index 4b1a8480cd952..5c018dc45bb5f 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/Auction.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/Auction.java @@ -81,14 +81,14 @@ public Auction decode( /** Extra auction properties. */ @JsonProperty - public final String itemName; + private final String itemName; @JsonProperty - public final String description; + private final String description; /** Initial bid price, in cents. */ @JsonProperty - public final long initialBid; + private final long initialBid; /** Reserve price, in cents. */ @JsonProperty @@ -111,7 +111,7 @@ public Auction decode( /** Additional arbitrary payload for performance testing. */ @JsonProperty - public final String extra; + private final String extra; // For Avro only. diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/AuctionCount.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/AuctionCount.java index e6d34504e1972..c83a4554fbaf8 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/AuctionCount.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/AuctionCount.java @@ -54,11 +54,9 @@ public AuctionCount decode( } }; - @JsonProperty - public final long auction; + @JsonProperty private final long auction; - @JsonProperty - public final long count; + @JsonProperty private final long count; // For Avro only. @SuppressWarnings("unused") diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/AuctionPrice.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/AuctionPrice.java index cb971e29e1234..43d0b275dae4e 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/AuctionPrice.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/AuctionPrice.java @@ -55,11 +55,11 @@ public AuctionPrice decode( }; @JsonProperty - public final long auction; + private final long auction; /** Price in cents. */ @JsonProperty - public final long price; + private final long price; // For Avro only. @SuppressWarnings("unused") diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/BidsPerSession.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/BidsPerSession.java index 26b6a414d39fb..6dddf34598df3 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/BidsPerSession.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/BidsPerSession.java @@ -56,10 +56,10 @@ public BidsPerSession decode( }; @JsonProperty - public final long personId; + private final long personId; @JsonProperty - public final long bidsPerSession; + private final long bidsPerSession; public BidsPerSession() { personId = 0; diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/Done.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/Done.java index 42999cd0a3c91..0c14e8f5fb1e8 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/Done.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/Done.java @@ -54,7 +54,7 @@ public Done decode( }; @JsonProperty - public final String message; + private final String message; // For Avro only. @SuppressWarnings("unused") diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/Event.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/Event.java index e2130c99fb4ae..1f1f096a8bc02 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/Event.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/Event.java @@ -138,19 +138,6 @@ public boolean hasAnnotation(String annotation) { } } - /** - * Remove {@code annotation} from event. (Used for debugging.) - */ - public Event withoutAnnotation(String annotation) { - if (newPerson != null) { - return new Event(newPerson.withoutAnnotation(annotation)); - } else if (newAuction != null) { - return new Event(newAuction.withoutAnnotation(annotation)); - } else { - return new Event(bid.withoutAnnotation(annotation)); - } - } - @Override public long sizeInBytes() { if (newPerson != null) { diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/IdNameReserve.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/IdNameReserve.java index cf1e571f5a139..17b8c4a9ac378 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/IdNameReserve.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/IdNameReserve.java @@ -60,14 +60,14 @@ public IdNameReserve decode( }; @JsonProperty - public final long id; + private final long id; @JsonProperty - public final String name; + private final String name; /** Reserve price in cents. */ @JsonProperty - public final long reserve; + private final long reserve; // For Avro only. @SuppressWarnings("unused") diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/NameCityStateId.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/NameCityStateId.java index 86d1738c3ee4d..28f25cd3a49f0 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/NameCityStateId.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/NameCityStateId.java @@ -62,16 +62,16 @@ public NameCityStateId decode( }; @JsonProperty - public final String name; + private final String name; @JsonProperty - public final String city; + private final String city; @JsonProperty - public final String state; + private final String state; @JsonProperty - public final long id; + private final long id; // For Avro only. @SuppressWarnings("unused") diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/Person.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/Person.java index 906df941798b5..c690fd445a56e 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/Person.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/Person.java @@ -77,10 +77,10 @@ public Person decode( public final String name; @JsonProperty - public final String emailAddress; + private final String emailAddress; @JsonProperty - public final String creditCard; + private final String creditCard; @JsonProperty public final String city; @@ -93,7 +93,7 @@ public Person decode( /** Additional arbitrary payload for performance testing. */ @JsonProperty - public final String extra; + private final String extra; // For Avro only. @SuppressWarnings("unused") diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/SellerPrice.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/SellerPrice.java index 68f2697c8162b..52ff540ac639d 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/SellerPrice.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/SellerPrice.java @@ -60,7 +60,7 @@ public SellerPrice decode( /** Price in cents. */ @JsonProperty - public final long price; + private final long price; // For Avro only. @SuppressWarnings("unused") diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/AbstractSimulator.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/AbstractSimulator.java index 270b5c3c9f017..1395182ba8b05 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/AbstractSimulator.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/AbstractSimulator.java @@ -37,7 +37,7 @@ */ public abstract class AbstractSimulator { /** Window size for action bucket sampling. */ - public static final Duration WINDOW_SIZE = Duration.standardMinutes(1); + private static final Duration WINDOW_SIZE = Duration.standardMinutes(1); /** Input event stream we should draw from. */ private final Iterator> input; @@ -77,7 +77,7 @@ public AbstractSimulator(Iterator> input) { /** Called by implementors of {@link #run}: Fetch the next input element. */ @Nullable - protected TimestampedValue nextInput() { + TimestampedValue nextInput() { if (!input.hasNext()) { return null; } @@ -90,7 +90,7 @@ protected TimestampedValue nextInput() { * Called by implementors of {@link #run}: Capture an intermediate result, for the purpose of * recording the expected activity of the query over time. */ - protected void addIntermediateResult(TimestampedValue result) { + void addIntermediateResult(TimestampedValue result) { NexmarkUtils.info("intermediate result: %s", result); updateCounts(result.getTimestamp()); } @@ -99,7 +99,7 @@ protected void addIntermediateResult(TimestampedValue result) { * Called by implementors of {@link #run}: Capture a final result, for the purpose of checking * semantic correctness. */ - protected void addResult(TimestampedValue result) { + void addResult(TimestampedValue result) { NexmarkUtils.info("result: %s", result); pendingResults.add(result); updateCounts(result.getTimestamp()); @@ -121,7 +121,7 @@ private void updateCounts(Instant timestamp) { } /** Called by implementors of {@link #run}: Record that no more results will be emitted. */ - protected void allDone() { + void allDone() { isDone = true; } diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/NexmarkQuery.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/NexmarkQuery.java index 0796ce5677a3a..09415c0cfc3ed 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/NexmarkQuery.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/NexmarkQuery.java @@ -46,10 +46,10 @@ public abstract class NexmarkQuery extends PTransform, PCollection>> { public static final TupleTag AUCTION_TAG = new TupleTag<>("auctions"); public static final TupleTag BID_TAG = new TupleTag<>("bids"); - protected static final TupleTag PERSON_TAG = new TupleTag<>("person"); + static final TupleTag PERSON_TAG = new TupleTag<>("person"); /** Predicate to detect a new person event. */ - protected static final SerializableFunction IS_NEW_PERSON = + private static final SerializableFunction IS_NEW_PERSON = new SerializableFunction() { @Override public Boolean apply(Event event) { @@ -58,7 +58,7 @@ public Boolean apply(Event event) { }; /** DoFn to convert a new person event to a person. */ - protected static final DoFn AS_PERSON = new DoFn() { + private static final DoFn AS_PERSON = new DoFn() { @ProcessElement public void processElement(ProcessContext c) { c.output(c.element().newPerson); @@ -66,7 +66,7 @@ public void processElement(ProcessContext c) { }; /** Predicate to detect a new auction event. */ - protected static final SerializableFunction IS_NEW_AUCTION = + private static final SerializableFunction IS_NEW_AUCTION = new SerializableFunction() { @Override public Boolean apply(Event event) { @@ -75,7 +75,7 @@ public Boolean apply(Event event) { }; /** DoFn to convert a new auction event to an auction. */ - protected static final DoFn AS_AUCTION = new DoFn() { + private static final DoFn AS_AUCTION = new DoFn() { @ProcessElement public void processElement(ProcessContext c) { c.output(c.element().newAuction); @@ -83,7 +83,7 @@ public void processElement(ProcessContext c) { }; /** Predicate to detect a new bid event. */ - protected static final SerializableFunction IS_BID = + private static final SerializableFunction IS_BID = new SerializableFunction() { @Override public Boolean apply(Event event) { @@ -92,7 +92,7 @@ public Boolean apply(Event event) { }; /** DoFn to convert a bid event to a bid. */ - protected static final DoFn AS_BID = new DoFn() { + private static final DoFn AS_BID = new DoFn() { @ProcessElement public void processElement(ProcessContext c) { c.output(c.element().bid); @@ -100,7 +100,7 @@ public void processElement(ProcessContext c) { }; /** Transform to key each person by their id. */ - protected static final ParDo.SingleOutput> PERSON_BY_ID = + static final ParDo.SingleOutput> PERSON_BY_ID = ParDo.of(new DoFn>() { @ProcessElement public void processElement(ProcessContext c) { @@ -109,7 +109,7 @@ public void processElement(ProcessContext c) { }); /** Transform to key each auction by its id. */ - protected static final ParDo.SingleOutput> AUCTION_BY_ID = + static final ParDo.SingleOutput> AUCTION_BY_ID = ParDo.of(new DoFn>() { @ProcessElement public void processElement(ProcessContext c) { @@ -118,7 +118,7 @@ public void processElement(ProcessContext c) { }); /** Transform to key each auction by its seller id. */ - protected static final ParDo.SingleOutput> AUCTION_BY_SELLER = + static final ParDo.SingleOutput> AUCTION_BY_SELLER = ParDo.of(new DoFn>() { @ProcessElement public void processElement(ProcessContext c) { @@ -127,7 +127,7 @@ public void processElement(ProcessContext c) { }); /** Transform to key each bid by it's auction id. */ - protected static final ParDo.SingleOutput> BID_BY_AUCTION = + static final ParDo.SingleOutput> BID_BY_AUCTION = ParDo.of(new DoFn>() { @ProcessElement public void processElement(ProcessContext c) { @@ -136,7 +136,7 @@ public void processElement(ProcessContext c) { }); /** Transform to project the auction id from each bid. */ - protected static final ParDo.SingleOutput BID_TO_AUCTION = + static final ParDo.SingleOutput BID_TO_AUCTION = ParDo.of(new DoFn() { @ProcessElement public void processElement(ProcessContext c) { @@ -145,7 +145,7 @@ public void processElement(ProcessContext c) { }); /** Transform to project the price from each bid. */ - protected static final ParDo.SingleOutput BID_TO_PRICE = + static final ParDo.SingleOutput BID_TO_PRICE = ParDo.of(new DoFn() { @ProcessElement public void processElement(ProcessContext c) { @@ -205,13 +205,13 @@ public PCollection expand(PCollection input) { } }; - protected final NexmarkConfiguration configuration; + final NexmarkConfiguration configuration; public final Monitor eventMonitor; public final Monitor resultMonitor; - public final Monitor endOfStreamMonitor; - protected final Counter fatalCounter; + private final Monitor endOfStreamMonitor; + private final Counter fatalCounter; - protected NexmarkQuery(NexmarkConfiguration configuration, String name) { + NexmarkQuery(NexmarkConfiguration configuration, String name) { super(name); this.configuration = configuration; if (configuration.debug) { diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/NexmarkQueryModel.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/NexmarkQueryModel.java index 1ad909988f021..bfa668bc929a2 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/NexmarkQueryModel.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/NexmarkQueryModel.java @@ -43,7 +43,7 @@ public abstract class NexmarkQueryModel implements Serializable { public final NexmarkConfiguration configuration; - public NexmarkQueryModel(NexmarkConfiguration configuration) { + NexmarkQueryModel(NexmarkConfiguration configuration) { this.configuration = configuration; } @@ -51,7 +51,7 @@ public NexmarkQueryModel(NexmarkConfiguration configuration) { * Return the start of the most recent window of {@code size} and {@code period} which ends * strictly before {@code timestamp}. */ - public static Instant windowStart(Duration size, Duration period, Instant timestamp) { + static Instant windowStart(Duration size, Duration period, Instant timestamp) { long ts = timestamp.getMillis(); long p = period.getMillis(); long lim = ts - ts % p; @@ -60,7 +60,7 @@ public static Instant windowStart(Duration size, Duration period, Instant timest } /** Convert {@code itr} to strings capturing values, timestamps and order. */ - protected static List toValueTimestampOrder(Iterator> itr) { + static List toValueTimestampOrder(Iterator> itr) { List strings = new ArrayList<>(); while (itr.hasNext()) { strings.add(itr.next().toString()); @@ -69,7 +69,7 @@ protected static List toValueTimestampOrder(Iterator List toValueOrder(Iterator> itr) { + static List toValueOrder(Iterator> itr) { List strings = new ArrayList<>(); while (itr.hasNext()) { strings.add(itr.next().getValue().toString()); @@ -78,7 +78,7 @@ protected static List toValueOrder(Iterator> itr } /** Convert {@code itr} to strings capturing values only. */ - protected static Set toValue(Iterator> itr) { + static Set toValue(Iterator> itr) { Set strings = new HashSet<>(); while (itr.hasNext()) { strings.add(itr.next().getValue().toString()); @@ -90,7 +90,7 @@ protected static Set toValue(Iterator> itr) { public abstract AbstractSimulator simulator(); /** Return sub-sequence of results which are significant for model. */ - protected Iterable> relevantResults( + Iterable> relevantResults( Iterable> results) { return results; } @@ -104,8 +104,6 @@ protected Iterable> relevantResults( /** Return assertion to use on results of pipeline for this query. */ public SerializableFunction>, Void> assertionFor() { final Collection expectedStrings = toCollection(simulator().results()); - final String[] expectedStringsArray = - expectedStrings.toArray(new String[expectedStrings.size()]); return new SerializableFunction>, Void>() { @Override @@ -113,9 +111,6 @@ public Void apply(Iterable> actual) { Collection actualStrings = toCollection(relevantResults(actual).iterator()); Assert.assertThat("wrong pipeline output", actualStrings, IsEqual.equalTo(expectedStrings)); -//compare without order -// Assert.assertThat("wrong pipeline output", actualStrings, -// IsIterableContainingInAnyOrder.containsInAnyOrder(expectedStringsArray)); return null; } }; diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query0Model.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query0Model.java index 6fb6613735d34..8e655917fd24d 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query0Model.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query0Model.java @@ -32,7 +32,7 @@ public class Query0Model extends NexmarkQueryModel { /** * Simulator for query 0. */ - private class Simulator extends AbstractSimulator { + private static class Simulator extends AbstractSimulator { public Simulator(NexmarkConfiguration configuration) { super(NexmarkUtils.standardEventIterator(configuration)); } diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query10.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query10.java index c919691e7aba8..516dab1e793a5 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query10.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query10.java @@ -35,7 +35,7 @@ import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.metrics.Counter; import org.apache.beam.sdk.metrics.Metrics; -import org.apache.beam.sdk.options.GcsOptions; +import org.apache.beam.sdk.extensions.gcp.options.GcsOptions; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.GroupByKey; import org.apache.beam.sdk.transforms.ParDo; @@ -101,7 +101,7 @@ public OutputFile( @Override public String toString() { - return String.format("%s %s %d %s %s\n", maxTimestamp, shard, index, timing, filename); + return String.format("%s %s %d %s %s%n", maxTimestamp, shard, index, timing, filename); } } @@ -130,8 +130,6 @@ public void setMaxNumWorkers(int maxNumWorkers) { /** * Return channel for writing bytes to GCS. - * - * @throws IOException */ private WritableByteChannel openWritableGcsFile(GcsOptions options, String filename) throws IOException { diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query11.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query11.java index fd936a9d4e097..6db9bcf81523b 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query11.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query11.java @@ -63,14 +63,13 @@ private PCollection applyTyped(PCollection events) { Repeatedly.forever(AfterPane.elementCountAtLeast(configuration.maxLogEvents))) .discardingFiredPanes() .withAllowedLateness(Duration.standardSeconds(configuration.occasionalDelaySec / 2))); - PCollection bidsPerSession = biddersWindowed.apply(Count.perElement()) + return biddersWindowed.apply(Count.perElement()) .apply(name + ".ToResult", ParDo.of(new DoFn, BidsPerSession>() { @ProcessElement public void processElement(ProcessContext c) { c.output(new BidsPerSession(c.element().getKey(), c.element().getValue())); } })); - return bidsPerSession; } @Override diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query1Model.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query1Model.java index 03886874c78c8..5d4de45e1d7bd 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query1Model.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query1Model.java @@ -34,7 +34,7 @@ public class Query1Model extends NexmarkQueryModel implements Serializable { /** * Simulator for query 1. */ - private class Simulator extends AbstractSimulator { + private static class Simulator extends AbstractSimulator { public Simulator(NexmarkConfiguration configuration) { super(NexmarkUtils.standardEventIterator(configuration)); } diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query3.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query3.java index 71364ba6285ff..f74b78db4b4b9 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query3.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query3.java @@ -17,7 +17,6 @@ */ package org.apache.beam.integration.nexmark.queries; -import java.io.IOException; import java.util.ArrayList; import java.util.List; import org.apache.beam.integration.nexmark.NexmarkConfiguration; @@ -30,6 +29,13 @@ import org.apache.beam.sdk.coders.ListCoder; import org.apache.beam.sdk.metrics.Counter; import org.apache.beam.sdk.metrics.Metrics; +import org.apache.beam.sdk.state.TimeDomain; +import org.apache.beam.sdk.state.Timer; +import org.apache.beam.sdk.state.TimerSpec; +import org.apache.beam.sdk.state.TimerSpecs; +import org.apache.beam.sdk.state.ValueState; +import org.apache.beam.sdk.state.StateSpec; +import org.apache.beam.sdk.state.StateSpecs; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.Filter; import org.apache.beam.sdk.transforms.ParDo; @@ -41,13 +47,6 @@ import org.apache.beam.sdk.transforms.windowing.GlobalWindows; import org.apache.beam.sdk.transforms.windowing.Repeatedly; 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.state.StateSpec; -import org.apache.beam.sdk.util.state.StateSpecs; -import org.apache.beam.sdk.util.state.ValueState; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; import org.joda.time.Duration; @@ -176,18 +175,18 @@ protected PCollection applyPrim(PCollection events) { */ private static class JoinDoFn extends DoFn, KV> { - private int maxAuctionsWaitingTime; + private final int maxAuctionsWaitingTime; private static final String AUCTIONS = "auctions"; private static final String PERSON = "person"; @StateId(PERSON) - private static final StateSpec> personSpec = + private static final StateSpec> personSpec = StateSpecs.value(Person.CODER); private static final String PERSON_STATE_EXPIRING = "personStateExpiring"; @StateId(AUCTIONS) - private final StateSpec>> auctionsSpec = + private final StateSpec>> auctionsSpec = StateSpecs.value(ListCoder.of(Auction.CODER)); @TimerId(PERSON_STATE_EXPIRING) @@ -219,8 +218,7 @@ public void processElement( ProcessContext c, @TimerId(PERSON_STATE_EXPIRING) Timer timer, @StateId(PERSON) ValueState personState, - @StateId(AUCTIONS) ValueState> auctionsState) - throws IOException { + @StateId(AUCTIONS) ValueState> auctionsState) { // We would *almost* implement this by rewindowing into the global window and // running a combiner over the result. The combiner's accumulator would be the // state we use below. However, combiners cannot emit intermediate results, thus diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query3Model.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query3Model.java index 6b98e2a449fa7..f415709dceff9 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query3Model.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query3Model.java @@ -42,7 +42,7 @@ public class Query3Model extends NexmarkQueryModel implements Serializable { /** * Simulator for query 3. */ - private class Simulator extends AbstractSimulator { + private static class Simulator extends AbstractSimulator { /** Auctions, indexed by seller id. */ private final Multimap newAuctions; diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query4Model.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query4Model.java index 634a58e44f034..269e47aa50220 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query4Model.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query4Model.java @@ -93,8 +93,9 @@ private void averages(Instant end) { } totals.put(category, total); } - for (long category : counts.keySet()) { - long count = counts.get(category); + for (Map.Entry entry : counts.entrySet()) { + long category = entry.getKey(); + long count = entry.getValue(); long total = totals.get(category); TimestampedValue result = TimestampedValue.of( new CategoryPrice(category, Math.round((double) total / count), true), lastTimestamp); diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query5.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query5.java index 18ce5789eaf4c..194433055b488 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query5.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query5.java @@ -18,7 +18,7 @@ package org.apache.beam.integration.nexmark.queries; import java.util.ArrayList; -import java.util.Arrays; +import java.util.Collections; import java.util.List; import org.apache.beam.integration.nexmark.NexmarkConfiguration; import org.apache.beam.integration.nexmark.NexmarkUtils; @@ -80,7 +80,7 @@ private PCollection applyTyped(PCollection events) { ParDo.of(new DoFn, KV, Long>>() { @ProcessElement public void processElement(ProcessContext c) { - c.output(KV.of(Arrays.asList(c.element().getKey()), c.element().getValue())); + c.output(KV.of(Collections.singletonList(c.element().getKey()), c.element().getValue())); } })) diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query6.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query6.java index 65789abd132a9..ea39ede396d94 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query6.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query6.java @@ -86,9 +86,7 @@ public List addInput(List accumulator, Bid input) { public List mergeAccumulators(Iterable> accumulators) { List result = new ArrayList<>(); for (List accumulator : accumulators) { - for (Bid bid : accumulator) { - result.add(bid); - } + result.addAll(accumulator); } Collections.sort(result, Bid.ASCENDING_TIME_THEN_PRICE); if (result.size() > maxNumBids) { diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query6Model.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query6Model.java index 0691714a7158c..9cb8b3d9fb4b5 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query6Model.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query6Model.java @@ -86,8 +86,9 @@ private void captureWinningBid(Auction auction, Bid bid, Instant timestamp) { protected void run() { TimestampedValue timestampedWinningBid = nextInput(); if (timestampedWinningBid == null) { - for (long seller : numWinningBidsPerSeller.keySet()) { - long count = numWinningBidsPerSeller.get(seller); + for (Map.Entry entry : numWinningBidsPerSeller.entrySet()) { + long seller = entry.getKey(); + long count = entry.getValue(); long total = totalWinningBidPricesPerSeller.get(seller); addResult(TimestampedValue.of( new SellerPrice(seller, Math.round((double) total / count)), lastTimestamp)); diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/WinningBids.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/WinningBids.java index 11a4d38c91872..52891a77bb31b 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/WinningBids.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/WinningBids.java @@ -25,8 +25,8 @@ import java.io.OutputStream; import java.io.Serializable; import java.util.ArrayList; -import java.util.Arrays; import java.util.Collection; +import java.util.Collections; import java.util.List; import java.util.Map; import java.util.TreeMap; @@ -77,7 +77,7 @@ */ public class WinningBids extends PTransform, PCollection> { /** Windows for open auctions and bids. */ - private static class AuctionOrBidWindow extends IntervalWindow implements Serializable { + private static class AuctionOrBidWindow extends IntervalWindow { /** Id of auction this window is for. */ public final long auction; @@ -104,9 +104,7 @@ private AuctionOrBidWindow( /** Return an auction window for {@code auction}. */ public static AuctionOrBidWindow forAuction(Instant timestamp, Auction auction) { - AuctionOrBidWindow result = - new AuctionOrBidWindow(timestamp, new Instant(auction.expires), auction.id, true); - return result; + return new AuctionOrBidWindow(timestamp, new Instant(auction.expires), auction.id, true); } /** @@ -127,9 +125,8 @@ public static AuctionOrBidWindow forBid( // Instead, we will just give the bid a finite window which expires at // the upper bound of auctions assuming the auction starts at the same time as the bid, // and assuming the system is running at its lowest event rate (as per interEventDelayUs). - AuctionOrBidWindow result = new AuctionOrBidWindow( + return new AuctionOrBidWindow( timestamp, timestamp.plus(expectedAuctionDurationMs * 2), bid.auction, false); - return result; } /** Is this an auction window? */ @@ -171,8 +168,7 @@ public AuctionOrBidWindow decode(InputStream inStream, Coder.Context context) throws IOException, CoderException { IntervalWindow superWindow = SUPER_CODER.decode(inStream, Coder.Context.NESTED); long auction = ID_CODER.decode(inStream, Coder.Context.NESTED); - boolean isAuctionWindow = - INT_CODER.decode(inStream, Coder.Context.NESTED) == 0 ? false : true; + boolean isAuctionWindow = INT_CODER.decode(inStream, Context.NESTED) != 0; return new AuctionOrBidWindow( superWindow.start(), superWindow.end(), auction, isAuctionWindow); } @@ -194,15 +190,16 @@ public Collection assignWindows(AssignContext c) { Event event = c.element(); if (event.newAuction != null) { // Assign auctions to an auction window which expires at the auction's close. - return Arrays.asList(AuctionOrBidWindow.forAuction(c.timestamp(), event.newAuction)); + return Collections + .singletonList(AuctionOrBidWindow.forAuction(c.timestamp(), event.newAuction)); } else if (event.bid != null) { // Assign bids to a temporary bid window which will later be merged into the appropriate // auction window. - return Arrays.asList( + return Collections.singletonList( AuctionOrBidWindow.forBid(expectedAuctionDurationMs, c.timestamp(), event.bid)); } else { // Don't assign people to any window. They will thus be dropped. - return Arrays.asList(); + return Collections.emptyList(); } } @@ -226,8 +223,9 @@ public void mergeWindows(MergeContext c) throws Exception { // Merge all 'bid' windows into their corresponding 'auction' window, provided the // auction has not expired. - for (long auction : idToTrueAuctionWindow.keySet()) { - AuctionOrBidWindow auctionWindow = idToTrueAuctionWindow.get(auction); + for (Map.Entry entry : idToTrueAuctionWindow.entrySet()) { + long auction = entry.getKey(); + AuctionOrBidWindow auctionWindow = entry.getValue(); List bidWindows = idToBidAuctionWindows.get(auction); if (bidWindows != null) { List toBeMerged = new ArrayList<>(); @@ -296,8 +294,8 @@ public WinningBids(String name, NexmarkConfiguration configuration) { configuration.firstEventRate, configuration.nextEventRate, configuration.rateUnit, configuration.numEventGenerators); long longestDelayUs = 0; - for (int i = 0; i < interEventDelayUs.length; i++) { - longestDelayUs = Math.max(longestDelayUs, interEventDelayUs[i]); + for (long interEventDelayU : interEventDelayUs) { + longestDelayUs = Math.max(longestDelayUs, interEventDelayU); } // Adjust for proportion of auction events amongst all events. longestDelayUs = diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/sources/Generator.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/sources/Generator.java index 012d4e65007b2..2a2732bac798b 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/sources/Generator.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/sources/Generator.java @@ -123,8 +123,8 @@ public Checkpoint decode( @Override public void verifyDeterministic() throws NonDeterministicException {} }; - private long numEvents; - private long wallclockBaseTime; + private final long numEvents; + private final long wallclockBaseTime; private Checkpoint(long numEvents, long wallclockBaseTime) { this.numEvents = numEvents; @@ -403,8 +403,8 @@ private static long nextLong(Random random, long n) { if (n < Integer.MAX_VALUE) { return random.nextInt((int) n); } else { - // TODO: Very skewed distribution! Bad! - return Math.abs(random.nextLong()) % n; + // WARNING: Very skewed distribution! Bad! + return Math.abs(random.nextLong() % n); } } @@ -470,14 +470,13 @@ private Auction nextAuction(Random random, long timestamp) { long category = GeneratorConfig.FIRST_CATEGORY_ID + random.nextInt(NUM_CATEGORIES); long initialBid = nextPrice(random); - long dateTime = timestamp; long expires = timestamp + nextAuctionLengthMs(random, timestamp); String name = nextString(random, 20); String desc = nextString(random, 100); long reserve = initialBid + nextPrice(random); int currentSize = 8 + name.length() + desc.length() + 8 + 8 + 8 + 8 + 8; String extra = nextExtra(random, currentSize, config.configuration.avgAuctionByteSize); - return new Auction(id, name, desc, initialBid, reserve, dateTime, expires, seller, category, + return new Auction(id, name, desc, initialBid, reserve, timestamp, expires, seller, category, extra); } diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/sources/GeneratorConfig.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/sources/GeneratorConfig.java index 3caaf5179cf96..5799bb2071ba1 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/sources/GeneratorConfig.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/sources/GeneratorConfig.java @@ -42,7 +42,7 @@ public class GeneratorConfig implements Serializable { */ public static final int PERSON_PROPORTION = 1; public static final int AUCTION_PROPORTION = 3; - public static final int BID_PROPORTION = 46; + private static final int BID_PROPORTION = 46; public static final int PROPORTION_DENOMINATOR = PERSON_PROPORTION + AUCTION_PROPORTION + BID_PROPORTION; @@ -55,12 +55,12 @@ public class GeneratorConfig implements Serializable { * Delay between events, in microseconds. If the array has more than one entry then * the rate is changed every {@link #stepLengthSec}, and wraps around. */ - public final long[] interEventDelayUs; + private final long[] interEventDelayUs; /** * Delay before changing the current inter-event delay. */ - public final long stepLengthSec; + private final long stepLengthSec; /** * Time for first event (ms since epoch). @@ -88,13 +88,13 @@ public class GeneratorConfig implements Serializable { * True period of epoch in milliseconds. Derived from above. * (Ie time to run through cycle for all interEventDelayUs entries). */ - public final long epochPeriodMs; + private final long epochPeriodMs; /** * Number of events per epoch. Derived from above. * (Ie number of events to run through cycle for all interEventDelayUs entries). */ - public final long eventsPerEpoch; + private final long eventsPerEpoch; public GeneratorConfig( NexmarkConfiguration configuration, long baseTime, long firstEventId, @@ -121,10 +121,10 @@ public GeneratorConfig( long eventsPerEpoch = 0; long epochPeriodMs = 0; if (interEventDelayUs.length > 1) { - for (int i = 0; i < interEventDelayUs.length; i++) { - long numEventsForThisCycle = (stepLengthSec * 1_000_000L) / interEventDelayUs[i]; + for (long interEventDelayU : interEventDelayUs) { + long numEventsForThisCycle = (stepLengthSec * 1_000_000L) / interEventDelayU; eventsPerEpoch += numEventsForThisCycle; - epochPeriodMs += (numEventsForThisCycle * interEventDelayUs[i]) / 1000L; + epochPeriodMs += (numEventsForThisCycle * interEventDelayU) / 1000L; } } this.eventsPerEpoch = eventsPerEpoch; @@ -248,16 +248,16 @@ public KV timestampAndInterEventDelayUsForEvent(long eventNumber) { long epoch = eventNumber / eventsPerEpoch; long n = eventNumber % eventsPerEpoch; long offsetInEpochMs = 0; - for (int i = 0; i < interEventDelayUs.length; i++) { - long numEventsForThisCycle = (stepLengthSec * 1_000_000L) / interEventDelayUs[i]; + for (long interEventDelayU : interEventDelayUs) { + long numEventsForThisCycle = (stepLengthSec * 1_000_000L) / interEventDelayU; if (n < numEventsForThisCycle) { - long offsetInCycleUs = n * interEventDelayUs[i]; + long offsetInCycleUs = n * interEventDelayU; long timestamp = baseTime + epoch * epochPeriodMs + offsetInEpochMs + (offsetInCycleUs / 1000L); - return KV.of(timestamp, interEventDelayUs[i]); + return KV.of(timestamp, interEventDelayU); } n -= numEventsForThisCycle; - offsetInEpochMs += (numEventsForThisCycle * interEventDelayUs[i]) / 1000L; + offsetInEpochMs += (numEventsForThisCycle * interEventDelayU) / 1000L; } throw new RuntimeException("internal eventsPerEpoch incorrect"); // can't reach } diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/sources/UnboundedEventSource.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/sources/UnboundedEventSource.java index c3c6eb031e875..09d945d439fd6 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/sources/UnboundedEventSource.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/sources/UnboundedEventSource.java @@ -116,7 +116,7 @@ private class EventReader extends UnboundedReader { private TimestampedValue currentEvent; /** Events which have been held back so as to force them to be late. */ - private Queue heldBackEvents = new PriorityQueue<>(); + private final Queue heldBackEvents = new PriorityQueue<>(); public EventReader(Generator generator) { this.generator = generator; diff --git a/integration/java/nexmark/src/test/java/org/apache/beam/integration/nexmark/sources/UnboundedEventSourceTest.java b/integration/java/nexmark/src/test/java/org/apache/beam/integration/nexmark/sources/UnboundedEventSourceTest.java index 15e17a8b64e41..1d04e2a381d5d 100644 --- a/integration/java/nexmark/src/test/java/org/apache/beam/integration/nexmark/sources/UnboundedEventSourceTest.java +++ b/integration/java/nexmark/src/test/java/org/apache/beam/integration/nexmark/sources/UnboundedEventSourceTest.java @@ -53,8 +53,8 @@ private GeneratorConfig makeConfig(long n) { * confirming reading events match the model events. */ private static class EventIdChecker { - private Set seenPersonIds = new HashSet<>(); - private Set seenAuctionIds = new HashSet<>(); + private final Set seenPersonIds = new HashSet<>(); + private final Set seenAuctionIds = new HashSet<>(); public void add(Event event) { if (event.newAuction != null) { @@ -90,7 +90,6 @@ public void resumeFromCheckpoint() throws IOException { EventIdChecker checker = new EventIdChecker(); PipelineOptions options = TestPipeline.testingPipelineOptions(); - Pipeline p = TestPipeline.create(options); UnboundedEventSource source = new UnboundedEventSource(config, 1, 0, false); UnboundedReader reader = source.createReader(options, null); diff --git a/integration/pom.xml b/integration/pom.xml index 4839da5acbb4f..31f293ed7a2b8 100644 --- a/integration/pom.xml +++ b/integration/pom.xml @@ -30,6 +30,20 @@ pom Apache Beam :: Integration Tests + + + release + + + + org.codehaus.mojo + findbugs-maven-plugin + + + + + + java From 902050b0b276e22ab002e8efb390bbaa01e18e99 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Isma=C3=ABl=20Mej=C3=ADa?= Date: Fri, 5 May 2017 10:47:46 +0200 Subject: [PATCH 306/346] Disable use of GcsIOChannelFactory on query10 Make NexmarkOptions depend on GcpOptions instead of PubsubOptions issue #21 --- integration/java/nexmark/pom.xml | 5 ----- .../beam/integration/nexmark/NexmarkOptions.java | 7 ++++++- .../beam/integration/nexmark/queries/Query10.java | 13 +++++++------ 3 files changed, 13 insertions(+), 12 deletions(-) diff --git a/integration/java/nexmark/pom.xml b/integration/java/nexmark/pom.xml index 8a65c0fa7446f..35fe0f330fb75 100644 --- a/integration/java/nexmark/pom.xml +++ b/integration/java/nexmark/pom.xml @@ -218,11 +218,6 @@ google-api-services-bigquery - - com.google.cloud.bigdataoss - gcsio - - com.fasterxml.jackson.core diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkOptions.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkOptions.java index e1c1af2d9a46c..f162fd6164538 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkOptions.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkOptions.java @@ -18,14 +18,19 @@ package org.apache.beam.integration.nexmark; import javax.annotation.Nullable; +import org.apache.beam.sdk.extensions.gcp.options.GcpOptions; import org.apache.beam.sdk.io.gcp.pubsub.PubsubOptions; +import org.apache.beam.sdk.options.ApplicationNameOptions; 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.StreamingOptions; /** * Command line flags. */ -public interface NexmarkOptions extends PubsubOptions { +public interface NexmarkOptions + extends ApplicationNameOptions, GcpOptions, PipelineOptions, StreamingOptions { @Description("Which suite to run. Default is to use command line arguments for one job.") @Default.Enum("DEFAULT") NexmarkSuite getSuite(); diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query10.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query10.java index 516dab1e793a5..c868666f3242f 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query10.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query10.java @@ -50,7 +50,6 @@ import org.apache.beam.sdk.transforms.windowing.PaneInfo.Timing; import org.apache.beam.sdk.transforms.windowing.Repeatedly; import org.apache.beam.sdk.transforms.windowing.Window; -import org.apache.beam.sdk.util.GcsIOChannelFactory; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; import org.joda.time.Duration; @@ -133,11 +132,13 @@ public void setMaxNumWorkers(int maxNumWorkers) { */ private WritableByteChannel openWritableGcsFile(GcsOptions options, String filename) throws IOException { - WritableByteChannel channel = - GcsIOChannelFactory.fromOptions(options).create(filename, "text/plain"); - checkState(channel instanceof GoogleCloudStorageWriteChannel); - ((GoogleCloudStorageWriteChannel) channel).setUploadBufferSize(CHANNEL_BUFFER); - return channel; + //TODO Decide what to do about this one +// WritableByteChannel channel = +// GcsIOChannelFactory.fromOptions(options).create(filename, "text/plain"); +// checkState(channel instanceof GoogleCloudStorageWriteChannel); +// ((GoogleCloudStorageWriteChannel) channel).setUploadBufferSize(CHANNEL_BUFFER); +// return channel; + throw new UnsupportedOperationException("Disabled after removal of GcsIOChannelFactory"); } /** Return a short string to describe {@code timing}. */ From 77eabbaaddad88784c8ce2e775b4b8e8fea3f868 Mon Sep 17 00:00:00 2001 From: Etienne Chauchot Date: Fri, 5 May 2017 15:19:07 +0200 Subject: [PATCH 307/346] Clean some code that is specific to Dataflow --- .../integration/nexmark/NexmarkRunner.java | 106 ------------------ 1 file changed, 106 deletions(-) diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkRunner.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkRunner.java index 6df76f092600c..935544e92bf3d 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkRunner.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkRunner.java @@ -157,9 +157,6 @@ public NexmarkRunner(OptionT options) { this.options = options; } - // ================================================================================ - // Overridden by each runner. - // ================================================================================ /** * Is this query running in streaming mode? @@ -414,7 +411,6 @@ private NexmarkPerf currentPerf( perf.shutdownDelaySec = (now - resultEnd) / 1000.0; } - perf.jobId = getJobId(result); // As soon as available, try to capture cumulative cost at this point too. NexmarkPerf.ProgressSnapshot snapshot = new NexmarkPerf.ProgressSnapshot(); @@ -429,105 +425,6 @@ private NexmarkPerf currentPerf( return perf; } - private String getJobId(PipelineResult job) { - return ""; - } - - // TODO specific to dataflow, see if we can find an equivalent -/* - protected MetricType getMetricType(MetricUpdate metric) { - String metricName = metric.getKey().metricName().name(); - if (metricName.endsWith("windmill-system-watermark")) { - return MetricType.SYSTEM_WATERMARK; - } else if (metricName.endsWith("windmill-data-watermark")) { - return MetricType.DATA_WATERMARK; - } else { - return MetricType.OTHER; - } - } -*/ - - /** - * Check that watermarks are not too far behind. - * - *

              Returns a list of errors detected. - */ - // TODO specific to dataflow, see if we can find an equivalent - /* - private List checkWatermarks(DataflowPipelineJob job, long startMsSinceEpoch) { - long now = System.currentTimeMillis(); - List errors = new ArrayList<>(); - try { - JobMetrics metricResponse = job.getDataflowClient() - .projects() - .jobs() - .getMetrics(job.getProjectId(), job.getJobId()) - .execute(); - List metrics = metricResponse.getMetrics(); - - - - if (metrics != null) { - boolean foundWatermarks = false; - for (MetricUpdate metric : metrics) { - MetricType type = getMetricType(metric); - if (type == MetricType.OTHER) { - continue; - } - foundWatermarks = true; - @SuppressWarnings("unchecked") - BigDecimal scalar = (BigDecimal) metric.getScalar(); - if (scalar.signum() < 0) { - continue; - } - Instant value = - new Instant(scalar.divideToIntegralValue(new BigDecimal(1000)).longValueExact()); - Instant updateTime = Instant.parse(metric.getUpdateTime()); - - if (options.getWatermarkValidationDelaySeconds() == null - || now > startMsSinceEpoch - + Duration.standardSeconds(options.getWatermarkValidationDelaySeconds()) - .getMillis()) { - Duration threshold = null; - if (type == MetricType.SYSTEM_WATERMARK && options.getMaxSystemLagSeconds() != null) { - threshold = Duration.standardSeconds(options.getMaxSystemLagSeconds()); - } else if (type == MetricType.DATA_WATERMARK - && options.getMaxDataLagSeconds() != null) { - threshold = Duration.standardSeconds(options.getMaxDataLagSeconds()); - } - - if (threshold != null && value.isBefore(updateTime.minus(threshold))) { - String msg = String.format("High lag for %s: %s vs %s (allowed lag of %s)", - metric.getKey().metricName().name(), value, updateTime, threshold); - errors.add(msg); - NexmarkUtils.console(msg); - } - } - } - if (!foundWatermarks) { - NexmarkUtils.console("No known watermarks in update: " + metrics); - if (now > startMsSinceEpoch + Duration.standardMinutes(5).getMillis()) { - errors.add("No known watermarks found. Metrics were " + metrics); - } - } - } - } catch (IOException e) { - NexmarkUtils.console("Warning: failed to get JobMetrics: " + e); - } - - return errors; - } -*/ - - // TODO specific to dataflow, see if we can find an equivalent -/* - enum MetricType { - SYSTEM_WATERMARK, - DATA_WATERMARK, - OTHER - } -*/ - /** * Build and run a pipeline using specified options. */ @@ -643,9 +540,6 @@ private NexmarkPerf monitor(NexmarkQuery query) { String.format("Streaming query was stuck for %d min", quietFor.getStandardMinutes())); } - // TODO specific to dataflow, see if we can find an equivalent -// errors.addAll(checkWatermarks(job, startMsSinceEpoch)); - if (waitingForShutdown) { try { job.cancel(); From 683680b1655e79d696a1d0f4588753a7d8ff2b82 Mon Sep 17 00:00:00 2001 From: Etienne Chauchot Date: Tue, 9 May 2017 10:17:06 +0200 Subject: [PATCH 308/346] Rename NexmarkDriver to Main and NexmarkRunner to NexmarkLauncher --- .../nexmark/{NexmarkDriver.java => Main.java} | 10 +++++----- .../{NexmarkRunner.java => NexmarkLauncher.java} | 4 ++-- 2 files changed, 7 insertions(+), 7 deletions(-) rename integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/{NexmarkDriver.java => Main.java} (97%) rename integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/{NexmarkRunner.java => NexmarkLauncher.java} (99%) diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkDriver.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Main.java similarity index 97% rename from integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkDriver.java rename to integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Main.java index a982a8d7bf692..da4d446387531 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkDriver.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Main.java @@ -52,12 +52,12 @@ *

              See * http://datalab.cs.pdx.edu/niagaraST/NEXMark/ */ -public class NexmarkDriver { +public class Main { /** * Entry point. */ - void runAll(OptionT options, NexmarkRunner runner) { + void runAll(OptionT options, NexmarkLauncher nexmarkLauncher) { Instant start = Instant.now(); Map baseline = loadBaseline(options.getBaselineFilename()); Map actual = new LinkedHashMap<>(); @@ -67,7 +67,7 @@ void runAll(OptionT options, NexmarkRunner runner) { try { // Run all the configurations. for (NexmarkConfiguration configuration : configurations) { - NexmarkPerf perf = runner.run(configuration); + NexmarkPerf perf = nexmarkLauncher.run(configuration); if (perf != null) { if (perf.errors == null || perf.errors.size() > 0) { successful = false; @@ -298,7 +298,7 @@ public static void main(String[] args) { NexmarkOptions options = PipelineOptionsFactory.fromArgs(args) .withValidation() .as(NexmarkOptions.class); - NexmarkRunner runner = new NexmarkRunner<>(options); - new NexmarkDriver<>().runAll(options, runner); + NexmarkLauncher nexmarkLauncher = new NexmarkLauncher<>(options); + new Main<>().runAll(options, nexmarkLauncher); } } diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkRunner.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkLauncher.java similarity index 99% rename from integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkRunner.java rename to integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkLauncher.java index 935544e92bf3d..ea4ff586ad2a5 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkRunner.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkLauncher.java @@ -91,7 +91,7 @@ /** * Run a single Nexmark query using a given configuration. */ -public class NexmarkRunner { +public class NexmarkLauncher { /** * Minimum number of samples needed for 'stead-state' rate calculation. */ @@ -153,7 +153,7 @@ public class NexmarkRunner { @Nullable private String queryName; - public NexmarkRunner(OptionT options) { + public NexmarkLauncher(OptionT options) { this.options = options; } From dbd1b155c32c19ce7a6d0c0f0dffb318c9ccdde7 Mon Sep 17 00:00:00 2001 From: Etienne Chauchot Date: Tue, 9 May 2017 11:48:00 +0200 Subject: [PATCH 309/346] Change benchmark workload settings Update configuration of events generation to add some variation Update execution matrix (issue #45) --- integration/java/nexmark/README.md | 207 +++++++++++------- .../nexmark/NexmarkConfiguration.java | 10 +- 2 files changed, 128 insertions(+), 89 deletions(-) diff --git a/integration/java/nexmark/README.md b/integration/java/nexmark/README.md index a3549f4173987..a9acd63c51708 100644 --- a/integration/java/nexmark/README.md +++ b/integration/java/nexmark/README.md @@ -30,14 +30,14 @@ These are multiple queries over a three entities model representing on online au - **Auction** represents an item under auction. - **Bid** represents a bid for an item under auction. -The queries exercise many aspects of dataflow model on Beam: +The queries exercise many aspects of Beam model: * **Query1**: What are the bid values in Euro's? Illustrates a simple map. * **Query2**: What are the auctions with particular auction numbers? Illustrates a simple filter. * **Query3**: Who is selling in particular US states? - Illustrates an incremental join (using per-key state) and filter. + Illustrates an incremental join (using per-key state and timer) and filter. * **Query4**: What is the average selling price for each auction category? Illustrates complex join (using custom window functions) and @@ -71,19 +71,17 @@ We have augmented the original queries with five more: compared with event time in non-Global windows for all the other queries. -The queries can be executed using a 'Driver' for a given backend. -Currently the supported drivers are: +We can specify the Beam runner to use with maven profiles, available profiles are: -* **NexmarkApexDriver** for running via the Apex runner. -* **NexmarkDirectDriver** for running locally on a single machine. -* **NexmarkGoogleDriver** for running on the Google Cloud Dataflow service. - Requires a Google Cloud account. -* **NexmarkFlinkDriver** for running on a Flink cluster. Requires the - cluster to be established and the Nexmark jar to be distributed to - each worker. -* **NexmarkSparkDriver** for running on a Spark cluster. +* direct-runner +* spark-runner +* flink-runner +* apex-runner + +The runner must also be specified like in any other Beam pipeline using + + --runner -Other drivers are straightforward. Test data is deterministically synthesized on demand. The test data may be synthesized in the same pipeline as the query itself, @@ -97,11 +95,6 @@ The query results may be: * Send to BigQuery. * Discarded. -Options are provided for measuring progress, measuring overall -pipeline performance, and comparing that performance against a known -baseline. However that machinery has only been implemented against -the Google Cloud Dataflow driver. - # Configuration ## Common configuration parameters @@ -119,45 +112,48 @@ Run query N --query=N ## Available Suites +The suite to run can be chosen using this configuration parameter: -- DEFAULT: Test default configuration with query 0. -- SMOKE: Run the 12 default configurations. -- STRESS: Like smoke but for 1m events. -- FULL_THROTTLE: Like SMOKE but 100m events. + --suite=SUITE - --suite=SMOKE +Available suites are: +* DEFAULT: Test default configuration with query 0. +* SMOKE: Run the 12 default configurations. +* STRESS: Like smoke but for 1m events. +* FULL_THROTTLE: Like SMOKE but 100m events. -### Apex specific configuration + - --suite=SMOKE --manageResources=false --monitorJobs=true +## Apex specific configuration -### Dataflow specific configuration + --manageResources=false --monitorJobs=false - --query=0 --suite=SMOKE --manageResources=false --monitorJobs=true \ +## Dataflow specific configuration + + --manageResources=false --monitorJobs=true \ --enforceEncodability=false --enforceImmutability=false --project= \ --zone= \ --workerMachineType=n1-highmem-8 \ - --stagingLocation= - - --runner=BlockingDataflowRunner \ + --stagingLocation= \ + --runner=DataflowRunner \ --tempLocation=gs://talend-imejia/nexmark/temp/ \ --stagingLocation=gs://talend-imejia/nexmark/temp/staging/ \ --filesToStage=target/beam-integration-java-0.7.0-SNAPSHOT.jar -### Direct specific configuration +## Direct specific configuration - --suite=SMOKE --manageResources=false --monitorJobs=true \ + --manageResources=false --monitorJobs=true \ --enforceEncodability=false --enforceImmutability=false -### Flink specific configuration +## Flink specific configuration - --suite=SMOKE --manageResources=false --monitorJobs=true \ - --flinkMaster=[local] --parallelism=#numcores + --manageResources=false --monitorJobs=true \ + --flinkMaster=local --parallelism=#numcores -### Spark specific configuration +## Spark specific configuration - --suite=SMOKE --manageResources=false --monitorJobs=true \ + --manageResources=false --monitorJobs=true \ --sparkMaster=local \ -Dspark.ui.enabled=false -DSPARK_LOCAL_IP=localhost -Dsun.io.serialization.extendedDebugInfo=true @@ -167,39 +163,39 @@ Open issues are tracked [here](https://github.com../../../../../issues): ## Batch / Synthetic / Local -| Query | Direct | Spark | Flink | Apex | -| ----: | ------------------------------ | ------------------------------ | ------------------------------ | ------------------------------- | -| 0 | ok | ok | ok | ok | -| 1 | ok | ok | ok | ok | -| 2 | ok | ok | ok | ok | -| 3 | [#7](../../../../../issues/7) | [#7](../../../../../issues/7) | [#7](../../../../../issues/7) | [#7](../../../../../issues/7) | -| 4 | ok | ok | [#2](../../../../../issues/2) | ok | -| 5 | ok | ok | ok | ok | -| 6 | ok | ok | [#2](../../../../../issues/2) | ok | -| 7 | ok | ok | ok | [#24](../../../../../issues/24) | -| 8 | ok | ok | ok | ok | -| 9 | ok | ok | [#2](../../../../../issues/2) | ok | -| 10 | [#5](../../../../../issues/5) | ok | ok | ok | -| 11 | ok | ok | ok | ok | -| 12 | ok | ok | ok | ok | +| Query | Direct | Spark | Flink | Apex | +| ----: | ------ | ------------------------------------------------------------ | ---------------------------------------------------------- | ------------------------------------------------------------ | +| 0 | ok | ok | ok | ok | +| 1 | ok | ok | ok | ok | +| 2 | ok | ok | ok | ok | +| 3 | ok | [BEAM-1035](https://issues.apache.org/jira/browse/BEAM-1035) | ok | [BEAM-1037](https://issues.apache.org/jira/browse/BEAM-1037) | +| 4 | ok | ok | ok | ok | +| 5 | ok | ok | ok | ok | +| 6 | ok | ok | ok | ok | +| 7 | ok | ok | ok | ok | +| 8 | ok | ok | ok | ok | +| 9 | ok | ok | ok | ok | +| 10 | ok | ok | ok | ok | +| 11 | ok | ok | ok | ok | +| 12 | ok | ok | ok | ok | ## Streaming / Synthetic / Local -| Query | Direct | Spark | Flink | Apex | -| ----: | ------------------------------ | ------------------------------ | ------------------------------ | ------------------------------ | -| 0 | ok | | | ok | -| 1 | ok | | | ok | -| 2 | ok | | | ok | -| 3 | [#7](../../../../../issues/7) | | | [#7](../../../../../issues/7) | -| 4 | ok | | | ok | -| 5 | ok | | | ok | -| 6 | ok | | | ok | -| 7 | ok | | | ? | -| 8 | ok | | | ok | -| 9 | ok | | | ok | -| 10 | [#5](../../../../../issues/5) | | | ? | -| 11 | ok | | | Ok | -| 12 | ok | | | Ok | +| Query | Direct | Spark | Flink | Apex | +| ----: | ------ | ------------------------------------------------------------ | ---------------------------------------------------------- | ------------------------------------------------------------ | +| 0 | ok | ok | [BEAM-593](https://issues.apache.org/jira/browse/BEAM-593) | ok | +| 1 | ok | ok | [BEAM-593](https://issues.apache.org/jira/browse/BEAM-593) | ok | +| 2 | ok | ok | [BEAM-593](https://issues.apache.org/jira/browse/BEAM-593) | ok | +| 3 | ok | [BEAM-1035](https://issues.apache.org/jira/browse/BEAM-1035) | [BEAM-593](https://issues.apache.org/jira/browse/BEAM-593) | [BEAM-1037](https://issues.apache.org/jira/browse/BEAM-1037) | +| 4 | ok | ok | [BEAM-593](https://issues.apache.org/jira/browse/BEAM-593) | ok | +| 5 | ok | ok | [BEAM-593](https://issues.apache.org/jira/browse/BEAM-593) | ok | +| 6 | ok | ok | [BEAM-593](https://issues.apache.org/jira/browse/BEAM-593) | ok | +| 7 | ok | [BEAM-2112](https://issues.apache.org/jira/browse/BEAM-2112) | [BEAM-593](https://issues.apache.org/jira/browse/BEAM-593) | ok | +| 8 | ok | ok | [BEAM-593](https://issues.apache.org/jira/browse/BEAM-593) | ok | +| 9 | ok | ok | [BEAM-593](https://issues.apache.org/jira/browse/BEAM-593) | ok | +| 10 | ok | ok | [BEAM-593](https://issues.apache.org/jira/browse/BEAM-593) | ok | +| 11 | ok | ok | [BEAM-593](https://issues.apache.org/jira/browse/BEAM-593) | ok | +| 12 | ok | ok | [BEAM-593](https://issues.apache.org/jira/browse/BEAM-593) | ok | ## Batch / Synthetic / Cluster @@ -219,26 +215,63 @@ TODO # Running Nexmark -## Running on the DirectRunner (local) +## Running SMOKE suite on the DirectRunner (local) Batch Mode --Dexec.classpathScope="test" + mvn exec:java -Dexec.mainClass=org.apache.beam.integration.nexmark.Main -Pdirect-runner -Dexec.args="--runner=DirectRunner --suite=SMOKE --streaming=false --manageResources=false --monitorJobs=true --enforceEncodability=true --enforceImmutability=true" + +Streaming Mode + + mvn exec:java -Dexec.mainClass=org.apache.beam.integration.nexmark.Main -Pdirect-runner -Dexec.args="--runner=DirectRunner --suite=SMOKE --streaming=true --manageResources=false --monitorJobs=true --enforceEncodability=true --enforceImmutability=true" + - mvn exec:java -Dexec.mainClass=org.apache.beam.integration.nexmark.drivers.NexmarkDirectDriver -Dexec.args="--suite=SMOKE --streaming=false --numEventGenerators=4 --manageResources=false --monitorJobs=false --enforceEncodability=false --enforceImmutability=false" +## Running SMOKE suite on the SparkRunner (local) + +Batch Mode + + mvn exec:java -Dexec.mainClass=org.apache.beam.integration.nexmark.Main -Pspark-runner "-Dexec.args=--runner=SparkRunner --suite=SMOKE --streamTimeout=60 --streaming=false --manageResources=false --monitorJobs=true" Streaming Mode - mvn exec:java -Dexec.mainClass=org.apache.beam.integration.nexmark.drivers.NexmarkDirectDriver -Dexec.args="--suite=SMOKE --streaming=true --numEventGenerators=4 --manageResources=false --monitorJobs=false --enforceEncodability=false --enforceImmutability=false" + mvn exec:java -Dexec.mainClass=org.apache.beam.integration.nexmark.Main -Pspark-runner "-Dexec.args=--runner=SparkRunner --suite=SMOKE --streamTimeout=60 --streaming=true --manageResources=false --monitorJobs=true" + + +## Running SMOKE suite on the FlinkRunner (local) -## Running on Google Cloud Dataflow +Batch Mode + + mvn exec:java -Dexec.mainClass=org.apache.beam.integration.nexmark.Main -Pflink-runner "-Dexec.args=--runner=FlinkRunner --suite=SMOKE --streamTimeout=60 --streaming=false --manageResources=false --monitorJobs=true --flinkMaster=local" + +Streaming Mode + + mvn exec:java -Dexec.mainClass=org.apache.beam.integration.nexmark.Main -Pflink-runner "-Dexec.args=--runner=FlinkRunner --suite=SMOKE --streamTimeout=60 --streaming=true --manageResources=false --monitorJobs=true --flinkMaster=local" + + +## Running SMOKE suite on the ApexRunner (local) + +Batch Mode + + mvn exec:java -Dexec.mainClass=org.apache.beam.integration.nexmark.Main -Papex-runner "-Dexec.args=--runner=ApexRunner --suite=SMOKE --streamTimeout=60 --streaming=false --manageResources=false --monitorJobs=false" + +Streaming Mode + + mvn exec:java -Dexec.mainClass=org.apache.beam.integration.nexmark.Main -Papex-runner "-Dexec.args=--runner=ApexRunner --suite=SMOKE --streamTimeout=60 --streaming=true --manageResources=false --monitorJobs=false" + + +## Running SMOKE suite on Google Cloud Dataflow + +Building package + + mvn clean package -Pdataflow-runner + +Submit to Google Dataflow service -An example invocation for **Query10** on the Google Cloud Dataflow -service. ``` -java -cp integration/java/target/java-integration-all-bundled-0.2.0-incubating-SNAPSHOT.jar \ - org.apache.beam.integration.nexmark.drivers.NexmarkGoogleDriver \ +java -cp integration/java/target/beam-integration-java-nexmark-bundled-2.1.0-SNAPSHOT.jar \ + org.apache.beam.integration.nexmark.Main \ + --runner=DataflowRunner --project= \ --zone= \ --workerMachineType=n1-highmem-8 \ @@ -253,7 +286,7 @@ java -cp integration/java/target/java-integration-all-bundled-0.2.0-incubating-S --numEventGenerators=64 \ --numWorkers=16 \ --maxNumWorkers=16 \ - --query=10 \ + --suite=SMOKE \ --firstEventRate=100000 \ --nextEventRate=100000 \ --ratePeriodSec=3600 \ @@ -270,8 +303,9 @@ java -cp integration/java/target/java-integration-all-bundled-0.2.0-incubating-S ``` ``` -java -cp integration/java/target/java-integration-all-bundled-0.2.0-incubating-SNAPSHOT.jar \ - org.apache.beam.integration.nexmark.drivers.NexmarkGoogleDriver \ +java -cp integration/java/target/beam-integration-java-nexmark-bundled-2.1.0-SNAPSHOT.jar \ + org.apache.beam.integration.nexmark.Main \ + --runner=DataflowRunner --project= \ --zone= \ --workerMachineType=n1-highmem-8 \ @@ -285,7 +319,7 @@ java -cp integration/java/target/java-integration-all-bundled-0.2.0-incubating-S --monitorJobs=false \ --numWorkers=64 \ --maxNumWorkers=64 \ - --query=10 \ + --suite=SMOKE \ --usePubsubPublishTime=true \ --outputPath= \ --windowSizeSec=600 \ @@ -294,8 +328,13 @@ java -cp integration/java/target/java-integration-all-bundled-0.2.0-incubating-S --experiments=enable_custom_pubsub_source ``` -## Running on Flink +## Running query 0 on a Spark cluster with yarn + +Building package + + mvn clean package -Pspark-runner + +Submit to the cluster + + spark-submit --master yarn-client --class org.apache.beam.integration.nexmark.Main --driver-memory 512m --executor-memory 512m --executor-cores 1 beam-integration-java-nexmark-bundled-2.1.0-SNAPSHOT.jar --runner=SparkRunner --query=0 --streamTimeout=60 --streaming=false --manageResources=false --monitorJobs=true -See [BEAM_ON_FLINK_ON_GCP](./BEAM_ON_FLINK_ON_GCP.md) for instructions -on running a NexMark pipeline using Flink hosted on a Google Compute -Platform cluster. diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkConfiguration.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkConfiguration.java index 1da08b410cbde..5a8cb7182d01b 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkConfiguration.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkConfiguration.java @@ -140,15 +140,15 @@ public class NexmarkConfiguration implements Serializable { /** Ratio of bids to 'hot' auctions compared to all other auctions. */ @JsonProperty - public int hotAuctionRatio = 1; + public int hotAuctionRatio = 2; /** Ratio of auctions for 'hot' sellers compared to all other people. */ @JsonProperty - public int hotSellersRatio = 1; + public int hotSellersRatio = 4; /** Ratio of bids for 'hot' bidders compared to all other people. */ @JsonProperty - public int hotBiddersRatio = 1; + public int hotBiddersRatio = 4; /** Window size, in seconds, for queries 3, 5, 7 and 8. */ @JsonProperty @@ -211,13 +211,13 @@ public class NexmarkConfiguration implements Serializable { * Length of occasional delay to impose on events (in seconds). */ @JsonProperty - public long occasionalDelaySec = 0; + public long occasionalDelaySec = 3; /** * Probability that an event will be delayed by delayS. */ @JsonProperty - public double probDelayedEvent = 0.0; + public double probDelayedEvent = 0.1; /** * Maximum size of each log file (in events). For Query10 only. From 69953a0b803896a982347b6bb821a922f6970d2b Mon Sep 17 00:00:00 2001 From: Etienne Chauchot Date: Tue, 9 May 2017 12:08:10 +0200 Subject: [PATCH 310/346] Remove references to dataflow in generic classes --- .../main/java/org/apache/beam/integration/nexmark/Main.java | 4 ++-- .../org/apache/beam/integration/nexmark/NexmarkOptions.java | 2 +- .../org/apache/beam/integration/nexmark/NexmarkUtils.java | 2 +- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Main.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Main.java index da4d446387531..4c23651c04c61 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Main.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Main.java @@ -36,7 +36,7 @@ import org.joda.time.Instant; /** - * An implementation of the 'NEXMark queries' for Google Dataflow. + * An implementation of the 'NEXMark queries' for Beam. * These are multiple queries over a three table schema representing an online auction system: *

              - * The queries exercise many aspects of streaming dataflow. + * The queries exercise many aspects of the Beam model. * *

              We synthesize the creation of people, auctions and bids in real-time. The data is not * particularly sensible. diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkOptions.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkOptions.java index f162fd6164538..9afffaa721a25 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkOptions.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkOptions.java @@ -37,7 +37,7 @@ public interface NexmarkOptions void setSuite(NexmarkSuite suite); - @Description("If true, and using the DataflowPipelineRunner, monitor the jobs as they run.") + @Description("If true, monitor the jobs as they run.") @Default.Boolean(false) boolean getMonitorJobs(); diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkUtils.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkUtils.java index f6215e9f15d6f..ea851af944abf 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkUtils.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkUtils.java @@ -289,7 +289,7 @@ public int stepLengthSec(int ratePeriodSec) { private static final boolean LOG_ERROR = true; /** - * Set to true to log directly to stdout on VM. You can watch the results in real-time with: + * Set to true to log directly to stdout. If run using Google Dataflow, you can watch the results in real-time with: * tail -f /var/log/dataflow/streaming-harness/harness-stdout.log */ private static final boolean LOG_TO_CONSOLE = false; From 3d5c3d009b441a8085189f9d4ed1926a4042f816 Mon Sep 17 00:00:00 2001 From: Etienne Chauchot Date: Tue, 9 May 2017 15:25:54 +0200 Subject: [PATCH 311/346] Migrate to Beam 2.1.0-SNAPSHOT --- integration/java/nexmark/pom.xml | 2 +- .../integration/nexmark/NexmarkLauncher.java | 4 +- .../integration/nexmark/NexmarkUtils.java | 43 ++++++++---------- .../integration/nexmark/model/Auction.java | 45 +++++++++---------- .../integration/nexmark/model/AuctionBid.java | 13 +++--- .../nexmark/model/AuctionCount.java | 14 +++--- .../nexmark/model/AuctionPrice.java | 13 +++--- .../beam/integration/nexmark/model/Bid.java | 25 +++++------ .../nexmark/model/BidsPerSession.java | 13 +++--- .../nexmark/model/CategoryPrice.java | 18 ++++---- .../beam/integration/nexmark/model/Done.java | 10 ++--- .../beam/integration/nexmark/model/Event.java | 24 +++++----- .../nexmark/model/IdNameReserve.java | 17 ++++--- .../nexmark/model/NameCityStateId.java | 22 +++++---- .../integration/nexmark/model/Person.java | 38 ++++++++-------- .../nexmark/model/SellerPrice.java | 13 +++--- .../nexmark/queries/WinningBids.java | 16 +++---- .../nexmark/sources/Generator.java | 19 +++----- integration/java/pom.xml | 2 +- integration/pom.xml | 2 +- 20 files changed, 163 insertions(+), 190 deletions(-) diff --git a/integration/java/nexmark/pom.xml b/integration/java/nexmark/pom.xml index 35fe0f330fb75..86b88bdb1dc72 100644 --- a/integration/java/nexmark/pom.xml +++ b/integration/java/nexmark/pom.xml @@ -23,7 +23,7 @@ org.apache.beam beam-integration-java-parent - 0.7.0-SNAPSHOT + 2.1.0-SNAPSHOT ../pom.xml diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkLauncher.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkLauncher.java index ea4ff586ad2a5..db53191be2e22 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkLauncher.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkLauncher.java @@ -739,7 +739,7 @@ private PCollection sourceEventsFromPubsub(Pipeline p, long now) { NexmarkUtils.console("Reading events from Pubsub %s", shortSubscription); PubsubIO.Read io = - PubsubIO.readPubsubMessagesWithAttributes().fromSubscription(shortSubscription) + PubsubIO.readMessagesWithAttributes().fromSubscription(shortSubscription) .withIdAttribute(NexmarkUtils.PUBSUB_ID); if (!configuration.usePubsubPublishTime) { io = io.withTimestampAttribute(NexmarkUtils.PUBSUB_TIMESTAMP); @@ -784,7 +784,7 @@ private void sinkEventsToPubsub(PCollection events, long now) { NexmarkUtils.console("Writing events to Pubsub %s", shortTopic); PubsubIO.Write io = - PubsubIO.writePubsubMessages().to(shortTopic) + PubsubIO.writeMessages().to(shortTopic) .withIdAttribute(NexmarkUtils.PUBSUB_ID); if (!configuration.usePubsubPublishTime) { io = io.withTimestampAttribute(NexmarkUtils.PUBSUB_TIMESTAMP); diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkUtils.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkUtils.java index ea851af944abf..7707429f18818 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkUtils.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkUtils.java @@ -351,25 +351,25 @@ public static void setupPipeline(CoderStrategy coderStrategy, Pipeline p) { CoderRegistry registry = p.getCoderRegistry(); switch (coderStrategy) { case HAND: - registry.registerCoder(Auction.class, Auction.CODER); - registry.registerCoder(AuctionBid.class, AuctionBid.CODER); - registry.registerCoder(AuctionCount.class, AuctionCount.CODER); - registry.registerCoder(AuctionPrice.class, AuctionPrice.CODER); - registry.registerCoder(Bid.class, Bid.CODER); - registry.registerCoder(CategoryPrice.class, CategoryPrice.CODER); - registry.registerCoder(Event.class, Event.CODER); - registry.registerCoder(IdNameReserve.class, IdNameReserve.CODER); - registry.registerCoder(NameCityStateId.class, NameCityStateId.CODER); - registry.registerCoder(Person.class, Person.CODER); - registry.registerCoder(SellerPrice.class, SellerPrice.CODER); - registry.registerCoder(Done.class, Done.CODER); - registry.registerCoder(BidsPerSession.class, BidsPerSession.CODER); + registry.registerCoderForClass(Auction.class, Auction.CODER); + registry.registerCoderForClass(AuctionBid.class, AuctionBid.CODER); + registry.registerCoderForClass(AuctionCount.class, AuctionCount.CODER); + registry.registerCoderForClass(AuctionPrice.class, AuctionPrice.CODER); + registry.registerCoderForClass(Bid.class, Bid.CODER); + registry.registerCoderForClass(CategoryPrice.class, CategoryPrice.CODER); + registry.registerCoderForClass(Event.class, Event.CODER); + registry.registerCoderForClass(IdNameReserve.class, IdNameReserve.CODER); + registry.registerCoderForClass(NameCityStateId.class, NameCityStateId.CODER); + registry.registerCoderForClass(Person.class, Person.CODER); + registry.registerCoderForClass(SellerPrice.class, SellerPrice.CODER); + registry.registerCoderForClass(Done.class, Done.CODER); + registry.registerCoderForClass(BidsPerSession.class, BidsPerSession.CODER); break; case AVRO: - registry.setFallbackCoderProvider(AvroCoder.PROVIDER); + registry.registerCoderProvider(AvroCoder.getCoderProvider()); break; case JAVA: - registry.setFallbackCoderProvider(SerializableCoder.PROVIDER); + registry.registerCoderProvider(SerializableCoder.getCoderProvider()); break; } } @@ -621,22 +621,17 @@ public CastingCoder(Coder trueCoder) { } @Override - public void encode(KnownSize value, OutputStream outStream, Context context) + public void encode(KnownSize value, OutputStream outStream) throws CoderException, IOException { @SuppressWarnings("unchecked") T typedValue = (T) value; - trueCoder.encode(typedValue, outStream, context); + trueCoder.encode(typedValue, outStream); } @Override - public KnownSize decode(InputStream inStream, Context context) + public KnownSize decode(InputStream inStream) throws CoderException, IOException { - return trueCoder.decode(inStream, context); - } - - @Override - public List> getComponents() { - return ImmutableList.of(trueCoder); + return trueCoder.decode(inStream); } } diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/Auction.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/Auction.java index 5c018dc45bb5f..9f5d7c0afdb60 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/Auction.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/Auction.java @@ -39,35 +39,34 @@ public class Auction implements KnownSize, Serializable { public static final Coder CODER = new CustomCoder() { @Override - public void encode(Auction value, OutputStream outStream, - Coder.Context context) + public void encode(Auction value, OutputStream outStream) throws CoderException, IOException { - LONG_CODER.encode(value.id, outStream, Context.NESTED); - STRING_CODER.encode(value.itemName, outStream, Context.NESTED); - STRING_CODER.encode(value.description, outStream, Context.NESTED); - LONG_CODER.encode(value.initialBid, outStream, Context.NESTED); - LONG_CODER.encode(value.reserve, outStream, Context.NESTED); - LONG_CODER.encode(value.dateTime, outStream, Context.NESTED); - LONG_CODER.encode(value.expires, outStream, Context.NESTED); - LONG_CODER.encode(value.seller, outStream, Context.NESTED); - LONG_CODER.encode(value.category, outStream, Context.NESTED); - STRING_CODER.encode(value.extra, outStream, Context.NESTED); + LONG_CODER.encode(value.id, outStream); + STRING_CODER.encode(value.itemName, outStream); + STRING_CODER.encode(value.description, outStream); + LONG_CODER.encode(value.initialBid, outStream); + LONG_CODER.encode(value.reserve, outStream); + LONG_CODER.encode(value.dateTime, outStream); + LONG_CODER.encode(value.expires, outStream); + LONG_CODER.encode(value.seller, outStream); + LONG_CODER.encode(value.category, outStream); + STRING_CODER.encode(value.extra, outStream); } @Override public Auction decode( - InputStream inStream, Coder.Context context) + InputStream inStream) throws CoderException, IOException { - long id = LONG_CODER.decode(inStream, Context.NESTED); - String itemName = STRING_CODER.decode(inStream, Context.NESTED); - String description = STRING_CODER.decode(inStream, Context.NESTED); - long initialBid = LONG_CODER.decode(inStream, Context.NESTED); - long reserve = LONG_CODER.decode(inStream, Context.NESTED); - long dateTime = LONG_CODER.decode(inStream, Context.NESTED); - long expires = LONG_CODER.decode(inStream, Context.NESTED); - long seller = LONG_CODER.decode(inStream, Context.NESTED); - long category = LONG_CODER.decode(inStream, Context.NESTED); - String extra = STRING_CODER.decode(inStream, Context.NESTED); + long id = LONG_CODER.decode(inStream); + String itemName = STRING_CODER.decode(inStream); + String description = STRING_CODER.decode(inStream); + long initialBid = LONG_CODER.decode(inStream); + long reserve = LONG_CODER.decode(inStream); + long dateTime = LONG_CODER.decode(inStream); + long expires = LONG_CODER.decode(inStream); + long seller = LONG_CODER.decode(inStream); + long category = LONG_CODER.decode(inStream); + String extra = STRING_CODER.decode(inStream); return new Auction( id, itemName, description, initialBid, reserve, dateTime, expires, seller, category, extra); diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/AuctionBid.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/AuctionBid.java index b1d9ec2c5055d..b9d79db8b9c0a 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/AuctionBid.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/AuctionBid.java @@ -34,19 +34,18 @@ public class AuctionBid implements KnownSize, Serializable { public static final Coder CODER = new CustomCoder() { @Override - public void encode(AuctionBid value, OutputStream outStream, - Coder.Context context) + public void encode(AuctionBid value, OutputStream outStream) throws CoderException, IOException { - Auction.CODER.encode(value.auction, outStream, Context.NESTED); - Bid.CODER.encode(value.bid, outStream, Context.NESTED); + Auction.CODER.encode(value.auction, outStream); + Bid.CODER.encode(value.bid, outStream); } @Override public AuctionBid decode( - InputStream inStream, Coder.Context context) + InputStream inStream) throws CoderException, IOException { - Auction auction = Auction.CODER.decode(inStream, Context.NESTED); - Bid bid = Bid.CODER.decode(inStream, Context.NESTED); + Auction auction = Auction.CODER.decode(inStream); + Bid bid = Bid.CODER.decode(inStream); return new AuctionBid(auction, bid); } }; diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/AuctionCount.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/AuctionCount.java index c83a4554fbaf8..0e643ff152770 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/AuctionCount.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/AuctionCount.java @@ -37,19 +37,17 @@ public class AuctionCount implements KnownSize, Serializable { public static final Coder CODER = new CustomCoder() { @Override - public void encode(AuctionCount value, OutputStream outStream, - Coder.Context context) + public void encode(AuctionCount value, OutputStream outStream) throws CoderException, IOException { - LONG_CODER.encode(value.auction, outStream, Context.NESTED); - LONG_CODER.encode(value.count, outStream, Context.NESTED); + LONG_CODER.encode(value.auction, outStream); + LONG_CODER.encode(value.count, outStream); } @Override - public AuctionCount decode( - InputStream inStream, Coder.Context context) + public AuctionCount decode(InputStream inStream) throws CoderException, IOException { - long auction = LONG_CODER.decode(inStream, Context.NESTED); - long count = LONG_CODER.decode(inStream, Context.NESTED); + long auction = LONG_CODER.decode(inStream); + long count = LONG_CODER.decode(inStream); return new AuctionCount(auction, count); } }; diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/AuctionPrice.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/AuctionPrice.java index 43d0b275dae4e..7d51a21385b79 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/AuctionPrice.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/AuctionPrice.java @@ -37,19 +37,18 @@ public class AuctionPrice implements KnownSize, Serializable { public static final Coder CODER = new CustomCoder() { @Override - public void encode(AuctionPrice value, OutputStream outStream, - Coder.Context context) + public void encode(AuctionPrice value, OutputStream outStream) throws CoderException, IOException { - LONG_CODER.encode(value.auction, outStream, Context.NESTED); - LONG_CODER.encode(value.price, outStream, Context.NESTED); + LONG_CODER.encode(value.auction, outStream); + LONG_CODER.encode(value.price, outStream); } @Override public AuctionPrice decode( - InputStream inStream, Coder.Context context) + InputStream inStream) throws CoderException, IOException { - long auction = LONG_CODER.decode(inStream, Context.NESTED); - long price = LONG_CODER.decode(inStream, Context.NESTED); + long auction = LONG_CODER.decode(inStream); + long price = LONG_CODER.decode(inStream); return new AuctionPrice(auction, price); } }; diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/Bid.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/Bid.java index faeb928307cc0..4fa9ea07c829f 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/Bid.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/Bid.java @@ -40,25 +40,24 @@ public class Bid implements KnownSize, Serializable { public static final Coder CODER = new CustomCoder() { @Override - public void encode(Bid value, OutputStream outStream, - Coder.Context context) + public void encode(Bid value, OutputStream outStream) throws CoderException, IOException { - LONG_CODER.encode(value.auction, outStream, Context.NESTED); - LONG_CODER.encode(value.bidder, outStream, Context.NESTED); - LONG_CODER.encode(value.price, outStream, Context.NESTED); - LONG_CODER.encode(value.dateTime, outStream, Context.NESTED); - STRING_CODER.encode(value.extra, outStream, Context.NESTED); + LONG_CODER.encode(value.auction, outStream); + LONG_CODER.encode(value.bidder, outStream); + LONG_CODER.encode(value.price, outStream); + LONG_CODER.encode(value.dateTime, outStream); + STRING_CODER.encode(value.extra, outStream); } @Override public Bid decode( - InputStream inStream, Coder.Context context) + InputStream inStream) throws CoderException, IOException { - long auction = LONG_CODER.decode(inStream, Context.NESTED); - long bidder = LONG_CODER.decode(inStream, Context.NESTED); - long price = LONG_CODER.decode(inStream, Context.NESTED); - long dateTime = LONG_CODER.decode(inStream, Context.NESTED); - String extra = STRING_CODER.decode(inStream, Context.NESTED); + long auction = LONG_CODER.decode(inStream); + long bidder = LONG_CODER.decode(inStream); + long price = LONG_CODER.decode(inStream); + long dateTime = LONG_CODER.decode(inStream); + String extra = STRING_CODER.decode(inStream); return new Bid(auction, bidder, price, dateTime, extra); } diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/BidsPerSession.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/BidsPerSession.java index 6dddf34598df3..3211456fe6a8c 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/BidsPerSession.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/BidsPerSession.java @@ -37,19 +37,18 @@ public class BidsPerSession implements KnownSize, Serializable { public static final Coder CODER = new CustomCoder() { @Override - public void encode(BidsPerSession value, OutputStream outStream, - Coder.Context context) + public void encode(BidsPerSession value, OutputStream outStream) throws CoderException, IOException { - LONG_CODER.encode(value.personId, outStream, Context.NESTED); - LONG_CODER.encode(value.bidsPerSession, outStream, Context.NESTED); + LONG_CODER.encode(value.personId, outStream); + LONG_CODER.encode(value.bidsPerSession, outStream); } @Override public BidsPerSession decode( - InputStream inStream, Coder.Context context) + InputStream inStream) throws CoderException, IOException { - long personId = LONG_CODER.decode(inStream, Context.NESTED); - long bidsPerSession = LONG_CODER.decode(inStream, Context.NESTED); + long personId = LONG_CODER.decode(inStream); + long bidsPerSession = LONG_CODER.decode(inStream); return new BidsPerSession(personId, bidsPerSession); } @Override public void verifyDeterministic() throws NonDeterministicException {} diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/CategoryPrice.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/CategoryPrice.java index ccb2bc7ed4228..2678198470a29 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/CategoryPrice.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/CategoryPrice.java @@ -39,21 +39,19 @@ public class CategoryPrice implements KnownSize, Serializable { public static final Coder CODER = new CustomCoder() { @Override - public void encode(CategoryPrice value, OutputStream outStream, - Coder.Context context) + public void encode(CategoryPrice value, OutputStream outStream) throws CoderException, IOException { - LONG_CODER.encode(value.category, outStream, Context.NESTED); - LONG_CODER.encode(value.price, outStream, Context.NESTED); - INT_CODER.encode(value.isLast ? 1 : 0, outStream, Context.NESTED); + LONG_CODER.encode(value.category, outStream); + LONG_CODER.encode(value.price, outStream); + INT_CODER.encode(value.isLast ? 1 : 0, outStream); } @Override - public CategoryPrice decode( - InputStream inStream, Coder.Context context) + public CategoryPrice decode(InputStream inStream) throws CoderException, IOException { - long category = LONG_CODER.decode(inStream, Context.NESTED); - long price = LONG_CODER.decode(inStream, Context.NESTED); - boolean isLast = INT_CODER.decode(inStream, context) != 0; + long category = LONG_CODER.decode(inStream); + long price = LONG_CODER.decode(inStream); + boolean isLast = INT_CODER.decode(inStream) != 0; return new CategoryPrice(category, price, isLast); } @Override public void verifyDeterministic() throws NonDeterministicException {} diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/Done.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/Done.java index 0c14e8f5fb1e8..b0a88d4642feb 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/Done.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/Done.java @@ -37,17 +37,15 @@ public class Done implements KnownSize, Serializable { public static final Coder CODER = new CustomCoder() { @Override - public void encode(Done value, OutputStream outStream, - Coder.Context context) + public void encode(Done value, OutputStream outStream) throws CoderException, IOException { - STRING_CODER.encode(value.message, outStream, Context.NESTED); + STRING_CODER.encode(value.message, outStream); } @Override - public Done decode( - InputStream inStream, Coder.Context context) + public Done decode(InputStream inStream) throws CoderException, IOException { - String message = STRING_CODER.decode(inStream, Context.NESTED); + String message = STRING_CODER.decode(inStream); return new Done(message); } @Override public void verifyDeterministic() throws NonDeterministicException {} diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/Event.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/Event.java index 1f1f096a8bc02..d8138331bc8d7 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/Event.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/Event.java @@ -36,17 +36,17 @@ public class Event implements KnownSize, Serializable { public static final Coder CODER = new CustomCoder() { @Override - public void encode(Event value, OutputStream outStream, Coder.Context context) + public void encode(Event value, OutputStream outStream) throws CoderException, IOException { if (value.newPerson != null) { - INT_CODER.encode(0, outStream, Context.NESTED); - Person.CODER.encode(value.newPerson, outStream, Context.NESTED); + INT_CODER.encode(0, outStream); + Person.CODER.encode(value.newPerson, outStream); } else if (value.newAuction != null) { - INT_CODER.encode(1, outStream, Context.NESTED); - Auction.CODER.encode(value.newAuction, outStream, Context.NESTED); + INT_CODER.encode(1, outStream); + Auction.CODER.encode(value.newAuction, outStream); } else if (value.bid != null) { - INT_CODER.encode(2, outStream, Context.NESTED); - Bid.CODER.encode(value.bid, outStream, Context.NESTED); + INT_CODER.encode(2, outStream); + Bid.CODER.encode(value.bid, outStream); } else { throw new RuntimeException("invalid event"); } @@ -54,17 +54,17 @@ public void encode(Event value, OutputStream outStream, Coder.Context context) @Override public Event decode( - InputStream inStream, Coder.Context context) + InputStream inStream) throws CoderException, IOException { - int tag = INT_CODER.decode(inStream, context); + int tag = INT_CODER.decode(inStream); if (tag == 0) { - Person person = Person.CODER.decode(inStream, Context.NESTED); + Person person = Person.CODER.decode(inStream); return new Event(person); } else if (tag == 1) { - Auction auction = Auction.CODER.decode(inStream, Context.NESTED); + Auction auction = Auction.CODER.decode(inStream); return new Event(auction); } else if (tag == 2) { - Bid bid = Bid.CODER.decode(inStream, Context.NESTED); + Bid bid = Bid.CODER.decode(inStream); return new Event(bid); } else { throw new RuntimeException("invalid event encoding"); diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/IdNameReserve.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/IdNameReserve.java index 17b8c4a9ac378..8cade4e8996da 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/IdNameReserve.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/IdNameReserve.java @@ -39,21 +39,20 @@ public class IdNameReserve implements KnownSize, Serializable { public static final Coder CODER = new CustomCoder() { @Override - public void encode(IdNameReserve value, OutputStream outStream, - Coder.Context context) + public void encode(IdNameReserve value, OutputStream outStream) throws CoderException, IOException { - LONG_CODER.encode(value.id, outStream, Context.NESTED); - STRING_CODER.encode(value.name, outStream, Context.NESTED); - LONG_CODER.encode(value.reserve, outStream, Context.NESTED); + LONG_CODER.encode(value.id, outStream); + STRING_CODER.encode(value.name, outStream); + LONG_CODER.encode(value.reserve, outStream); } @Override public IdNameReserve decode( - InputStream inStream, Coder.Context context) + InputStream inStream) throws CoderException, IOException { - long id = LONG_CODER.decode(inStream, Context.NESTED); - String name = STRING_CODER.decode(inStream, Context.NESTED); - long reserve = LONG_CODER.decode(inStream, Context.NESTED); + long id = LONG_CODER.decode(inStream); + String name = STRING_CODER.decode(inStream); + long reserve = LONG_CODER.decode(inStream); return new IdNameReserve(id, name, reserve); } @Override public void verifyDeterministic() throws NonDeterministicException {} diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/NameCityStateId.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/NameCityStateId.java index 28f25cd3a49f0..37bd3c69468f7 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/NameCityStateId.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/NameCityStateId.java @@ -39,23 +39,21 @@ public class NameCityStateId implements KnownSize, Serializable { public static final Coder CODER = new CustomCoder() { @Override - public void encode(NameCityStateId value, OutputStream outStream, - Coder.Context context) + public void encode(NameCityStateId value, OutputStream outStream) throws CoderException, IOException { - STRING_CODER.encode(value.name, outStream, Context.NESTED); - STRING_CODER.encode(value.city, outStream, Context.NESTED); - STRING_CODER.encode(value.state, outStream, Context.NESTED); - LONG_CODER.encode(value.id, outStream, Context.NESTED); + STRING_CODER.encode(value.name, outStream); + STRING_CODER.encode(value.city, outStream); + STRING_CODER.encode(value.state, outStream); + LONG_CODER.encode(value.id, outStream); } @Override - public NameCityStateId decode( - InputStream inStream, Coder.Context context) + public NameCityStateId decode(InputStream inStream) throws CoderException, IOException { - String name = STRING_CODER.decode(inStream, Context.NESTED); - String city = STRING_CODER.decode(inStream, Context.NESTED); - String state = STRING_CODER.decode(inStream, Context.NESTED); - long id = LONG_CODER.decode(inStream, Context.NESTED); + String name = STRING_CODER.decode(inStream); + String city = STRING_CODER.decode(inStream); + String state = STRING_CODER.decode(inStream); + long id = LONG_CODER.decode(inStream); return new NameCityStateId(name, city, state, id); } @Override public void verifyDeterministic() throws NonDeterministicException {} diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/Person.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/Person.java index c690fd445a56e..bde587dd5e345 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/Person.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/Person.java @@ -38,31 +38,29 @@ public class Person implements KnownSize, Serializable { private static final Coder STRING_CODER = StringUtf8Coder.of(); public static final Coder CODER = new CustomCoder() { @Override - public void encode(Person value, OutputStream outStream, - Coder.Context context) + public void encode(Person value, OutputStream outStream) throws CoderException, IOException { - LONG_CODER.encode(value.id, outStream, Context.NESTED); - STRING_CODER.encode(value.name, outStream, Context.NESTED); - STRING_CODER.encode(value.emailAddress, outStream, Context.NESTED); - STRING_CODER.encode(value.creditCard, outStream, Context.NESTED); - STRING_CODER.encode(value.city, outStream, Context.NESTED); - STRING_CODER.encode(value.state, outStream, Context.NESTED); - LONG_CODER.encode(value.dateTime, outStream, Context.NESTED); - STRING_CODER.encode(value.extra, outStream, Context.NESTED); + LONG_CODER.encode(value.id, outStream); + STRING_CODER.encode(value.name, outStream); + STRING_CODER.encode(value.emailAddress, outStream); + STRING_CODER.encode(value.creditCard, outStream); + STRING_CODER.encode(value.city, outStream); + STRING_CODER.encode(value.state, outStream); + LONG_CODER.encode(value.dateTime, outStream); + STRING_CODER.encode(value.extra, outStream); } @Override - public Person decode( - InputStream inStream, Coder.Context context) + public Person decode(InputStream inStream) throws CoderException, IOException { - long id = LONG_CODER.decode(inStream, Context.NESTED); - String name = STRING_CODER.decode(inStream, Context.NESTED); - String emailAddress = STRING_CODER.decode(inStream, Context.NESTED); - String creditCard = STRING_CODER.decode(inStream, Context.NESTED); - String city = STRING_CODER.decode(inStream, Context.NESTED); - String state = STRING_CODER.decode(inStream, Context.NESTED); - long dateTime = LONG_CODER.decode(inStream, Context.NESTED); - String extra = STRING_CODER.decode(inStream, Context.NESTED); + long id = LONG_CODER.decode(inStream); + String name = STRING_CODER.decode(inStream); + String emailAddress = STRING_CODER.decode(inStream); + String creditCard = STRING_CODER.decode(inStream); + String city = STRING_CODER.decode(inStream); + String state = STRING_CODER.decode(inStream); + long dateTime = LONG_CODER.decode(inStream); + String extra = STRING_CODER.decode(inStream); return new Person(id, name, emailAddress, creditCard, city, state, dateTime, extra); } @Override public void verifyDeterministic() throws NonDeterministicException {} diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/SellerPrice.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/SellerPrice.java index 52ff540ac639d..61537f6be9bf0 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/SellerPrice.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/SellerPrice.java @@ -37,19 +37,18 @@ public class SellerPrice implements KnownSize, Serializable { public static final Coder CODER = new CustomCoder() { @Override - public void encode(SellerPrice value, OutputStream outStream, - Coder.Context context) + public void encode(SellerPrice value, OutputStream outStream) throws CoderException, IOException { - LONG_CODER.encode(value.seller, outStream, Context.NESTED); - LONG_CODER.encode(value.price, outStream, Context.NESTED); + LONG_CODER.encode(value.seller, outStream); + LONG_CODER.encode(value.price, outStream); } @Override public SellerPrice decode( - InputStream inStream, Coder.Context context) + InputStream inStream) throws CoderException, IOException { - long seller = LONG_CODER.decode(inStream, Context.NESTED); - long price = LONG_CODER.decode(inStream, Context.NESTED); + long seller = LONG_CODER.decode(inStream); + long price = LONG_CODER.decode(inStream); return new SellerPrice(seller, price); } @Override public void verifyDeterministic() throws NonDeterministicException {} diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/WinningBids.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/WinningBids.java index 52891a77bb31b..bd6c2edd1e72a 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/WinningBids.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/WinningBids.java @@ -156,19 +156,19 @@ public static AuctionOrBidWindowCoder of() { } @Override - public void encode(AuctionOrBidWindow window, OutputStream outStream, Coder.Context context) + public void encode(AuctionOrBidWindow window, OutputStream outStream) throws IOException, CoderException { - SUPER_CODER.encode(window, outStream, Coder.Context.NESTED); - ID_CODER.encode(window.auction, outStream, Coder.Context.NESTED); - INT_CODER.encode(window.isAuctionWindow ? 1 : 0, outStream, Coder.Context.NESTED); + SUPER_CODER.encode(window, outStream); + ID_CODER.encode(window.auction, outStream); + INT_CODER.encode(window.isAuctionWindow ? 1 : 0, outStream); } @Override - public AuctionOrBidWindow decode(InputStream inStream, Coder.Context context) + public AuctionOrBidWindow decode(InputStream inStream) throws IOException, CoderException { - IntervalWindow superWindow = SUPER_CODER.decode(inStream, Coder.Context.NESTED); - long auction = ID_CODER.decode(inStream, Coder.Context.NESTED); - boolean isAuctionWindow = INT_CODER.decode(inStream, Context.NESTED) != 0; + IntervalWindow superWindow = SUPER_CODER.decode(inStream); + long auction = ID_CODER.decode(inStream); + boolean isAuctionWindow = INT_CODER.decode(inStream) != 0; return new AuctionOrBidWindow( superWindow.start(), superWindow.end(), auction, isAuctionWindow); } diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/sources/Generator.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/sources/Generator.java index 2a2732bac798b..4f548cdd600b8 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/sources/Generator.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/sources/Generator.java @@ -102,22 +102,17 @@ public static class Checkpoint implements UnboundedSource.CheckpointMark { /** Coder for this class. */ public static final Coder CODER_INSTANCE = new CustomCoder() { - @Override - public void encode( - Checkpoint value, - OutputStream outStream, - Coder.Context context) - throws CoderException, IOException { - LONG_CODER.encode(value.numEvents, outStream, Context.NESTED); - LONG_CODER.encode(value.wallclockBaseTime, outStream, Context.NESTED); + @Override public void encode(Checkpoint value, OutputStream outStream) + throws CoderException, IOException { + LONG_CODER.encode(value.numEvents, outStream); + LONG_CODER.encode(value.wallclockBaseTime, outStream); } @Override - public Checkpoint decode( - InputStream inStream, Coder.Context context) + public Checkpoint decode(InputStream inStream) throws CoderException, IOException { - long numEvents = LONG_CODER.decode(inStream, Context.NESTED); - long wallclockBaseTime = LONG_CODER.decode(inStream, Context.NESTED); + long numEvents = LONG_CODER.decode(inStream); + long wallclockBaseTime = LONG_CODER.decode(inStream); return new Checkpoint(numEvents, wallclockBaseTime); } @Override public void verifyDeterministic() throws NonDeterministicException {} diff --git a/integration/java/pom.xml b/integration/java/pom.xml index dcad4c3a76625..b0c38531a5d51 100644 --- a/integration/java/pom.xml +++ b/integration/java/pom.xml @@ -22,7 +22,7 @@ org.apache.beam beam-integration-parent - 0.7.0-SNAPSHOT + 2.1.0-SNAPSHOT ../pom.xml diff --git a/integration/pom.xml b/integration/pom.xml index 31f293ed7a2b8..42548191d65e9 100644 --- a/integration/pom.xml +++ b/integration/pom.xml @@ -22,7 +22,7 @@ org.apache.beam beam-parent - 0.7.0-SNAPSHOT + 2.1.0-SNAPSHOT ../pom.xml From ee500b28086f1261101395dc0b7b23f197ba19d9 Mon Sep 17 00:00:00 2001 From: Etienne Chauchot Date: Tue, 30 May 2017 18:00:00 +0100 Subject: [PATCH 312/346] Fix query10 log messages issue #5 and issue #51 --- integration/java/nexmark/pom.xml | 6 +++ .../integration/nexmark/queries/Query10.java | 39 ++++++++----------- 2 files changed, 22 insertions(+), 23 deletions(-) diff --git a/integration/java/nexmark/pom.xml b/integration/java/nexmark/pom.xml index 86b88bdb1dc72..664a410813226 100644 --- a/integration/java/nexmark/pom.xml +++ b/integration/java/nexmark/pom.xml @@ -254,6 +254,12 @@ slf4j-api + + org.slf4j + slf4j-jdk14 + runtime + + com.google.code.findbugs jsr305 diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query10.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query10.java index c868666f3242f..378d01e12f4e9 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query10.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query10.java @@ -17,9 +17,6 @@ */ package org.apache.beam.integration.nexmark.queries; -import static com.google.common.base.Preconditions.checkState; - -import com.google.cloud.hadoop.gcsio.GoogleCloudStorageWriteChannel; import java.io.IOException; import java.io.OutputStream; import java.io.Serializable; @@ -33,9 +30,9 @@ import org.apache.beam.integration.nexmark.model.Event; import org.apache.beam.integration.nexmark.model.KnownSize; import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.extensions.gcp.options.GcsOptions; import org.apache.beam.sdk.metrics.Counter; import org.apache.beam.sdk.metrics.Metrics; -import org.apache.beam.sdk.extensions.gcp.options.GcsOptions; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.GroupByKey; import org.apache.beam.sdk.transforms.ParDo; @@ -57,7 +54,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; - /** * Query "10", 'Log to sharded files' (Not in original suite.) * @@ -132,12 +128,9 @@ public void setMaxNumWorkers(int maxNumWorkers) { */ private WritableByteChannel openWritableGcsFile(GcsOptions options, String filename) throws IOException { - //TODO Decide what to do about this one -// WritableByteChannel channel = -// GcsIOChannelFactory.fromOptions(options).create(filename, "text/plain"); -// checkState(channel instanceof GoogleCloudStorageWriteChannel); -// ((GoogleCloudStorageWriteChannel) channel).setUploadBufferSize(CHANNEL_BUFFER); -// return channel; + //TODO + // Fix after PR: right now this is a specific Google added use case + // Discuss it on ML: shall we keep GCS or use HDFS or use a generic beam filesystem way. throw new UnsupportedOperationException("Disabled after removal of GcsIOChannelFactory"); } @@ -192,7 +185,7 @@ private PCollection applyTyped(PCollection events) { public void processElement(ProcessContext c) { if (c.element().hasAnnotation("LATE")) { lateCounter.inc(); - LOG.error("Observed late: %s", c.element()); + LOG.info("Observed late: %s", c.element()); } else { onTimeCounter.inc(); } @@ -240,11 +233,11 @@ public void processElement(ProcessContext c, BoundedWindow window) { } } String shard = c.element().getKey(); - LOG.error( + LOG.info(String.format( "%s with timestamp %s has %d actually late and %d on-time " + "elements in pane %s for window %s", shard, c.timestamp(), numLate, numOnTime, c.pane(), - window.maxTimestamp()); + window.maxTimestamp())); if (c.pane().getTiming() == PaneInfo.Timing.LATE) { if (numLate == 0) { LOG.error( @@ -283,11 +276,11 @@ public void processElement(ProcessContext c, BoundedWindow window) String shard = c.element().getKey(); GcsOptions options = c.getPipelineOptions().as(GcsOptions.class); OutputFile outputFile = outputFileFor(window, shard, c.pane()); - LOG.error( + LOG.info(String.format( "Writing %s with record timestamp %s, window timestamp %s, pane %s", - shard, c.timestamp(), window.maxTimestamp(), c.pane()); + shard, c.timestamp(), window.maxTimestamp(), c.pane())); if (outputFile.filename != null) { - LOG.error("Beginning write to '%s'", outputFile.filename); + LOG.info("Beginning write to '%s'", outputFile.filename); int n = 0; try (OutputStream output = Channels.newOutputStream(openWritableGcsFile(options, outputFile @@ -296,12 +289,12 @@ public void processElement(ProcessContext c, BoundedWindow window) Event.CODER.encode(event, output, Coder.Context.OUTER); writtenRecordsCounter.inc(); if (++n % 10000 == 0) { - LOG.error("So far written %d records to '%s'", n, + LOG.info("So far written %d records to '%s'", n, outputFile.filename); } } } - LOG.error("Written all %d records to '%s'", n, outputFile.filename); + LOG.info("Written all %d records to '%s'", n, outputFile.filename); } savedFileCounter.inc(); c.output(KV.of(null, outputFile)); @@ -341,23 +334,23 @@ public void processElement(ProcessContext c, BoundedWindow window) LOG.error("ERROR! Unexpected ON_TIME pane index: %s", c.pane()); } else { GcsOptions options = c.getPipelineOptions().as(GcsOptions.class); - LOG.error( + LOG.info( "Index with record timestamp %s, window timestamp %s, pane %s", c.timestamp(), window.maxTimestamp(), c.pane()); @Nullable String filename = indexPathFor(window); if (filename != null) { - LOG.error("Beginning write to '%s'", filename); + LOG.info("Beginning write to '%s'", filename); int n = 0; try (OutputStream output = Channels.newOutputStream( openWritableGcsFile(options, filename))) { for (OutputFile outputFile : c.element().getValue()) { - output.write(outputFile.toString().getBytes()); + output.write(outputFile.toString().getBytes("UTF-8")); n++; } } - LOG.error("Written all %d lines to '%s'", n, filename); + LOG.info("Written all %d lines to '%s'", n, filename); } c.output( new Done("written for timestamp " + window.maxTimestamp())); From 6c116709fff06f7faa491a090f441f618931d256 Mon Sep 17 00:00:00 2001 From: Etienne Chauchot Date: Tue, 30 May 2017 18:00:00 +0100 Subject: [PATCH 313/346] Improve NexmarkUtils: improve diskBusy() and remove unneeded randomization code - Use state API in NexmarkUtils.diskBusy() - Remove commented code for direct runner randomization disabling: direct runner no more allows disabling randomization and queries and UT pass --- .../integration/nexmark/NexmarkUtils.java | 87 +++++++++++-------- .../nexmark/queries/NexmarkQuery.java | 2 +- 2 files changed, 52 insertions(+), 37 deletions(-) diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkUtils.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkUtils.java index 7707429f18818..792669033969f 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkUtils.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkUtils.java @@ -18,14 +18,12 @@ package org.apache.beam.integration.nexmark; import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.common.collect.ImmutableList; import com.google.common.hash.Hashing; import java.io.IOException; import java.io.InputStream; import java.io.OutputStream; import java.nio.charset.StandardCharsets; import java.util.Iterator; -import java.util.List; import org.apache.beam.integration.nexmark.model.Auction; import org.apache.beam.integration.nexmark.model.AuctionBid; import org.apache.beam.integration.nexmark.model.AuctionCount; @@ -66,6 +64,7 @@ import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.GlobalWindows; import org.apache.beam.sdk.transforms.windowing.Window; +import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PBegin; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.TimestampedValue; @@ -289,8 +288,8 @@ public int stepLengthSec(int ratePeriodSec) { private static final boolean LOG_ERROR = true; /** - * Set to true to log directly to stdout. If run using Google Dataflow, you can watch the results in real-time with: - * tail -f /var/log/dataflow/streaming-harness/harness-stdout.log + * Set to true to log directly to stdout. If run using Google Dataflow, you can watch the results + * in real-time with: tail -f /var/log/dataflow/streaming-harness/harness-stdout.log */ private static final boolean LOG_TO_CONSOLE = false; @@ -340,14 +339,6 @@ public static void console(String format, Object... args) { * Setup pipeline with codes and some other options. */ public static void setupPipeline(CoderStrategy coderStrategy, Pipeline p) { - //TODO Ismael check -// PipelineRunner runner = p.getRunner(); -// if (runner instanceof DirectRunner) { -// // Disable randomization of output since we want to check batch and streaming match the -// // model both locally and on the cloud. -// ((DirectRunner) runner).withUnorderednessTesting(false); -// } - CoderRegistry registry = p.getCoderRegistry(); switch (coderStrategy) { case HAND: @@ -565,35 +556,59 @@ public void processElement(ProcessContext c) { }); } - private static final StateSpec> DUMMY_TAG = - StateSpecs.value(ByteArrayCoder.of()); private static final int MAX_BUFFER_SIZE = 1 << 24; + private static class DiskBusyTransform extends PTransform, PCollection>{ + + private long bytes; + + private DiskBusyTransform(long bytes) { + this.bytes = bytes; + } + + @Override public PCollection expand(PCollection input) { + // Add dummy key to be able to use State API + PCollection> kvCollection = input.apply("diskBusy.keyElements", ParDo.of(new DoFn>() { + + @ProcessElement public void processElement(ProcessContext context) { + context.output(KV.of(0, context.element())); + } + })); + // Apply actual transform that generates disk IO using state API + PCollection output = kvCollection.apply("diskBusy.generateIO", ParDo.of(new DoFn, T>() { + + private static final String DISK_BUSY = "diskBusy"; + + @StateId(DISK_BUSY) private final StateSpec> spec = StateSpecs + .value(ByteArrayCoder.of()); + + @ProcessElement public void processElement(ProcessContext c, + @StateId(DISK_BUSY) ValueState state) { + long remain = bytes; + long now = System.currentTimeMillis(); + while (remain > 0) { + long thisBytes = Math.min(remain, MAX_BUFFER_SIZE); + remain -= thisBytes; + byte[] arr = new byte[(int) thisBytes]; + for (int i = 0; i < thisBytes; i++) { + arr[i] = (byte) now; + } + state.write(arr); + now = System.currentTimeMillis(); + } + c.output(c.element().getValue()); + } + })); + return output; + } + } + + /** * Return a transform to write given number of bytes to durable store on every record. */ - public static ParDo.SingleOutput diskBusy(String name, final long bytes) { - return ParDo.of(new DoFn() { - @ProcessElement - public void processElement(ProcessContext c) { - long remain = bytes; -// long now = System.currentTimeMillis(); - while (remain > 0) { - //TODO Ismael google on state - long thisBytes = Math.min(remain, MAX_BUFFER_SIZE); - remain -= thisBytes; -// byte[] arr = new byte[(int) thisBytes]; -// for (int i = 0; i < thisBytes; i++) { -// arr[i] = (byte) now; -// } -// ValueState state = c.windowingInternals().stateInternals().state( -// StateNamespaces.global(), DUMMY_TAG); -// state.write(arr); -// now = System.currentTimeMillis(); - } - c.output(c.element()); - } - }); + public static PTransform, PCollection> diskBusy(final long bytes) { + return new DiskBusyTransform<>(bytes); } /** diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/NexmarkQuery.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/NexmarkQuery.java index 09415c0cfc3ed..8b74282694ff4 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/NexmarkQuery.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/NexmarkQuery.java @@ -253,7 +253,7 @@ public PCollection> expand(PCollection events if (configuration.diskBusyBytes > 0) { // Slow down by forcing bytes to durable store. events = events.apply(name + ".DiskBusy", - NexmarkUtils.diskBusy(name, configuration.diskBusyBytes)); + NexmarkUtils.diskBusy(configuration.diskBusyBytes)); } // Run the query. From 2f9b4948fd60a749ada832d003acf0bd84875fcb Mon Sep 17 00:00:00 2001 From: Etienne Chauchot Date: Tue, 30 May 2017 18:00:00 +0100 Subject: [PATCH 314/346] Clean, fix findbugs, fix checkstyle --- .../nexmark/NexmarkConfiguration.java | 9 +- .../integration/nexmark/NexmarkLauncher.java | 62 +++++------- .../integration/nexmark/NexmarkOptions.java | 3 +- .../integration/nexmark/NexmarkSuite.java | 4 +- .../beam/integration/nexmark/model/Event.java | 99 ++++++++++--------- .../nexmark/queries/Query0Model.java | 1 - .../nexmark/queries/Query1Model.java | 1 - .../integration/nexmark/queries/Query3.java | 8 +- .../integration/nexmark/queries/Query5.java | 68 +++++++------ .../integration/nexmark/queries/Query7.java | 2 +- .../nexmark/queries/Query7Model.java | 1 - .../nexmark/queries/WinningBids.java | 37 ++++++- .../nexmark/queries/WinningBidsSimulator.java | 1 - .../nexmark/sources/Generator.java | 36 +++++-- .../nexmark/sources/GeneratorConfig.java | 29 +++--- .../nexmark/queries/QueryTest.java | 6 +- .../sources/UnboundedEventSourceTest.java | 6 +- 17 files changed, 211 insertions(+), 162 deletions(-) diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkConfiguration.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkConfiguration.java index 5a8cb7182d01b..2faf3f523ed8d 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkConfiguration.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkConfiguration.java @@ -19,7 +19,6 @@ import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.core.JsonProcessingException; - import java.io.IOException; import java.io.Serializable; import java.util.Objects; @@ -359,11 +358,11 @@ public void overrideFromOptions(NexmarkOptions options) { } /** - * Return clone of configuration with given label. + * Return copy of configuration with given label. */ - @Override - public NexmarkConfiguration clone() { - NexmarkConfiguration result = new NexmarkConfiguration(); + public NexmarkConfiguration copy() { + NexmarkConfiguration result; + result = new NexmarkConfiguration(); result.debug = debug; result.query = query; result.sourceType = sourceType; diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkLauncher.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkLauncher.java index db53191be2e22..a60997529a018 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkLauncher.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkLauncher.java @@ -87,11 +87,13 @@ import org.apache.beam.sdk.values.TupleTag; import org.apache.beam.sdk.values.TupleTagList; import org.joda.time.Duration; +import org.slf4j.LoggerFactory; /** * Run a single Nexmark query using a given configuration. */ public class NexmarkLauncher { + private static final org.slf4j.Logger LOG = LoggerFactory.getLogger(NexmarkLauncher.class); /** * Minimum number of samples needed for 'stead-state' rate calculation. */ @@ -165,13 +167,6 @@ private boolean isStreaming() { return options.isStreaming(); } - /** - * Return number of cores per worker. - */ - protected int coresPerWorker() { - return 4; - } - /** * Return maximum number of workers. */ @@ -185,7 +180,6 @@ private int maxNumWorkers() { */ private long getCounterMetric(PipelineResult result, String namespace, String name, long defaultValue) { - //TODO Ismael calc this only once MetricQueryResults metrics = result.metrics().queryMetrics( MetricsFilter.builder().addNameFilter(MetricNameFilter.named(namespace, name)).build()); Iterable> counters = metrics.counters(); @@ -193,7 +187,7 @@ private long getCounterMetric(PipelineResult result, String namespace, String na MetricResult metricResult = counters.iterator().next(); return metricResult.attempted(); } catch (NoSuchElementException e) { - //TODO Ismael + LOG.error("Failed to get metric {}, from namespace {}", name, namespace); } return defaultValue; } @@ -209,15 +203,20 @@ private long getDistributionMetric(PipelineResult result, String namespace, Stri Iterable> distributions = metrics.distributions(); try { MetricResult distributionResult = distributions.iterator().next(); - if (distType.equals(DistributionType.MIN)) { - return distributionResult.attempted().min(); - } else if (distType.equals(DistributionType.MAX)) { - return distributionResult.attempted().max(); - } else { - //TODO Ismael + switch (distType) + { + case MIN: + return distributionResult.attempted().min(); + case MAX: + return distributionResult.attempted().max(); + default: + return defaultValue; } } catch (NoSuchElementException e) { - //TODO Ismael + LOG.error( + "Failed to get distribution metric {} for namespace {}", + name, + namespace); } return defaultValue; } @@ -228,7 +227,9 @@ private enum DistributionType {MIN, MAX} * Return the current value for a time counter, or -1 if can't be retrieved. */ private long getTimestampMetric(long now, long value) { - //TODO Ismael improve doc + // timestamp metrics are used to monitor time of execution of transforms. + // If result timestamp metric is too far from now, consider that metric is erroneous + if (Math.abs(value - now) > Duration.standardDays(10000).getMillis()) { return -1; } @@ -437,16 +438,6 @@ interface PipelineBuilder { */ private void invokeBuilderForPublishOnlyPipeline(PipelineBuilder builder) { builder.build(options); -// throw new UnsupportedOperationException( -// "Cannot use --pubSubMode=COMBINED with DirectRunner"); - } - - /** - * If monitoring, wait until the publisher pipeline has run long enough to establish - * a backlog on the Pubsub topic. Otherwise, return immediately. - */ - private void waitForPublisherPreload() { - throw new UnsupportedOperationException(); } /** @@ -606,11 +597,7 @@ private NexmarkPerf monitor(NexmarkQuery query) { publisherJob.waitUntilFinish(Duration.standardMinutes(5)); } catch (IOException e) { throw new RuntimeException("Unable to cancel publisher job: ", e); - } //TODO Ismael -// catch (InterruptedException e) { -// Thread.interrupted(); -// throw new RuntimeException("Interrupted: publish job still running.", e); -// } + } } return perf; @@ -755,7 +742,7 @@ public void processElement(ProcessContext c) { Event event = CoderUtils.decodeFromByteArray(Event.CODER, payload); c.output(event); } catch (CoderException e) { - // TODO Log decoding Event error + LOG.error("Error while decoding Event from pusbSub message: serialization error"); } } })); @@ -798,7 +785,8 @@ public void processElement(ProcessContext c) { byte[] payload = CoderUtils.encodeToByteArray(Event.CODER, c.element()); c.output(new PubsubMessage(payload, new HashMap())); } catch (CoderException e1) { - // TODO Log encoding Event error + LOG.error("Error while sending Event {} to pusbSub: serialization error", + c.element().toString()); } } }) @@ -1130,7 +1118,8 @@ public NexmarkPerf run(NexmarkConfiguration runConfiguration) { sinkEventsToAvro(source); } - // Special hacks for Query 10 (big logger). + // Query 10 logs all events to Google Cloud storage files. It could generate a lot of logs, + // so, set parallelism. Also set the output path where to write log files. if (configuration.query == 10) { String path = null; if (options.getOutputPath() != null && !options.getOutputPath().isEmpty()) { @@ -1158,9 +1147,6 @@ public NexmarkPerf run(NexmarkConfiguration runConfiguration) { sink(results, now); } - if (publisherResult != null) { - waitForPublisherPreload(); - } mainResult = p.run(); mainResult.waitUntilFinish(Duration.standardSeconds(configuration.streamTimeout)); return monitor(query); diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkOptions.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkOptions.java index 9afffaa721a25..fbd3e74acfe69 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkOptions.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkOptions.java @@ -19,7 +19,6 @@ import javax.annotation.Nullable; import org.apache.beam.sdk.extensions.gcp.options.GcpOptions; -import org.apache.beam.sdk.io.gcp.pubsub.PubsubOptions; import org.apache.beam.sdk.options.ApplicationNameOptions; import org.apache.beam.sdk.options.Default; import org.apache.beam.sdk.options.Description; @@ -119,7 +118,7 @@ public interface NexmarkOptions @Nullable Integer getStreamTimeout(); - void setStreamTimeout(Integer preloadSeconds); + void setStreamTimeout(Integer streamTimeout); @Description("Number of unbounded sources to create events.") @Nullable diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkSuite.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkSuite.java index be7d7b80d3b7c..0d98a5dbc164d 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkSuite.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkSuite.java @@ -57,7 +57,7 @@ private static List defaultConf() { private static List smoke() { List configurations = new ArrayList<>(); for (int query = 0; query <= 12; query++) { - NexmarkConfiguration configuration = NexmarkConfiguration.DEFAULT.clone(); + NexmarkConfiguration configuration = NexmarkConfiguration.DEFAULT.copy(); configuration.query = query; configuration.numEvents = 100_000; if (query == 4 || query == 6 || query == 9) { @@ -103,7 +103,7 @@ private static List fullThrottle() { public Iterable getConfigurations(NexmarkOptions options) { Set results = new LinkedHashSet<>(); for (NexmarkConfiguration configuration : configurations) { - NexmarkConfiguration result = configuration.clone(); + NexmarkConfiguration result = configuration.copy(); result.overrideFromOptions(options); results.add(result); } diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/Event.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/Event.java index d8138331bc8d7..0e1672eb3801c 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/Event.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/Event.java @@ -23,55 +23,65 @@ import java.io.Serializable; import javax.annotation.Nullable; import org.apache.beam.sdk.coders.Coder; -import org.apache.beam.sdk.coders.CoderException; import org.apache.beam.sdk.coders.CustomCoder; import org.apache.beam.sdk.coders.VarIntCoder; /** - * An event in the auction system, either a (new) {@link Person}, a (new) {@link Auction}, - * or a {@link Bid}. + * An event in the auction system, either a (new) {@link Person}, a (new) {@link Auction}, or a + * {@link Bid}. */ public class Event implements KnownSize, Serializable { - private static final Coder INT_CODER = VarIntCoder.of(); + private enum Tag { + PERSON(0), + AUCTION(1), + BID(2); - public static final Coder CODER = new CustomCoder() { - @Override - public void encode(Event value, OutputStream outStream) - throws CoderException, IOException { - if (value.newPerson != null) { - INT_CODER.encode(0, outStream); - Person.CODER.encode(value.newPerson, outStream); - } else if (value.newAuction != null) { - INT_CODER.encode(1, outStream); - Auction.CODER.encode(value.newAuction, outStream); - } else if (value.bid != null) { - INT_CODER.encode(2, outStream); - Bid.CODER.encode(value.bid, outStream); - } else { - throw new RuntimeException("invalid event"); - } - } + private int value = -1; - @Override - public Event decode( - InputStream inStream) - throws CoderException, IOException { - int tag = INT_CODER.decode(inStream); - if (tag == 0) { - Person person = Person.CODER.decode(inStream); - return new Event(person); - } else if (tag == 1) { - Auction auction = Auction.CODER.decode(inStream); - return new Event(auction); - } else if (tag == 2) { - Bid bid = Bid.CODER.decode(inStream); - return new Event(bid); - } else { - throw new RuntimeException("invalid event encoding"); - } + Tag(int value){ + this.value = value; } - @Override public void verifyDeterministic() throws NonDeterministicException {} - }; + } + private static final Coder INT_CODER = VarIntCoder.of(); + + public static final Coder CODER = + new CustomCoder() { + @Override + public void encode(Event value, OutputStream outStream) throws IOException { + if (value.newPerson != null) { + INT_CODER.encode(Tag.PERSON.value, outStream); + Person.CODER.encode(value.newPerson, outStream); + } else if (value.newAuction != null) { + INT_CODER.encode(Tag.AUCTION.value, outStream); + Auction.CODER.encode(value.newAuction, outStream); + } else if (value.bid != null) { + INT_CODER.encode(Tag.BID.value, outStream); + Bid.CODER.encode(value.bid, outStream); + } else { + throw new RuntimeException("invalid event"); + } + } + + @Override + public Event decode(InputStream inStream) throws IOException { + int tag = INT_CODER.decode(inStream); + if (tag == Tag.PERSON.value) { + Person person = Person.CODER.decode(inStream); + return new Event(person); + } else if (tag == Tag.AUCTION.value) { + Auction auction = Auction.CODER.decode(inStream); + return new Event(auction); + } else if (tag == Tag.BID.value) { + Bid bid = Bid.CODER.decode(inStream); + return new Event(bid); + } else { + throw new RuntimeException("invalid event encoding"); + } + } + + @Override + public void verifyDeterministic() throws NonDeterministicException {} + }; @Nullable @org.apache.avro.reflect.Nullable @@ -111,10 +121,7 @@ public Event(Bid bid) { this.bid = bid; } - /** - * Return a copy of event which captures {@code annotation}. - * (Used for debugging). - */ + /** Return a copy of event which captures {@code annotation}. (Used for debugging). */ public Event withAnnotation(String annotation) { if (newPerson != null) { return new Event(newPerson.withAnnotation(annotation)); @@ -125,9 +132,7 @@ public Event withAnnotation(String annotation) { } } - /** - * Does event have {@code annotation}? (Used for debugging.) - */ + /** Does event have {@code annotation}? (Used for debugging.) */ public boolean hasAnnotation(String annotation) { if (newPerson != null) { return newPerson.hasAnnotation(annotation); diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query0Model.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query0Model.java index 8e655917fd24d..e2522b8b03698 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query0Model.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query0Model.java @@ -45,7 +45,6 @@ protected void run() { return; } addResult(timestampedEvent); - //TODO test fails because offset of some hundreds of ms beween expect and actual } } diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query1Model.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query1Model.java index 5d4de45e1d7bd..f07db80b38292 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query1Model.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query1Model.java @@ -57,7 +57,6 @@ protected void run() { TimestampedValue result = TimestampedValue.of(resultBid, timestampedEvent.getTimestamp()); addResult(result); - //TODO test fails because offset of some hundreds of ms beween expect and actual } } diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query3.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query3.java index f74b78db4b4b9..f2b66d7ade38e 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query3.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query3.java @@ -29,13 +29,13 @@ import org.apache.beam.sdk.coders.ListCoder; import org.apache.beam.sdk.metrics.Counter; import org.apache.beam.sdk.metrics.Metrics; +import org.apache.beam.sdk.state.StateSpec; +import org.apache.beam.sdk.state.StateSpecs; import org.apache.beam.sdk.state.TimeDomain; import org.apache.beam.sdk.state.Timer; import org.apache.beam.sdk.state.TimerSpec; import org.apache.beam.sdk.state.TimerSpecs; import org.apache.beam.sdk.state.ValueState; -import org.apache.beam.sdk.state.StateSpec; -import org.apache.beam.sdk.state.StateSpecs; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.Filter; import org.apache.beam.sdk.transforms.ParDo; @@ -243,9 +243,9 @@ public void processElement( theNewPerson = newPerson; } else { if (theNewPerson.equals(newPerson)) { - LOG.error("**** duplicate person {} ****", theNewPerson); + LOG.error("Duplicate person {}", theNewPerson); } else { - LOG.error("**** conflicting persons {} and {} ****", theNewPerson, newPerson); + LOG.error("Conflicting persons {} and {}", theNewPerson, newPerson); } fatalCounter.inc(); continue; diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query5.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query5.java index 194433055b488..bdf3e5f15e588 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query5.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query5.java @@ -63,56 +63,64 @@ private PCollection applyTyped(PCollection events) { // Only want the bid events. .apply(JUST_BIDS) // Window the bids into sliding windows. - .apply(Window.into( - SlidingWindows.of(Duration.standardSeconds(configuration.windowSizeSec)) - .every(Duration.standardSeconds(configuration.windowPeriodSec)))) + .apply( + Window.into( + SlidingWindows.of(Duration.standardSeconds(configuration.windowSizeSec)) + .every(Duration.standardSeconds(configuration.windowPeriodSec)))) // Project just the auction id. .apply("BidToAuction", BID_TO_AUCTION) // Count the number of bids per auction id. .apply(Count.perElement()) - // We'll want to keep all auctions with the maximal number of bids. + // We'll want to keep all auctions with the maximal number of bids. // Start by lifting each into a singleton list. // need to do so because bellow combine returns a list of auctions in the key in case of // equal number of bids. Combine needs to have same input type and return type. - .apply(name + ".ToSingletons", - ParDo.of(new DoFn, KV, Long>>() { + .apply( + name + ".ToSingletons", + ParDo.of( + new DoFn, KV, Long>>() { @ProcessElement public void processElement(ProcessContext c) { - c.output(KV.of(Collections.singletonList(c.element().getKey()), c.element().getValue())); + c.output( + KV.of( + Collections.singletonList(c.element().getKey()), + c.element().getValue())); } })) // Keep only the auction ids with the most bids. .apply( - Combine - .globally(new Combine.BinaryCombineFn, Long>>() { - @Override - public KV, Long> apply( - KV, Long> left, KV, Long> right) { - List leftBestAuctions = left.getKey(); - long leftCount = left.getValue(); - List rightBestAuctions = right.getKey(); - long rightCount = right.getValue(); - if (leftCount > rightCount) { - return left; - } else if (leftCount < rightCount) { - return right; - } else { - List newBestAuctions = new ArrayList<>(); - newBestAuctions.addAll(leftBestAuctions); - newBestAuctions.addAll(rightBestAuctions); - return KV.of(newBestAuctions, leftCount); - } - } - }) + Combine.globally( + new Combine.BinaryCombineFn, Long>>() { + @Override + public KV, Long> apply( + KV, Long> left, KV, Long> right) { + List leftBestAuctions = left.getKey(); + long leftCount = left.getValue(); + List rightBestAuctions = right.getKey(); + long rightCount = right.getValue(); + if (leftCount > rightCount) { + return left; + } else if (leftCount < rightCount) { + return right; + } else { + List newBestAuctions = new ArrayList<>(); + newBestAuctions.addAll(leftBestAuctions); + newBestAuctions.addAll(rightBestAuctions); + return KV.of(newBestAuctions, leftCount); + } + } + }) .withoutDefaults() .withFanout(configuration.fanout)) // Project into result. - .apply(name + ".Select", - ParDo.of(new DoFn, Long>, AuctionCount>() { + .apply( + name + ".Select", + ParDo.of( + new DoFn, Long>, AuctionCount>() { @ProcessElement public void processElement(ProcessContext c) { long count = c.element().getValue(); diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query7.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query7.java index 2a94ca9950f58..217d0d4ef1c9f 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query7.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query7.java @@ -63,7 +63,7 @@ private PCollection applyTyped(PCollection events) { // requires an additional scan per window, with the associated cost of snapshotted state and // its I/O. We'll keep this implementation since it illustrates the use of side inputs. final PCollectionView maxPriceView = - slidingBids // + slidingBids .apply("BidToPrice", BID_TO_PRICE) .apply(Max.longsGlobally().withFanout(configuration.fanout).asSingletonView()); diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query7Model.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query7Model.java index 5c039f9580b96..0ada5e8c3e90f 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query7Model.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query7Model.java @@ -111,7 +111,6 @@ protected void run() { } // Keep only the highest bids. captureBid(event.bid); - //TODO test fails because offset of some hundreds of ms between expect and actual } } diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/WinningBids.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/WinningBids.java index bd6c2edd1e72a..d4ca1773047fe 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/WinningBids.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/WinningBids.java @@ -23,12 +23,12 @@ import java.io.IOException; import java.io.InputStream; import java.io.OutputStream; -import java.io.Serializable; import java.util.ArrayList; import java.util.Collection; import java.util.Collections; import java.util.List; import java.util.Map; +import java.util.Objects; import java.util.TreeMap; import org.apache.beam.integration.nexmark.NexmarkConfiguration; import org.apache.beam.integration.nexmark.NexmarkUtils; @@ -139,6 +139,24 @@ public String toString() { return String.format("AuctionOrBidWindow{start:%s; end:%s; auction:%d; isAuctionWindow:%s}", start(), end(), auction, isAuctionWindow); } + + @Override public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + if (!super.equals(o)) { + return false; + } + AuctionOrBidWindow that = (AuctionOrBidWindow) o; + return (isAuctionWindow == that.isAuctionWindow) && (auction == that.auction); + } + + @Override public int hashCode() { + return Objects.hash(isAuctionWindow, auction); + } } /** @@ -374,4 +392,21 @@ public void processElement(ProcessContext c) { } )); } + + @Override + public int hashCode() { + return Objects.hash(auctionOrBidWindowFn); + } + + @Override public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + + WinningBids that = (WinningBids) o; + return auctionOrBidWindowFn.equals(that.auctionOrBidWindowFn); + } } diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/WinningBidsSimulator.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/WinningBidsSimulator.java index 7d74f8f8e2a07..9624a9d69c891 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/WinningBidsSimulator.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/WinningBidsSimulator.java @@ -181,7 +181,6 @@ protected void run() { return; } addResult(result); - //TODO test fails because offset of some hundreds of ms beween expect and actual return; } diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/sources/Generator.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/sources/Generator.java index 4f548cdd600b8..f6deceb32f225 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/sources/Generator.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/sources/Generator.java @@ -26,6 +26,7 @@ import java.util.Arrays; import java.util.Iterator; import java.util.List; +import java.util.Objects; import java.util.Random; import org.apache.beam.integration.nexmark.model.Auction; import org.apache.beam.integration.nexmark.model.Bid; @@ -167,7 +168,7 @@ public NextEvent(long wallclockTimestamp, long eventTimestamp, Event event, long } /** - * Return a deep clone of next event with delay added to wallclock timestamp and + * Return a deep copy of next event with delay added to wallclock timestamp and * event annotate as 'LATE'. */ public NextEvent withDelay(long delayMs) { @@ -175,6 +176,26 @@ public NextEvent withDelay(long delayMs) { wallclockTimestamp + delayMs, eventTimestamp, event.withAnnotation("LATE"), watermark); } + @Override public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + + NextEvent nextEvent = (NextEvent) o; + + return (wallclockTimestamp == nextEvent.wallclockTimestamp + && eventTimestamp == nextEvent.eventTimestamp + && watermark == nextEvent.watermark + && event.equals(nextEvent.event)); + } + + @Override public int hashCode() { + return Objects.hash(wallclockTimestamp, eventTimestamp, watermark, event); + } + @Override public int compareTo(NextEvent other) { int i = Long.compare(wallclockTimestamp, other.wallclockTimestamp); @@ -221,11 +242,12 @@ public Checkpoint toCheckpoint() { } /** - * Return a deep clone of this generator. + * Return a deep copy of this generator. */ - @Override - public Generator clone() { - return new Generator(config.clone(), numEvents, wallclockBaseTime); + public Generator copy() { + checkNotNull(config); + Generator result = new Generator(config, numEvents, wallclockBaseTime); + return result; } /** @@ -243,9 +265,9 @@ public GeneratorConfig getCurrentConfig() { */ public GeneratorConfig splitAtEventId(long eventId) { long newMaxEvents = eventId - (config.firstEventId + config.firstEventNumber); - GeneratorConfig remainConfig = config.cloneWith(config.firstEventId, + GeneratorConfig remainConfig = config.copyWith(config.firstEventId, config.maxEvents - newMaxEvents, config.firstEventNumber + newMaxEvents); - config = config.cloneWith(config.firstEventId, newMaxEvents, config.firstEventNumber); + config = config.copyWith(config.firstEventId, newMaxEvents, config.firstEventNumber); return remainConfig; } diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/sources/GeneratorConfig.java b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/sources/GeneratorConfig.java index 5799bb2071ba1..95c276b0db0f9 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/sources/GeneratorConfig.java +++ b/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/sources/GeneratorConfig.java @@ -17,6 +17,8 @@ */ package org.apache.beam.integration.nexmark.sources; +import static com.google.common.base.Preconditions.checkNotNull; + import java.io.Serializable; import java.util.ArrayList; import java.util.List; @@ -29,6 +31,7 @@ * Parameters controlling how {@link Generator} synthesizes {@link Event} elements. */ public class GeneratorConfig implements Serializable { + /** * We start the ids at specific values to help ensure the queries find a match even on * small synthesized dataset sizes. @@ -132,18 +135,13 @@ public GeneratorConfig( } /** - * Return a clone of this config. - */ - @Override - public GeneratorConfig clone() { - return new GeneratorConfig(configuration, baseTime, firstEventId, maxEvents, firstEventNumber); - } - - /** - * Return clone of this config except with given parameters. + * Return a copy of this config. */ - public GeneratorConfig cloneWith(long firstEventId, long maxEvents, long firstEventNumber) { - return new GeneratorConfig(configuration, baseTime, firstEventId, maxEvents, firstEventNumber); + public GeneratorConfig copy() { + GeneratorConfig result; + result = new GeneratorConfig(configuration, baseTime, firstEventId, + maxEvents, firstEventNumber); + return result; } /** @@ -164,13 +162,20 @@ public List split(int n) { // Don't loose any events to round-down. subMaxEvents = maxEvents - subMaxEvents * (n - 1); } - results.add(cloneWith(subFirstEventId, subMaxEvents, firstEventNumber)); + results.add(copyWith(subFirstEventId, subMaxEvents, firstEventNumber)); subFirstEventId += subMaxEvents; } } return results; } + /** + * Return copy of this config except with given parameters. + */ + public GeneratorConfig copyWith(long firstEventId, long maxEvents, long firstEventNumber) { + return new GeneratorConfig(configuration, baseTime, firstEventId, maxEvents, firstEventNumber); + } + /** * Return an estimate of the bytes needed by {@code numEvents}. */ diff --git a/integration/java/nexmark/src/test/java/org/apache/beam/integration/nexmark/queries/QueryTest.java b/integration/java/nexmark/src/test/java/org/apache/beam/integration/nexmark/queries/QueryTest.java index b005d652b9ab8..64a8e4fee4f1a 100644 --- a/integration/java/nexmark/src/test/java/org/apache/beam/integration/nexmark/queries/QueryTest.java +++ b/integration/java/nexmark/src/test/java/org/apache/beam/integration/nexmark/queries/QueryTest.java @@ -37,7 +37,7 @@ /** Test the various NEXMark queries yield results coherent with their models. */ @RunWith(JUnit4.class) public class QueryTest { - private static final NexmarkConfiguration CONFIG = NexmarkConfiguration.DEFAULT.clone(); + private static final NexmarkConfiguration CONFIG = NexmarkConfiguration.DEFAULT.copy(); static { // careful, results of tests are linked to numEventGenerators because of timestamp generation @@ -55,12 +55,8 @@ private void queryMatchesModel( if (streamingMode) { results = p.apply(name + ".ReadUnBounded", NexmarkUtils.streamEventsSource(CONFIG)).apply(query); - //TODO Ismael this should not be called explicitly - results.setIsBoundedInternal(PCollection.IsBounded.UNBOUNDED); } else { results = p.apply(name + ".ReadBounded", NexmarkUtils.batchEventsSource(CONFIG)).apply(query); - //TODO Ismael this should not be called explicitly - results.setIsBoundedInternal(PCollection.IsBounded.BOUNDED); } PAssert.that(results).satisfies(model.assertionFor()); PipelineResult result = p.run(); diff --git a/integration/java/nexmark/src/test/java/org/apache/beam/integration/nexmark/sources/UnboundedEventSourceTest.java b/integration/java/nexmark/src/test/java/org/apache/beam/integration/nexmark/sources/UnboundedEventSourceTest.java index 1d04e2a381d5d..1ecc33e536936 100644 --- a/integration/java/nexmark/src/test/java/org/apache/beam/integration/nexmark/sources/UnboundedEventSourceTest.java +++ b/integration/java/nexmark/src/test/java/org/apache/beam/integration/nexmark/sources/UnboundedEventSourceTest.java @@ -28,7 +28,6 @@ import org.apache.beam.integration.nexmark.NexmarkConfiguration; import org.apache.beam.integration.nexmark.model.Event; -import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.io.UnboundedSource.CheckpointMark; import org.apache.beam.sdk.io.UnboundedSource.UnboundedReader; import org.apache.beam.sdk.options.PipelineOptions; @@ -95,12 +94,11 @@ public void resumeFromCheckpoint() throws IOException { while (n > 0) { int m = Math.min(459 + random.nextInt(455), n); - System.out.printf("reading %d...\n", m); + System.out.printf("reading %d...%n", m); checker.add(m, reader, modelGenerator); n -= m; - System.out.printf("splitting with %d remaining...\n", n); + System.out.printf("splitting with %d remaining...%n", n); CheckpointMark checkpointMark = reader.getCheckpointMark(); - assertTrue(checkpointMark instanceof Generator.Checkpoint); reader = source.createReader(options, (Generator.Checkpoint) checkpointMark); } From f4333df77267d5207f0f23ae62e79b171a00e8a7 Mon Sep 17 00:00:00 2001 From: Etienne Chauchot Date: Thu, 15 Jun 2017 11:55:26 +0200 Subject: [PATCH 315/346] Move module beam-integration-java-nexmark to beam-sdks-java-nexmark --- integration/java/pom.xml | 37 ----------- integration/pom.xml | 51 --------------- pom.xml | 1 - {integration => sdks}/java/nexmark/README.md | 0 {integration => sdks}/java/nexmark/pom.xml | 6 +- .../org/apache/beam/sdk}/nexmark/Main.java | 9 ++- .../org/apache/beam/sdk}/nexmark/Monitor.java | 5 +- .../sdk}/nexmark/NexmarkConfiguration.java | 2 +- .../beam/sdk}/nexmark/NexmarkLauncher.java | 65 +++++++++---------- .../beam/sdk}/nexmark/NexmarkOptions.java | 2 +- .../apache/beam/sdk}/nexmark/NexmarkPerf.java | 3 +- .../beam/sdk}/nexmark/NexmarkSuite.java | 2 +- .../beam/sdk}/nexmark/NexmarkUtils.java | 50 +++++++------- .../beam/sdk}/nexmark/model/Auction.java | 4 +- .../beam/sdk}/nexmark/model/AuctionBid.java | 7 +- .../beam/sdk}/nexmark/model/AuctionCount.java | 4 +- .../beam/sdk}/nexmark/model/AuctionPrice.java | 4 +- .../apache/beam/sdk}/nexmark/model/Bid.java | 4 +- .../sdk}/nexmark/model/BidsPerSession.java | 4 +- .../sdk}/nexmark/model/CategoryPrice.java | 4 +- .../apache/beam/sdk}/nexmark/model/Done.java | 4 +- .../apache/beam/sdk}/nexmark/model/Event.java | 2 +- .../sdk}/nexmark/model/IdNameReserve.java | 4 +- .../beam/sdk}/nexmark/model/KnownSize.java | 2 +- .../sdk}/nexmark/model/NameCityStateId.java | 4 +- .../beam/sdk}/nexmark/model/Person.java | 4 +- .../beam/sdk}/nexmark/model/SellerPrice.java | 4 +- .../beam/sdk}/nexmark/model/package-info.java | 2 +- .../beam/sdk}/nexmark/package-info.java | 2 +- .../nexmark/queries/AbstractSimulator.java | 4 +- .../sdk}/nexmark/queries/NexmarkQuery.java | 18 ++--- .../nexmark/queries/NexmarkQueryModel.java | 6 +- .../beam/sdk}/nexmark/queries/Query0.java | 11 ++-- .../sdk}/nexmark/queries/Query0Model.java | 8 +-- .../beam/sdk}/nexmark/queries/Query1.java | 12 ++-- .../beam/sdk}/nexmark/queries/Query10.java | 12 ++-- .../beam/sdk}/nexmark/queries/Query11.java | 14 ++-- .../beam/sdk}/nexmark/queries/Query12.java | 14 ++-- .../sdk}/nexmark/queries/Query1Model.java | 10 +-- .../beam/sdk}/nexmark/queries/Query2.java | 14 ++-- .../sdk}/nexmark/queries/Query2Model.java | 12 ++-- .../beam/sdk}/nexmark/queries/Query3.java | 16 ++--- .../sdk}/nexmark/queries/Query3Model.java | 14 ++-- .../beam/sdk}/nexmark/queries/Query4.java | 20 +++--- .../sdk}/nexmark/queries/Query4Model.java | 16 ++--- .../beam/sdk}/nexmark/queries/Query5.java | 14 ++-- .../sdk}/nexmark/queries/Query5Model.java | 12 ++-- .../beam/sdk}/nexmark/queries/Query6.java | 18 ++--- .../sdk}/nexmark/queries/Query6Model.java | 16 ++--- .../beam/sdk}/nexmark/queries/Query7.java | 12 ++-- .../sdk}/nexmark/queries/Query7Model.java | 10 +-- .../beam/sdk}/nexmark/queries/Query8.java | 16 ++--- .../sdk}/nexmark/queries/Query8Model.java | 14 ++-- .../beam/sdk}/nexmark/queries/Query9.java | 12 ++-- .../sdk}/nexmark/queries/Query9Model.java | 4 +- .../sdk}/nexmark/queries/WinningBids.java | 16 ++--- .../nexmark/queries/WinningBidsSimulator.java | 14 ++-- .../sdk}/nexmark/queries/package-info.java | 2 +- .../nexmark/sources/BoundedEventSource.java | 6 +- .../beam/sdk}/nexmark/sources/Generator.java | 10 +-- .../sdk}/nexmark/sources/GeneratorConfig.java | 9 +-- .../nexmark/sources/UnboundedEventSource.java | 7 +- .../sdk}/nexmark/sources/package-info.java | 2 +- .../src/main/resources/log4j.properties | 0 .../beam/sdk}/nexmark/queries/QueryTest.java | 8 +-- .../sources/BoundedEventSourceTest.java | 7 +- .../sdk}/nexmark/sources/GeneratorTest.java | 5 +- .../sources/UnboundedEventSourceTest.java | 8 +-- sdks/java/pom.xml | 1 + 69 files changed, 309 insertions(+), 407 deletions(-) delete mode 100644 integration/java/pom.xml delete mode 100644 integration/pom.xml rename {integration => sdks}/java/nexmark/README.md (100%) rename {integration => sdks}/java/nexmark/pom.xml (97%) rename {integration/java/nexmark/src/main/java/org/apache/beam/integration => sdks/java/nexmark/src/main/java/org/apache/beam/sdk}/nexmark/Main.java (98%) rename {integration/java/nexmark/src/main/java/org/apache/beam/integration => sdks/java/nexmark/src/main/java/org/apache/beam/sdk}/nexmark/Monitor.java (96%) rename {integration/java/nexmark/src/main/java/org/apache/beam/integration => sdks/java/nexmark/src/main/java/org/apache/beam/sdk}/nexmark/NexmarkConfiguration.java (99%) rename {integration/java/nexmark/src/main/java/org/apache/beam/integration => sdks/java/nexmark/src/main/java/org/apache/beam/sdk}/nexmark/NexmarkLauncher.java (95%) rename {integration/java/nexmark/src/main/java/org/apache/beam/integration => sdks/java/nexmark/src/main/java/org/apache/beam/sdk}/nexmark/NexmarkOptions.java (99%) rename {integration/java/nexmark/src/main/java/org/apache/beam/integration => sdks/java/nexmark/src/main/java/org/apache/beam/sdk}/nexmark/NexmarkPerf.java (99%) rename {integration/java/nexmark/src/main/java/org/apache/beam/integration => sdks/java/nexmark/src/main/java/org/apache/beam/sdk}/nexmark/NexmarkSuite.java (98%) rename {integration/java/nexmark/src/main/java/org/apache/beam/integration => sdks/java/nexmark/src/main/java/org/apache/beam/sdk}/nexmark/NexmarkUtils.java (92%) rename {integration/java/nexmark/src/main/java/org/apache/beam/integration => sdks/java/nexmark/src/main/java/org/apache/beam/sdk}/nexmark/model/Auction.java (98%) rename {integration/java/nexmark/src/main/java/org/apache/beam/integration => sdks/java/nexmark/src/main/java/org/apache/beam/sdk}/nexmark/model/AuctionBid.java (92%) rename {integration/java/nexmark/src/main/java/org/apache/beam/integration => sdks/java/nexmark/src/main/java/org/apache/beam/sdk}/nexmark/model/AuctionCount.java (95%) rename {integration/java/nexmark/src/main/java/org/apache/beam/integration => sdks/java/nexmark/src/main/java/org/apache/beam/sdk}/nexmark/model/AuctionPrice.java (95%) rename {integration/java/nexmark/src/main/java/org/apache/beam/integration => sdks/java/nexmark/src/main/java/org/apache/beam/sdk}/nexmark/model/Bid.java (98%) rename {integration/java/nexmark/src/main/java/org/apache/beam/integration => sdks/java/nexmark/src/main/java/org/apache/beam/sdk}/nexmark/model/BidsPerSession.java (96%) rename {integration/java/nexmark/src/main/java/org/apache/beam/integration => sdks/java/nexmark/src/main/java/org/apache/beam/sdk}/nexmark/model/CategoryPrice.java (96%) rename {integration/java/nexmark/src/main/java/org/apache/beam/integration => sdks/java/nexmark/src/main/java/org/apache/beam/sdk}/nexmark/model/Done.java (95%) rename {integration/java/nexmark/src/main/java/org/apache/beam/integration => sdks/java/nexmark/src/main/java/org/apache/beam/sdk}/nexmark/model/Event.java (99%) rename {integration/java/nexmark/src/main/java/org/apache/beam/integration => sdks/java/nexmark/src/main/java/org/apache/beam/sdk}/nexmark/model/IdNameReserve.java (96%) rename {integration/java/nexmark/src/main/java/org/apache/beam/integration => sdks/java/nexmark/src/main/java/org/apache/beam/sdk}/nexmark/model/KnownSize.java (94%) rename {integration/java/nexmark/src/main/java/org/apache/beam/integration => sdks/java/nexmark/src/main/java/org/apache/beam/sdk}/nexmark/model/NameCityStateId.java (96%) rename {integration/java/nexmark/src/main/java/org/apache/beam/integration => sdks/java/nexmark/src/main/java/org/apache/beam/sdk}/nexmark/model/Person.java (97%) rename {integration/java/nexmark/src/main/java/org/apache/beam/integration => sdks/java/nexmark/src/main/java/org/apache/beam/sdk}/nexmark/model/SellerPrice.java (96%) rename {integration/java/nexmark/src/main/java/org/apache/beam/integration => sdks/java/nexmark/src/main/java/org/apache/beam/sdk}/nexmark/model/package-info.java (94%) rename {integration/java/nexmark/src/main/java/org/apache/beam/integration => sdks/java/nexmark/src/main/java/org/apache/beam/sdk}/nexmark/package-info.java (94%) rename {integration/java/nexmark/src/main/java/org/apache/beam/integration => sdks/java/nexmark/src/main/java/org/apache/beam/sdk}/nexmark/queries/AbstractSimulator.java (98%) rename {integration/java/nexmark/src/main/java/org/apache/beam/integration => sdks/java/nexmark/src/main/java/org/apache/beam/sdk}/nexmark/queries/NexmarkQuery.java (95%) rename {integration/java/nexmark/src/main/java/org/apache/beam/integration => sdks/java/nexmark/src/main/java/org/apache/beam/sdk}/nexmark/queries/NexmarkQueryModel.java (95%) rename {integration/java/nexmark/src/main/java/org/apache/beam/integration => sdks/java/nexmark/src/main/java/org/apache/beam/sdk}/nexmark/queries/Query0.java (90%) rename {integration/java/nexmark/src/main/java/org/apache/beam/integration => sdks/java/nexmark/src/main/java/org/apache/beam/sdk}/nexmark/queries/Query0Model.java (88%) rename {integration/java/nexmark/src/main/java/org/apache/beam/integration => sdks/java/nexmark/src/main/java/org/apache/beam/sdk}/nexmark/queries/Query1.java (86%) rename {integration/java/nexmark/src/main/java/org/apache/beam/integration => sdks/java/nexmark/src/main/java/org/apache/beam/sdk}/nexmark/queries/Query10.java (97%) rename {integration/java/nexmark/src/main/java/org/apache/beam/integration => sdks/java/nexmark/src/main/java/org/apache/beam/sdk}/nexmark/queries/Query11.java (87%) rename {integration/java/nexmark/src/main/java/org/apache/beam/integration => sdks/java/nexmark/src/main/java/org/apache/beam/sdk}/nexmark/queries/Query12.java (87%) rename {integration/java/nexmark/src/main/java/org/apache/beam/integration => sdks/java/nexmark/src/main/java/org/apache/beam/sdk}/nexmark/queries/Query1Model.java (88%) rename {integration/java/nexmark/src/main/java/org/apache/beam/integration => sdks/java/nexmark/src/main/java/org/apache/beam/sdk}/nexmark/queries/Query2.java (86%) rename {integration/java/nexmark/src/main/java/org/apache/beam/integration => sdks/java/nexmark/src/main/java/org/apache/beam/sdk}/nexmark/queries/Query2Model.java (87%) rename {integration/java/nexmark/src/main/java/org/apache/beam/integration => sdks/java/nexmark/src/main/java/org/apache/beam/sdk}/nexmark/queries/Query3.java (96%) rename {integration/java/nexmark/src/main/java/org/apache/beam/integration => sdks/java/nexmark/src/main/java/org/apache/beam/sdk}/nexmark/queries/Query3Model.java (90%) rename {integration/java/nexmark/src/main/java/org/apache/beam/integration => sdks/java/nexmark/src/main/java/org/apache/beam/sdk}/nexmark/queries/Query4.java (88%) rename {integration/java/nexmark/src/main/java/org/apache/beam/integration => sdks/java/nexmark/src/main/java/org/apache/beam/sdk}/nexmark/queries/Query4Model.java (93%) rename {integration/java/nexmark/src/main/java/org/apache/beam/integration => sdks/java/nexmark/src/main/java/org/apache/beam/sdk}/nexmark/queries/Query5.java (92%) rename {integration/java/nexmark/src/main/java/org/apache/beam/integration => sdks/java/nexmark/src/main/java/org/apache/beam/sdk}/nexmark/queries/Query5Model.java (94%) rename {integration/java/nexmark/src/main/java/org/apache/beam/integration => sdks/java/nexmark/src/main/java/org/apache/beam/sdk}/nexmark/queries/Query6.java (91%) rename {integration/java/nexmark/src/main/java/org/apache/beam/integration => sdks/java/nexmark/src/main/java/org/apache/beam/sdk}/nexmark/queries/Query6Model.java (90%) rename {integration/java/nexmark/src/main/java/org/apache/beam/integration => sdks/java/nexmark/src/main/java/org/apache/beam/sdk}/nexmark/queries/Query7.java (91%) rename {integration/java/nexmark/src/main/java/org/apache/beam/integration => sdks/java/nexmark/src/main/java/org/apache/beam/sdk}/nexmark/queries/Query7Model.java (93%) rename {integration/java/nexmark/src/main/java/org/apache/beam/integration => sdks/java/nexmark/src/main/java/org/apache/beam/sdk}/nexmark/queries/Query8.java (88%) rename {integration/java/nexmark/src/main/java/org/apache/beam/integration => sdks/java/nexmark/src/main/java/org/apache/beam/sdk}/nexmark/queries/Query8Model.java (92%) rename {integration/java/nexmark/src/main/java/org/apache/beam/integration => sdks/java/nexmark/src/main/java/org/apache/beam/sdk}/nexmark/queries/Query9.java (80%) rename {integration/java/nexmark/src/main/java/org/apache/beam/integration => sdks/java/nexmark/src/main/java/org/apache/beam/sdk}/nexmark/queries/Query9Model.java (92%) rename {integration/java/nexmark/src/main/java/org/apache/beam/integration => sdks/java/nexmark/src/main/java/org/apache/beam/sdk}/nexmark/queries/WinningBids.java (97%) rename {integration/java/nexmark/src/main/java/org/apache/beam/integration => sdks/java/nexmark/src/main/java/org/apache/beam/sdk}/nexmark/queries/WinningBidsSimulator.java (94%) rename {integration/java/nexmark/src/main/java/org/apache/beam/integration => sdks/java/nexmark/src/main/java/org/apache/beam/sdk}/nexmark/queries/package-info.java (94%) rename {integration/java/nexmark/src/main/java/org/apache/beam/integration => sdks/java/nexmark/src/main/java/org/apache/beam/sdk}/nexmark/sources/BoundedEventSource.java (97%) rename {integration/java/nexmark/src/main/java/org/apache/beam/integration => sdks/java/nexmark/src/main/java/org/apache/beam/sdk}/nexmark/sources/Generator.java (98%) rename {integration/java/nexmark/src/main/java/org/apache/beam/integration => sdks/java/nexmark/src/main/java/org/apache/beam/sdk}/nexmark/sources/GeneratorConfig.java (97%) rename {integration/java/nexmark/src/main/java/org/apache/beam/integration => sdks/java/nexmark/src/main/java/org/apache/beam/sdk}/nexmark/sources/UnboundedEventSource.java (98%) rename {integration/java/nexmark/src/main/java/org/apache/beam/integration => sdks/java/nexmark/src/main/java/org/apache/beam/sdk}/nexmark/sources/package-info.java (94%) rename {integration => sdks}/java/nexmark/src/main/resources/log4j.properties (100%) rename {integration/java/nexmark/src/test/java/org/apache/beam/integration => sdks/java/nexmark/src/test/java/org/apache/beam/sdk}/nexmark/queries/QueryTest.java (96%) rename {integration/java/nexmark/src/test/java/org/apache/beam/integration => sdks/java/nexmark/src/test/java/org/apache/beam/sdk}/nexmark/sources/BoundedEventSourceTest.java (93%) rename {integration/java/nexmark/src/test/java/org/apache/beam/integration => sdks/java/nexmark/src/test/java/org/apache/beam/sdk}/nexmark/sources/GeneratorTest.java (96%) rename {integration/java/nexmark/src/test/java/org/apache/beam/integration => sdks/java/nexmark/src/test/java/org/apache/beam/sdk}/nexmark/sources/UnboundedEventSourceTest.java (95%) diff --git a/integration/java/pom.xml b/integration/java/pom.xml deleted file mode 100644 index b0c38531a5d51..0000000000000 --- a/integration/java/pom.xml +++ /dev/null @@ -1,37 +0,0 @@ - - - - - 4.0.0 - - - org.apache.beam - beam-integration-parent - 2.1.0-SNAPSHOT - ../pom.xml - - - beam-integration-java-parent - pom - Apache Beam :: Integration Tests :: Java - - - nexmark - - - diff --git a/integration/pom.xml b/integration/pom.xml deleted file mode 100644 index 42548191d65e9..0000000000000 --- a/integration/pom.xml +++ /dev/null @@ -1,51 +0,0 @@ - - - - - 4.0.0 - - - org.apache.beam - beam-parent - 2.1.0-SNAPSHOT - ../pom.xml - - - beam-integration-parent - pom - Apache Beam :: Integration Tests - - - - release - - - - org.codehaus.mojo - findbugs-maven-plugin - - - - - - - - java - - - diff --git a/pom.xml b/pom.xml index bddbf1f90ae02..5fd12977972f3 100644 --- a/pom.xml +++ b/pom.xml @@ -187,7 +187,6 @@ sdks runners examples - integration sdks/java/javadoc diff --git a/integration/java/nexmark/README.md b/sdks/java/nexmark/README.md similarity index 100% rename from integration/java/nexmark/README.md rename to sdks/java/nexmark/README.md diff --git a/integration/java/nexmark/pom.xml b/sdks/java/nexmark/pom.xml similarity index 97% rename from integration/java/nexmark/pom.xml rename to sdks/java/nexmark/pom.xml index 664a410813226..c1b6025c5e685 100644 --- a/integration/java/nexmark/pom.xml +++ b/sdks/java/nexmark/pom.xml @@ -22,13 +22,13 @@ org.apache.beam - beam-integration-java-parent + beam-sdks-java-parent 2.1.0-SNAPSHOT ../pom.xml - beam-integration-java-nexmark - Apache Beam :: Integration Tests :: Java :: Nexmark + beam-sdks-java-nexmark + Apache Beam :: SDKs :: Java :: Nexmark jar diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Main.java b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/Main.java similarity index 98% rename from integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Main.java rename to sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/Main.java index 4c23651c04c61..ab2284c7d471e 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Main.java +++ b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/Main.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.integration.nexmark; +package org.apache.beam.sdk.nexmark; import java.io.IOException; import java.nio.charset.StandardCharsets; @@ -27,10 +27,9 @@ import java.util.List; import java.util.Map; import javax.annotation.Nullable; - -import org.apache.beam.integration.nexmark.model.Auction; -import org.apache.beam.integration.nexmark.model.Bid; -import org.apache.beam.integration.nexmark.model.Person; +import org.apache.beam.sdk.nexmark.model.Auction; +import org.apache.beam.sdk.nexmark.model.Bid; +import org.apache.beam.sdk.nexmark.model.Person; import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.joda.time.Duration; import org.joda.time.Instant; diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Monitor.java b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/Monitor.java similarity index 96% rename from integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Monitor.java rename to sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/Monitor.java index 2f0c56a8e34b2..f45c38709b4d8 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/Monitor.java +++ b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/Monitor.java @@ -15,14 +15,13 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.integration.nexmark; +package org.apache.beam.sdk.nexmark; import java.io.Serializable; - -import org.apache.beam.integration.nexmark.model.KnownSize; import org.apache.beam.sdk.metrics.Counter; import org.apache.beam.sdk.metrics.Distribution; import org.apache.beam.sdk.metrics.Metrics; +import org.apache.beam.sdk.nexmark.model.KnownSize; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.ParDo; diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkConfiguration.java b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/NexmarkConfiguration.java similarity index 99% rename from integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkConfiguration.java rename to sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/NexmarkConfiguration.java index 2faf3f523ed8d..904fcd5cbcbb0 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkConfiguration.java +++ b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/NexmarkConfiguration.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.integration.nexmark; +package org.apache.beam.sdk.nexmark; import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.core.JsonProcessingException; diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkLauncher.java b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/NexmarkLauncher.java similarity index 95% rename from integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkLauncher.java rename to sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/NexmarkLauncher.java index a60997529a018..550fbd2ce84a7 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkLauncher.java +++ b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/NexmarkLauncher.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.integration.nexmark; +package org.apache.beam.sdk.nexmark; import static com.google.common.base.Preconditions.checkState; @@ -34,36 +34,6 @@ import java.util.NoSuchElementException; import java.util.concurrent.ThreadLocalRandom; import javax.annotation.Nullable; -import org.apache.beam.integration.nexmark.model.Auction; -import org.apache.beam.integration.nexmark.model.Bid; -import org.apache.beam.integration.nexmark.model.Event; -import org.apache.beam.integration.nexmark.model.KnownSize; -import org.apache.beam.integration.nexmark.model.Person; -import org.apache.beam.integration.nexmark.queries.NexmarkQuery; -import org.apache.beam.integration.nexmark.queries.NexmarkQueryModel; -import org.apache.beam.integration.nexmark.queries.Query0; -import org.apache.beam.integration.nexmark.queries.Query0Model; -import org.apache.beam.integration.nexmark.queries.Query1; -import org.apache.beam.integration.nexmark.queries.Query10; -import org.apache.beam.integration.nexmark.queries.Query11; -import org.apache.beam.integration.nexmark.queries.Query12; -import org.apache.beam.integration.nexmark.queries.Query1Model; -import org.apache.beam.integration.nexmark.queries.Query2; -import org.apache.beam.integration.nexmark.queries.Query2Model; -import org.apache.beam.integration.nexmark.queries.Query3; -import org.apache.beam.integration.nexmark.queries.Query3Model; -import org.apache.beam.integration.nexmark.queries.Query4; -import org.apache.beam.integration.nexmark.queries.Query4Model; -import org.apache.beam.integration.nexmark.queries.Query5; -import org.apache.beam.integration.nexmark.queries.Query5Model; -import org.apache.beam.integration.nexmark.queries.Query6; -import org.apache.beam.integration.nexmark.queries.Query6Model; -import org.apache.beam.integration.nexmark.queries.Query7; -import org.apache.beam.integration.nexmark.queries.Query7Model; -import org.apache.beam.integration.nexmark.queries.Query8; -import org.apache.beam.integration.nexmark.queries.Query8Model; -import org.apache.beam.integration.nexmark.queries.Query9; -import org.apache.beam.integration.nexmark.queries.Query9Model; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.PipelineResult; import org.apache.beam.sdk.coders.CoderException; @@ -77,6 +47,36 @@ import org.apache.beam.sdk.metrics.MetricQueryResults; import org.apache.beam.sdk.metrics.MetricResult; import org.apache.beam.sdk.metrics.MetricsFilter; +import org.apache.beam.sdk.nexmark.model.Auction; +import org.apache.beam.sdk.nexmark.model.Bid; +import org.apache.beam.sdk.nexmark.model.Event; +import org.apache.beam.sdk.nexmark.model.KnownSize; +import org.apache.beam.sdk.nexmark.model.Person; +import org.apache.beam.sdk.nexmark.queries.NexmarkQuery; +import org.apache.beam.sdk.nexmark.queries.NexmarkQueryModel; +import org.apache.beam.sdk.nexmark.queries.Query0; +import org.apache.beam.sdk.nexmark.queries.Query0Model; +import org.apache.beam.sdk.nexmark.queries.Query1; +import org.apache.beam.sdk.nexmark.queries.Query10; +import org.apache.beam.sdk.nexmark.queries.Query11; +import org.apache.beam.sdk.nexmark.queries.Query12; +import org.apache.beam.sdk.nexmark.queries.Query1Model; +import org.apache.beam.sdk.nexmark.queries.Query2; +import org.apache.beam.sdk.nexmark.queries.Query2Model; +import org.apache.beam.sdk.nexmark.queries.Query3; +import org.apache.beam.sdk.nexmark.queries.Query3Model; +import org.apache.beam.sdk.nexmark.queries.Query4; +import org.apache.beam.sdk.nexmark.queries.Query4Model; +import org.apache.beam.sdk.nexmark.queries.Query5; +import org.apache.beam.sdk.nexmark.queries.Query5Model; +import org.apache.beam.sdk.nexmark.queries.Query6; +import org.apache.beam.sdk.nexmark.queries.Query6Model; +import org.apache.beam.sdk.nexmark.queries.Query7; +import org.apache.beam.sdk.nexmark.queries.Query7Model; +import org.apache.beam.sdk.nexmark.queries.Query8; +import org.apache.beam.sdk.nexmark.queries.Query8Model; +import org.apache.beam.sdk.nexmark.queries.Query9; +import org.apache.beam.sdk.nexmark.queries.Query9Model; import org.apache.beam.sdk.testing.PAssert; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.ParDo; @@ -203,8 +203,7 @@ private long getDistributionMetric(PipelineResult result, String namespace, Stri Iterable> distributions = metrics.distributions(); try { MetricResult distributionResult = distributions.iterator().next(); - switch (distType) - { + switch (distType) { case MIN: return distributionResult.attempted().min(); case MAX: diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkOptions.java b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/NexmarkOptions.java similarity index 99% rename from integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkOptions.java rename to sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/NexmarkOptions.java index fbd3e74acfe69..2a2a5a782a659 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkOptions.java +++ b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/NexmarkOptions.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.integration.nexmark; +package org.apache.beam.sdk.nexmark; import javax.annotation.Nullable; import org.apache.beam.sdk.extensions.gcp.options.GcpOptions; diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkPerf.java b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/NexmarkPerf.java similarity index 99% rename from integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkPerf.java rename to sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/NexmarkPerf.java index e7f59c84d2a8c..2edf4e8e6f2a1 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkPerf.java +++ b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/NexmarkPerf.java @@ -15,11 +15,10 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.integration.nexmark; +package org.apache.beam.sdk.nexmark; import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.core.JsonProcessingException; - import java.io.IOException; import java.util.List; import javax.annotation.Nullable; diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkSuite.java b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/NexmarkSuite.java similarity index 98% rename from integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkSuite.java rename to sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/NexmarkSuite.java index 0d98a5dbc164d..d38cb7bb9495d 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkSuite.java +++ b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/NexmarkSuite.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.integration.nexmark; +package org.apache.beam.sdk.nexmark; import java.util.ArrayList; import java.util.LinkedHashSet; diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkUtils.java b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/NexmarkUtils.java similarity index 92% rename from integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkUtils.java rename to sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/NexmarkUtils.java index 792669033969f..fa1ef165f612b 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/NexmarkUtils.java +++ b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/NexmarkUtils.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.integration.nexmark; +package org.apache.beam.sdk.nexmark; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.hash.Hashing; @@ -24,24 +24,6 @@ import java.io.OutputStream; import java.nio.charset.StandardCharsets; import java.util.Iterator; -import org.apache.beam.integration.nexmark.model.Auction; -import org.apache.beam.integration.nexmark.model.AuctionBid; -import org.apache.beam.integration.nexmark.model.AuctionCount; -import org.apache.beam.integration.nexmark.model.AuctionPrice; -import org.apache.beam.integration.nexmark.model.Bid; -import org.apache.beam.integration.nexmark.model.BidsPerSession; -import org.apache.beam.integration.nexmark.model.CategoryPrice; -import org.apache.beam.integration.nexmark.model.Done; -import org.apache.beam.integration.nexmark.model.Event; -import org.apache.beam.integration.nexmark.model.IdNameReserve; -import org.apache.beam.integration.nexmark.model.KnownSize; -import org.apache.beam.integration.nexmark.model.NameCityStateId; -import org.apache.beam.integration.nexmark.model.Person; -import org.apache.beam.integration.nexmark.model.SellerPrice; -import org.apache.beam.integration.nexmark.sources.BoundedEventSource; -import org.apache.beam.integration.nexmark.sources.Generator; -import org.apache.beam.integration.nexmark.sources.GeneratorConfig; -import org.apache.beam.integration.nexmark.sources.UnboundedEventSource; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.coders.AvroCoder; import org.apache.beam.sdk.coders.ByteArrayCoder; @@ -53,6 +35,24 @@ import org.apache.beam.sdk.io.Read; import org.apache.beam.sdk.metrics.Counter; import org.apache.beam.sdk.metrics.Metrics; +import org.apache.beam.sdk.nexmark.model.Auction; +import org.apache.beam.sdk.nexmark.model.AuctionBid; +import org.apache.beam.sdk.nexmark.model.AuctionCount; +import org.apache.beam.sdk.nexmark.model.AuctionPrice; +import org.apache.beam.sdk.nexmark.model.Bid; +import org.apache.beam.sdk.nexmark.model.BidsPerSession; +import org.apache.beam.sdk.nexmark.model.CategoryPrice; +import org.apache.beam.sdk.nexmark.model.Done; +import org.apache.beam.sdk.nexmark.model.Event; +import org.apache.beam.sdk.nexmark.model.IdNameReserve; +import org.apache.beam.sdk.nexmark.model.KnownSize; +import org.apache.beam.sdk.nexmark.model.NameCityStateId; +import org.apache.beam.sdk.nexmark.model.Person; +import org.apache.beam.sdk.nexmark.model.SellerPrice; +import org.apache.beam.sdk.nexmark.sources.BoundedEventSource; +import org.apache.beam.sdk.nexmark.sources.Generator; +import org.apache.beam.sdk.nexmark.sources.GeneratorConfig; +import org.apache.beam.sdk.nexmark.sources.UnboundedEventSource; import org.apache.beam.sdk.state.StateSpec; import org.apache.beam.sdk.state.StateSpecs; import org.apache.beam.sdk.state.ValueState; @@ -568,16 +568,18 @@ private DiskBusyTransform(long bytes) { @Override public PCollection expand(PCollection input) { // Add dummy key to be able to use State API - PCollection> kvCollection = input.apply("diskBusy.keyElements", ParDo.of(new DoFn>() { + PCollection> kvCollection = input + .apply("diskBusy.keyElements", ParDo.of(new DoFn>() { - @ProcessElement public void processElement(ProcessContext context) { - context.output(KV.of(0, context.element())); + @ProcessElement public void processElement(ProcessContext context) { + context.output(KV.of(0, context.element())); } })); // Apply actual transform that generates disk IO using state API - PCollection output = kvCollection.apply("diskBusy.generateIO", ParDo.of(new DoFn, T>() { + PCollection output = kvCollection + .apply("diskBusy.generateIO", ParDo.of(new DoFn, T>() { - private static final String DISK_BUSY = "diskBusy"; + private static final String DISK_BUSY = "diskBusy"; @StateId(DISK_BUSY) private final StateSpec> spec = StateSpecs .value(ByteArrayCoder.of()); diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/Auction.java b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/model/Auction.java similarity index 98% rename from integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/Auction.java rename to sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/model/Auction.java index 9f5d7c0afdb60..6a37ade016214 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/Auction.java +++ b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/model/Auction.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.integration.nexmark.model; +package org.apache.beam.sdk.nexmark.model; import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.core.JsonProcessingException; @@ -23,12 +23,12 @@ import java.io.InputStream; import java.io.OutputStream; import java.io.Serializable; -import org.apache.beam.integration.nexmark.NexmarkUtils; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.CoderException; import org.apache.beam.sdk.coders.CustomCoder; import org.apache.beam.sdk.coders.StringUtf8Coder; import org.apache.beam.sdk.coders.VarLongCoder; +import org.apache.beam.sdk.nexmark.NexmarkUtils; /** * An auction submitted by a person. diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/AuctionBid.java b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/model/AuctionBid.java similarity index 92% rename from integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/AuctionBid.java rename to sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/model/AuctionBid.java index b9d79db8b9c0a..cb1aac5045da0 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/AuctionBid.java +++ b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/model/AuctionBid.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.integration.nexmark.model; +package org.apache.beam.sdk.nexmark.model; import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.core.JsonProcessingException; @@ -23,13 +23,14 @@ import java.io.InputStream; import java.io.OutputStream; import java.io.Serializable; -import org.apache.beam.integration.nexmark.NexmarkUtils; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.CoderException; import org.apache.beam.sdk.coders.CustomCoder; +import org.apache.beam.sdk.nexmark.NexmarkUtils; +import org.apache.beam.sdk.nexmark.queries.WinningBids; /** - * Result of {@link org.apache.beam.integration.nexmark.queries.WinningBids} transform. + * Result of {@link WinningBids} transform. */ public class AuctionBid implements KnownSize, Serializable { public static final Coder CODER = new CustomCoder() { diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/AuctionCount.java b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/model/AuctionCount.java similarity index 95% rename from integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/AuctionCount.java rename to sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/model/AuctionCount.java index 0e643ff152770..4d15d25fae2fd 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/AuctionCount.java +++ b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/model/AuctionCount.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.integration.nexmark.model; +package org.apache.beam.sdk.nexmark.model; import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.core.JsonProcessingException; @@ -23,11 +23,11 @@ import java.io.InputStream; import java.io.OutputStream; import java.io.Serializable; -import org.apache.beam.integration.nexmark.NexmarkUtils; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.CoderException; import org.apache.beam.sdk.coders.CustomCoder; import org.apache.beam.sdk.coders.VarLongCoder; +import org.apache.beam.sdk.nexmark.NexmarkUtils; /** * Result of Query5. diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/AuctionPrice.java b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/model/AuctionPrice.java similarity index 95% rename from integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/AuctionPrice.java rename to sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/model/AuctionPrice.java index 7d51a21385b79..f4fe88119d699 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/AuctionPrice.java +++ b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/model/AuctionPrice.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.integration.nexmark.model; +package org.apache.beam.sdk.nexmark.model; import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.core.JsonProcessingException; @@ -23,11 +23,11 @@ import java.io.InputStream; import java.io.OutputStream; import java.io.Serializable; -import org.apache.beam.integration.nexmark.NexmarkUtils; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.CoderException; import org.apache.beam.sdk.coders.CustomCoder; import org.apache.beam.sdk.coders.VarLongCoder; +import org.apache.beam.sdk.nexmark.NexmarkUtils; /** * Result of Query2. diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/Bid.java b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/model/Bid.java similarity index 98% rename from integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/Bid.java rename to sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/model/Bid.java index 4fa9ea07c829f..b465e623b04a1 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/Bid.java +++ b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/model/Bid.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.integration.nexmark.model; +package org.apache.beam.sdk.nexmark.model; import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.core.JsonProcessingException; @@ -24,12 +24,12 @@ import java.io.OutputStream; import java.io.Serializable; import java.util.Comparator; -import org.apache.beam.integration.nexmark.NexmarkUtils; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.CoderException; import org.apache.beam.sdk.coders.CustomCoder; import org.apache.beam.sdk.coders.StringUtf8Coder; import org.apache.beam.sdk.coders.VarLongCoder; +import org.apache.beam.sdk.nexmark.NexmarkUtils; /** * A bid for an item on auction. diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/BidsPerSession.java b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/model/BidsPerSession.java similarity index 96% rename from integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/BidsPerSession.java rename to sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/model/BidsPerSession.java index 3211456fe6a8c..84e23e7eff178 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/BidsPerSession.java +++ b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/model/BidsPerSession.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.integration.nexmark.model; +package org.apache.beam.sdk.nexmark.model; import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.core.JsonProcessingException; @@ -23,11 +23,11 @@ import java.io.InputStream; import java.io.OutputStream; import java.io.Serializable; -import org.apache.beam.integration.nexmark.NexmarkUtils; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.CoderException; import org.apache.beam.sdk.coders.CustomCoder; import org.apache.beam.sdk.coders.VarLongCoder; +import org.apache.beam.sdk.nexmark.NexmarkUtils; /** * Result of query 11. diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/CategoryPrice.java b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/model/CategoryPrice.java similarity index 96% rename from integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/CategoryPrice.java rename to sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/model/CategoryPrice.java index 2678198470a29..3b33635969ea8 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/CategoryPrice.java +++ b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/model/CategoryPrice.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.integration.nexmark.model; +package org.apache.beam.sdk.nexmark.model; import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.core.JsonProcessingException; @@ -23,12 +23,12 @@ import java.io.InputStream; import java.io.OutputStream; import java.io.Serializable; -import org.apache.beam.integration.nexmark.NexmarkUtils; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.CoderException; import org.apache.beam.sdk.coders.CustomCoder; import org.apache.beam.sdk.coders.VarIntCoder; import org.apache.beam.sdk.coders.VarLongCoder; +import org.apache.beam.sdk.nexmark.NexmarkUtils; /** * Result of Query4. diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/Done.java b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/model/Done.java similarity index 95% rename from integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/Done.java rename to sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/model/Done.java index b0a88d4642feb..e285041c6d63c 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/Done.java +++ b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/model/Done.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.integration.nexmark.model; +package org.apache.beam.sdk.nexmark.model; import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.core.JsonProcessingException; @@ -23,11 +23,11 @@ import java.io.InputStream; import java.io.OutputStream; import java.io.Serializable; -import org.apache.beam.integration.nexmark.NexmarkUtils; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.CoderException; import org.apache.beam.sdk.coders.CustomCoder; import org.apache.beam.sdk.coders.StringUtf8Coder; +import org.apache.beam.sdk.nexmark.NexmarkUtils; /** * Result of query 10. diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/Event.java b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/model/Event.java similarity index 99% rename from integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/Event.java rename to sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/model/Event.java index 0e1672eb3801c..880cfe4d405c6 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/Event.java +++ b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/model/Event.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.integration.nexmark.model; +package org.apache.beam.sdk.nexmark.model; import java.io.IOException; import java.io.InputStream; diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/IdNameReserve.java b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/model/IdNameReserve.java similarity index 96% rename from integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/IdNameReserve.java rename to sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/model/IdNameReserve.java index 8cade4e8996da..0519f5d2547c0 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/IdNameReserve.java +++ b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/model/IdNameReserve.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.integration.nexmark.model; +package org.apache.beam.sdk.nexmark.model; import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.core.JsonProcessingException; @@ -23,12 +23,12 @@ import java.io.InputStream; import java.io.OutputStream; import java.io.Serializable; -import org.apache.beam.integration.nexmark.NexmarkUtils; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.CoderException; import org.apache.beam.sdk.coders.CustomCoder; import org.apache.beam.sdk.coders.StringUtf8Coder; import org.apache.beam.sdk.coders.VarLongCoder; +import org.apache.beam.sdk.nexmark.NexmarkUtils; /** * Result type of Query8. diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/KnownSize.java b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/model/KnownSize.java similarity index 94% rename from integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/KnownSize.java rename to sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/model/KnownSize.java index c742eac2105cc..45af3fc7f7025 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/KnownSize.java +++ b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/model/KnownSize.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.integration.nexmark.model; +package org.apache.beam.sdk.nexmark.model; /** * Interface for elements which can quickly estimate their encoded byte size. diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/NameCityStateId.java b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/model/NameCityStateId.java similarity index 96% rename from integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/NameCityStateId.java rename to sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/model/NameCityStateId.java index 37bd3c69468f7..55fca6204505e 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/NameCityStateId.java +++ b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/model/NameCityStateId.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.integration.nexmark.model; +package org.apache.beam.sdk.nexmark.model; import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.core.JsonProcessingException; @@ -23,12 +23,12 @@ import java.io.InputStream; import java.io.OutputStream; import java.io.Serializable; -import org.apache.beam.integration.nexmark.NexmarkUtils; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.CoderException; import org.apache.beam.sdk.coders.CustomCoder; import org.apache.beam.sdk.coders.StringUtf8Coder; import org.apache.beam.sdk.coders.VarLongCoder; +import org.apache.beam.sdk.nexmark.NexmarkUtils; /** * Result of Query3. diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/Person.java b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/model/Person.java similarity index 97% rename from integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/Person.java rename to sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/model/Person.java index bde587dd5e345..800f937eadee7 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/Person.java +++ b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/model/Person.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.integration.nexmark.model; +package org.apache.beam.sdk.nexmark.model; import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.core.JsonProcessingException; @@ -23,12 +23,12 @@ import java.io.InputStream; import java.io.OutputStream; import java.io.Serializable; -import org.apache.beam.integration.nexmark.NexmarkUtils; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.CoderException; import org.apache.beam.sdk.coders.CustomCoder; import org.apache.beam.sdk.coders.StringUtf8Coder; import org.apache.beam.sdk.coders.VarLongCoder; +import org.apache.beam.sdk.nexmark.NexmarkUtils; /** * A person either creating an auction or making a bid. diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/SellerPrice.java b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/model/SellerPrice.java similarity index 96% rename from integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/SellerPrice.java rename to sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/model/SellerPrice.java index 61537f6be9bf0..82b551c301f61 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/SellerPrice.java +++ b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/model/SellerPrice.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.integration.nexmark.model; +package org.apache.beam.sdk.nexmark.model; import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.core.JsonProcessingException; @@ -23,11 +23,11 @@ import java.io.InputStream; import java.io.OutputStream; import java.io.Serializable; -import org.apache.beam.integration.nexmark.NexmarkUtils; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.CoderException; import org.apache.beam.sdk.coders.CustomCoder; import org.apache.beam.sdk.coders.VarLongCoder; +import org.apache.beam.sdk.nexmark.NexmarkUtils; /** * Result of Query6. diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/package-info.java b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/model/package-info.java similarity index 94% rename from integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/package-info.java rename to sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/model/package-info.java index e1d611337d3e6..3b4bb6332d047 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/model/package-info.java +++ b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/model/package-info.java @@ -19,4 +19,4 @@ /** * Nexmark Benchmark Model. */ -package org.apache.beam.integration.nexmark.model; +package org.apache.beam.sdk.nexmark.model; diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/package-info.java b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/package-info.java similarity index 94% rename from integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/package-info.java rename to sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/package-info.java index df6f09f0ae24d..7500a247badff 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/package-info.java +++ b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/package-info.java @@ -18,4 +18,4 @@ /** * Nexmark. */ -package org.apache.beam.integration.nexmark; +package org.apache.beam.sdk.nexmark; diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/AbstractSimulator.java b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/AbstractSimulator.java similarity index 98% rename from integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/AbstractSimulator.java rename to sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/AbstractSimulator.java index 1395182ba8b05..6f4ad568a7db6 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/AbstractSimulator.java +++ b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/AbstractSimulator.java @@ -16,14 +16,14 @@ * limitations under the License. */ -package org.apache.beam.integration.nexmark.queries; +package org.apache.beam.sdk.nexmark.queries; import java.util.ArrayList; import java.util.Iterator; import java.util.List; import javax.annotation.Nullable; -import org.apache.beam.integration.nexmark.NexmarkUtils; +import org.apache.beam.sdk.nexmark.NexmarkUtils; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.values.TimestampedValue; import org.joda.time.Duration; diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/NexmarkQuery.java b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/NexmarkQuery.java similarity index 95% rename from integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/NexmarkQuery.java rename to sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/NexmarkQuery.java index 8b74282694ff4..d070058dff9b7 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/NexmarkQuery.java +++ b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/NexmarkQuery.java @@ -15,18 +15,18 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.integration.nexmark.queries; +package org.apache.beam.sdk.nexmark.queries; -import org.apache.beam.integration.nexmark.Monitor; -import org.apache.beam.integration.nexmark.NexmarkConfiguration; -import org.apache.beam.integration.nexmark.NexmarkUtils; -import org.apache.beam.integration.nexmark.model.Auction; -import org.apache.beam.integration.nexmark.model.Bid; -import org.apache.beam.integration.nexmark.model.Event; -import org.apache.beam.integration.nexmark.model.KnownSize; -import org.apache.beam.integration.nexmark.model.Person; import org.apache.beam.sdk.metrics.Counter; import org.apache.beam.sdk.metrics.Metrics; +import org.apache.beam.sdk.nexmark.Monitor; +import org.apache.beam.sdk.nexmark.NexmarkConfiguration; +import org.apache.beam.sdk.nexmark.NexmarkUtils; +import org.apache.beam.sdk.nexmark.model.Auction; +import org.apache.beam.sdk.nexmark.model.Bid; +import org.apache.beam.sdk.nexmark.model.Event; +import org.apache.beam.sdk.nexmark.model.KnownSize; +import org.apache.beam.sdk.nexmark.model.Person; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.Filter; import org.apache.beam.sdk.transforms.PTransform; diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/NexmarkQueryModel.java b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/NexmarkQueryModel.java similarity index 95% rename from integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/NexmarkQueryModel.java rename to sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/NexmarkQueryModel.java index bfa668bc929a2..1f093a0330463 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/NexmarkQueryModel.java +++ b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/NexmarkQueryModel.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.integration.nexmark.queries; +package org.apache.beam.sdk.nexmark.queries; import java.io.Serializable; import java.util.ArrayList; @@ -25,8 +25,8 @@ import java.util.List; import java.util.Set; -import org.apache.beam.integration.nexmark.NexmarkConfiguration; -import org.apache.beam.integration.nexmark.model.KnownSize; +import org.apache.beam.sdk.nexmark.NexmarkConfiguration; +import org.apache.beam.sdk.nexmark.model.KnownSize; import org.apache.beam.sdk.transforms.SerializableFunction; import org.apache.beam.sdk.values.TimestampedValue; diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query0.java b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/Query0.java similarity index 90% rename from integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query0.java rename to sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/Query0.java index 00a49a8865023..68bf78e8f69cc 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query0.java +++ b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/Query0.java @@ -15,20 +15,19 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.integration.nexmark.queries; +package org.apache.beam.sdk.nexmark.queries; import java.io.ByteArrayInputStream; import java.io.ByteArrayOutputStream; import java.io.IOException; - -import org.apache.beam.integration.nexmark.NexmarkConfiguration; -import org.apache.beam.integration.nexmark.NexmarkUtils; -import org.apache.beam.integration.nexmark.model.Event; -import org.apache.beam.integration.nexmark.model.KnownSize; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.CoderException; import org.apache.beam.sdk.metrics.Counter; import org.apache.beam.sdk.metrics.Metrics; +import org.apache.beam.sdk.nexmark.NexmarkConfiguration; +import org.apache.beam.sdk.nexmark.NexmarkUtils; +import org.apache.beam.sdk.nexmark.model.Event; +import org.apache.beam.sdk.nexmark.model.KnownSize; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.ParDo; import org.apache.beam.sdk.values.PCollection; diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query0Model.java b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/Query0Model.java similarity index 88% rename from integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query0Model.java rename to sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/Query0Model.java index e2522b8b03698..0e73a21079d36 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query0Model.java +++ b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/Query0Model.java @@ -15,14 +15,14 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.integration.nexmark.queries; +package org.apache.beam.sdk.nexmark.queries; import java.util.Collection; import java.util.Iterator; -import org.apache.beam.integration.nexmark.NexmarkConfiguration; -import org.apache.beam.integration.nexmark.NexmarkUtils; -import org.apache.beam.integration.nexmark.model.Event; +import org.apache.beam.sdk.nexmark.NexmarkConfiguration; +import org.apache.beam.sdk.nexmark.NexmarkUtils; +import org.apache.beam.sdk.nexmark.model.Event; import org.apache.beam.sdk.values.TimestampedValue; /** diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query1.java b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/Query1.java similarity index 86% rename from integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query1.java rename to sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/Query1.java index 8d90b701a9090..810cd8719bdb1 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query1.java +++ b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/Query1.java @@ -15,13 +15,13 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.integration.nexmark.queries; +package org.apache.beam.sdk.nexmark.queries; -import org.apache.beam.integration.nexmark.NexmarkConfiguration; -import org.apache.beam.integration.nexmark.NexmarkUtils; -import org.apache.beam.integration.nexmark.model.Bid; -import org.apache.beam.integration.nexmark.model.Event; -import org.apache.beam.integration.nexmark.model.KnownSize; +import org.apache.beam.sdk.nexmark.NexmarkConfiguration; +import org.apache.beam.sdk.nexmark.NexmarkUtils; +import org.apache.beam.sdk.nexmark.model.Bid; +import org.apache.beam.sdk.nexmark.model.Event; +import org.apache.beam.sdk.nexmark.model.KnownSize; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.ParDo; import org.apache.beam.sdk.values.PCollection; diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query10.java b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/Query10.java similarity index 97% rename from integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query10.java rename to sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/Query10.java index 378d01e12f4e9..1c4e44377afbf 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query10.java +++ b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/Query10.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.integration.nexmark.queries; +package org.apache.beam.sdk.nexmark.queries; import java.io.IOException; import java.io.OutputStream; @@ -24,15 +24,15 @@ import java.nio.channels.WritableByteChannel; import java.util.concurrent.ThreadLocalRandom; import javax.annotation.Nullable; -import org.apache.beam.integration.nexmark.NexmarkConfiguration; -import org.apache.beam.integration.nexmark.NexmarkUtils; -import org.apache.beam.integration.nexmark.model.Done; -import org.apache.beam.integration.nexmark.model.Event; -import org.apache.beam.integration.nexmark.model.KnownSize; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.extensions.gcp.options.GcsOptions; import org.apache.beam.sdk.metrics.Counter; import org.apache.beam.sdk.metrics.Metrics; +import org.apache.beam.sdk.nexmark.NexmarkConfiguration; +import org.apache.beam.sdk.nexmark.NexmarkUtils; +import org.apache.beam.sdk.nexmark.model.Done; +import org.apache.beam.sdk.nexmark.model.Event; +import org.apache.beam.sdk.nexmark.model.KnownSize; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.GroupByKey; import org.apache.beam.sdk.transforms.ParDo; diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query11.java b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/Query11.java similarity index 87% rename from integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query11.java rename to sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/Query11.java index 6db9bcf81523b..47e7c007d635f 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query11.java +++ b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/Query11.java @@ -15,14 +15,14 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.integration.nexmark.queries; +package org.apache.beam.sdk.nexmark.queries; -import org.apache.beam.integration.nexmark.NexmarkConfiguration; -import org.apache.beam.integration.nexmark.NexmarkUtils; -import org.apache.beam.integration.nexmark.model.Bid; -import org.apache.beam.integration.nexmark.model.BidsPerSession; -import org.apache.beam.integration.nexmark.model.Event; -import org.apache.beam.integration.nexmark.model.KnownSize; +import org.apache.beam.sdk.nexmark.NexmarkConfiguration; +import org.apache.beam.sdk.nexmark.NexmarkUtils; +import org.apache.beam.sdk.nexmark.model.Bid; +import org.apache.beam.sdk.nexmark.model.BidsPerSession; +import org.apache.beam.sdk.nexmark.model.Event; +import org.apache.beam.sdk.nexmark.model.KnownSize; import org.apache.beam.sdk.transforms.Count; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.ParDo; diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query12.java b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/Query12.java similarity index 87% rename from integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query12.java rename to sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/Query12.java index 20f45fb1d60a1..0f4b2328d11e5 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query12.java +++ b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/Query12.java @@ -15,14 +15,14 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.integration.nexmark.queries; +package org.apache.beam.sdk.nexmark.queries; -import org.apache.beam.integration.nexmark.NexmarkConfiguration; -import org.apache.beam.integration.nexmark.NexmarkUtils; -import org.apache.beam.integration.nexmark.model.Bid; -import org.apache.beam.integration.nexmark.model.BidsPerSession; -import org.apache.beam.integration.nexmark.model.Event; -import org.apache.beam.integration.nexmark.model.KnownSize; +import org.apache.beam.sdk.nexmark.NexmarkConfiguration; +import org.apache.beam.sdk.nexmark.NexmarkUtils; +import org.apache.beam.sdk.nexmark.model.Bid; +import org.apache.beam.sdk.nexmark.model.BidsPerSession; +import org.apache.beam.sdk.nexmark.model.Event; +import org.apache.beam.sdk.nexmark.model.KnownSize; import org.apache.beam.sdk.transforms.Count; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.ParDo; diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query1Model.java b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/Query1Model.java similarity index 88% rename from integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query1Model.java rename to sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/Query1Model.java index f07db80b38292..76c182ade4a9d 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query1Model.java +++ b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/Query1Model.java @@ -15,16 +15,16 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.integration.nexmark.queries; +package org.apache.beam.sdk.nexmark.queries; import java.io.Serializable; import java.util.Collection; import java.util.Iterator; -import org.apache.beam.integration.nexmark.NexmarkConfiguration; -import org.apache.beam.integration.nexmark.NexmarkUtils; -import org.apache.beam.integration.nexmark.model.Bid; -import org.apache.beam.integration.nexmark.model.Event; +import org.apache.beam.sdk.nexmark.NexmarkConfiguration; +import org.apache.beam.sdk.nexmark.NexmarkUtils; +import org.apache.beam.sdk.nexmark.model.Bid; +import org.apache.beam.sdk.nexmark.model.Event; import org.apache.beam.sdk.values.TimestampedValue; /** diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query2.java b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/Query2.java similarity index 86% rename from integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query2.java rename to sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/Query2.java index a365b973b7b8c..c5ab99289b959 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query2.java +++ b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/Query2.java @@ -15,14 +15,14 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.integration.nexmark.queries; +package org.apache.beam.sdk.nexmark.queries; -import org.apache.beam.integration.nexmark.NexmarkConfiguration; -import org.apache.beam.integration.nexmark.NexmarkUtils; -import org.apache.beam.integration.nexmark.model.AuctionPrice; -import org.apache.beam.integration.nexmark.model.Bid; -import org.apache.beam.integration.nexmark.model.Event; -import org.apache.beam.integration.nexmark.model.KnownSize; +import org.apache.beam.sdk.nexmark.NexmarkConfiguration; +import org.apache.beam.sdk.nexmark.NexmarkUtils; +import org.apache.beam.sdk.nexmark.model.AuctionPrice; +import org.apache.beam.sdk.nexmark.model.Bid; +import org.apache.beam.sdk.nexmark.model.Event; +import org.apache.beam.sdk.nexmark.model.KnownSize; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.Filter; import org.apache.beam.sdk.transforms.ParDo; diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query2Model.java b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/Query2Model.java similarity index 87% rename from integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query2Model.java rename to sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/Query2Model.java index e00992f39d5bd..33a1f8d2ccd02 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query2Model.java +++ b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/Query2Model.java @@ -15,17 +15,17 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.integration.nexmark.queries; +package org.apache.beam.sdk.nexmark.queries; import java.io.Serializable; import java.util.Collection; import java.util.Iterator; -import org.apache.beam.integration.nexmark.NexmarkConfiguration; -import org.apache.beam.integration.nexmark.NexmarkUtils; -import org.apache.beam.integration.nexmark.model.AuctionPrice; -import org.apache.beam.integration.nexmark.model.Bid; -import org.apache.beam.integration.nexmark.model.Event; +import org.apache.beam.sdk.nexmark.NexmarkConfiguration; +import org.apache.beam.sdk.nexmark.NexmarkUtils; +import org.apache.beam.sdk.nexmark.model.AuctionPrice; +import org.apache.beam.sdk.nexmark.model.Bid; +import org.apache.beam.sdk.nexmark.model.Event; import org.apache.beam.sdk.values.TimestampedValue; /** diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query3.java b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/Query3.java similarity index 96% rename from integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query3.java rename to sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/Query3.java index f2b66d7ade38e..6f8d72dae925f 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query3.java +++ b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/Query3.java @@ -15,20 +15,20 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.integration.nexmark.queries; +package org.apache.beam.sdk.nexmark.queries; import java.util.ArrayList; import java.util.List; -import org.apache.beam.integration.nexmark.NexmarkConfiguration; -import org.apache.beam.integration.nexmark.NexmarkUtils; -import org.apache.beam.integration.nexmark.model.Auction; -import org.apache.beam.integration.nexmark.model.Event; -import org.apache.beam.integration.nexmark.model.KnownSize; -import org.apache.beam.integration.nexmark.model.NameCityStateId; -import org.apache.beam.integration.nexmark.model.Person; import org.apache.beam.sdk.coders.ListCoder; import org.apache.beam.sdk.metrics.Counter; import org.apache.beam.sdk.metrics.Metrics; +import org.apache.beam.sdk.nexmark.NexmarkConfiguration; +import org.apache.beam.sdk.nexmark.NexmarkUtils; +import org.apache.beam.sdk.nexmark.model.Auction; +import org.apache.beam.sdk.nexmark.model.Event; +import org.apache.beam.sdk.nexmark.model.KnownSize; +import org.apache.beam.sdk.nexmark.model.NameCityStateId; +import org.apache.beam.sdk.nexmark.model.Person; import org.apache.beam.sdk.state.StateSpec; import org.apache.beam.sdk.state.StateSpecs; import org.apache.beam.sdk.state.TimeDomain; diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query3Model.java b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/Query3Model.java similarity index 90% rename from integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query3Model.java rename to sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/Query3Model.java index f415709dceff9..94f24cb545304 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query3Model.java +++ b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/Query3Model.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.integration.nexmark.queries; +package org.apache.beam.sdk.nexmark.queries; import com.google.common.collect.ArrayListMultimap; import com.google.common.collect.Multimap; @@ -26,12 +26,12 @@ import java.util.Iterator; import java.util.Map; -import org.apache.beam.integration.nexmark.NexmarkConfiguration; -import org.apache.beam.integration.nexmark.NexmarkUtils; -import org.apache.beam.integration.nexmark.model.Auction; -import org.apache.beam.integration.nexmark.model.Event; -import org.apache.beam.integration.nexmark.model.NameCityStateId; -import org.apache.beam.integration.nexmark.model.Person; +import org.apache.beam.sdk.nexmark.NexmarkConfiguration; +import org.apache.beam.sdk.nexmark.NexmarkUtils; +import org.apache.beam.sdk.nexmark.model.Auction; +import org.apache.beam.sdk.nexmark.model.Event; +import org.apache.beam.sdk.nexmark.model.NameCityStateId; +import org.apache.beam.sdk.nexmark.model.Person; import org.apache.beam.sdk.values.TimestampedValue; import org.joda.time.Instant; diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query4.java b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/Query4.java similarity index 88% rename from integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query4.java rename to sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/Query4.java index 9c0fe6d0609a7..3c1cf3b5c9d4a 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query4.java +++ b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/Query4.java @@ -15,17 +15,17 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.integration.nexmark.queries; +package org.apache.beam.sdk.nexmark.queries; -import org.apache.beam.integration.nexmark.Monitor; -import org.apache.beam.integration.nexmark.NexmarkConfiguration; -import org.apache.beam.integration.nexmark.NexmarkUtils; -import org.apache.beam.integration.nexmark.model.Auction; -import org.apache.beam.integration.nexmark.model.AuctionBid; -import org.apache.beam.integration.nexmark.model.Bid; -import org.apache.beam.integration.nexmark.model.CategoryPrice; -import org.apache.beam.integration.nexmark.model.Event; -import org.apache.beam.integration.nexmark.model.KnownSize; +import org.apache.beam.sdk.nexmark.Monitor; +import org.apache.beam.sdk.nexmark.NexmarkConfiguration; +import org.apache.beam.sdk.nexmark.NexmarkUtils; +import org.apache.beam.sdk.nexmark.model.Auction; +import org.apache.beam.sdk.nexmark.model.AuctionBid; +import org.apache.beam.sdk.nexmark.model.Bid; +import org.apache.beam.sdk.nexmark.model.CategoryPrice; +import org.apache.beam.sdk.nexmark.model.Event; +import org.apache.beam.sdk.nexmark.model.KnownSize; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.Mean; import org.apache.beam.sdk.transforms.ParDo; diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query4Model.java b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/Query4Model.java similarity index 93% rename from integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query4Model.java rename to sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/Query4Model.java index 269e47aa50220..84274a8f3f2d4 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query4Model.java +++ b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/Query4Model.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.integration.nexmark.queries; +package org.apache.beam.sdk.nexmark.queries; import java.io.Serializable; import java.util.ArrayList; @@ -24,13 +24,13 @@ import java.util.List; import java.util.Map; import java.util.TreeMap; -import org.apache.beam.integration.nexmark.NexmarkConfiguration; -import org.apache.beam.integration.nexmark.NexmarkUtils; -import org.apache.beam.integration.nexmark.model.Auction; -import org.apache.beam.integration.nexmark.model.AuctionBid; -import org.apache.beam.integration.nexmark.model.Bid; -import org.apache.beam.integration.nexmark.model.CategoryPrice; -import org.apache.beam.integration.nexmark.model.KnownSize; +import org.apache.beam.sdk.nexmark.NexmarkConfiguration; +import org.apache.beam.sdk.nexmark.NexmarkUtils; +import org.apache.beam.sdk.nexmark.model.Auction; +import org.apache.beam.sdk.nexmark.model.AuctionBid; +import org.apache.beam.sdk.nexmark.model.Bid; +import org.apache.beam.sdk.nexmark.model.CategoryPrice; +import org.apache.beam.sdk.nexmark.model.KnownSize; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.values.TimestampedValue; import org.joda.time.Duration; diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query5.java b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/Query5.java similarity index 92% rename from integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query5.java rename to sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/Query5.java index bdf3e5f15e588..d027cb31686ec 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query5.java +++ b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/Query5.java @@ -15,17 +15,17 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.integration.nexmark.queries; +package org.apache.beam.sdk.nexmark.queries; import java.util.ArrayList; import java.util.Collections; import java.util.List; -import org.apache.beam.integration.nexmark.NexmarkConfiguration; -import org.apache.beam.integration.nexmark.NexmarkUtils; -import org.apache.beam.integration.nexmark.model.AuctionCount; -import org.apache.beam.integration.nexmark.model.Bid; -import org.apache.beam.integration.nexmark.model.Event; -import org.apache.beam.integration.nexmark.model.KnownSize; +import org.apache.beam.sdk.nexmark.NexmarkConfiguration; +import org.apache.beam.sdk.nexmark.NexmarkUtils; +import org.apache.beam.sdk.nexmark.model.AuctionCount; +import org.apache.beam.sdk.nexmark.model.Bid; +import org.apache.beam.sdk.nexmark.model.Event; +import org.apache.beam.sdk.nexmark.model.KnownSize; import org.apache.beam.sdk.transforms.Combine; import org.apache.beam.sdk.transforms.Count; import org.apache.beam.sdk.transforms.DoFn; diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query5Model.java b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/Query5Model.java similarity index 94% rename from integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query5Model.java rename to sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/Query5Model.java index 24d9a00339593..7ed0709e88990 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query5Model.java +++ b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/Query5Model.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.integration.nexmark.queries; +package org.apache.beam.sdk.nexmark.queries; import java.io.Serializable; import java.util.ArrayList; @@ -24,11 +24,11 @@ import java.util.List; import java.util.Map; import java.util.TreeMap; -import org.apache.beam.integration.nexmark.NexmarkConfiguration; -import org.apache.beam.integration.nexmark.NexmarkUtils; -import org.apache.beam.integration.nexmark.model.AuctionCount; -import org.apache.beam.integration.nexmark.model.Bid; -import org.apache.beam.integration.nexmark.model.Event; +import org.apache.beam.sdk.nexmark.NexmarkConfiguration; +import org.apache.beam.sdk.nexmark.NexmarkUtils; +import org.apache.beam.sdk.nexmark.model.AuctionCount; +import org.apache.beam.sdk.nexmark.model.Bid; +import org.apache.beam.sdk.nexmark.model.Event; import org.apache.beam.sdk.values.TimestampedValue; import org.joda.time.Duration; import org.joda.time.Instant; diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query6.java b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/Query6.java similarity index 91% rename from integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query6.java rename to sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/Query6.java index ea39ede396d94..bc6b12c098836 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query6.java +++ b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/Query6.java @@ -15,20 +15,20 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.integration.nexmark.queries; +package org.apache.beam.sdk.nexmark.queries; import com.google.common.collect.Lists; import java.util.ArrayList; import java.util.Collections; import java.util.List; -import org.apache.beam.integration.nexmark.NexmarkConfiguration; -import org.apache.beam.integration.nexmark.NexmarkUtils; -import org.apache.beam.integration.nexmark.model.Auction; -import org.apache.beam.integration.nexmark.model.AuctionBid; -import org.apache.beam.integration.nexmark.model.Bid; -import org.apache.beam.integration.nexmark.model.Event; -import org.apache.beam.integration.nexmark.model.KnownSize; -import org.apache.beam.integration.nexmark.model.SellerPrice; +import org.apache.beam.sdk.nexmark.NexmarkConfiguration; +import org.apache.beam.sdk.nexmark.NexmarkUtils; +import org.apache.beam.sdk.nexmark.model.Auction; +import org.apache.beam.sdk.nexmark.model.AuctionBid; +import org.apache.beam.sdk.nexmark.model.Bid; +import org.apache.beam.sdk.nexmark.model.Event; +import org.apache.beam.sdk.nexmark.model.KnownSize; +import org.apache.beam.sdk.nexmark.model.SellerPrice; import org.apache.beam.sdk.transforms.Combine; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.ParDo; diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query6Model.java b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/Query6Model.java similarity index 90% rename from integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query6Model.java rename to sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/Query6Model.java index 9cb8b3d9fb4b5..b5152d8a0bb83 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query6Model.java +++ b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/Query6Model.java @@ -15,20 +15,20 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.integration.nexmark.queries; +package org.apache.beam.sdk.nexmark.queries; import java.io.Serializable; import java.util.Collection; import java.util.Iterator; import java.util.Map; import java.util.TreeMap; -import org.apache.beam.integration.nexmark.NexmarkConfiguration; -import org.apache.beam.integration.nexmark.NexmarkUtils; -import org.apache.beam.integration.nexmark.model.Auction; -import org.apache.beam.integration.nexmark.model.AuctionBid; -import org.apache.beam.integration.nexmark.model.Bid; -import org.apache.beam.integration.nexmark.model.KnownSize; -import org.apache.beam.integration.nexmark.model.SellerPrice; +import org.apache.beam.sdk.nexmark.NexmarkConfiguration; +import org.apache.beam.sdk.nexmark.NexmarkUtils; +import org.apache.beam.sdk.nexmark.model.Auction; +import org.apache.beam.sdk.nexmark.model.AuctionBid; +import org.apache.beam.sdk.nexmark.model.Bid; +import org.apache.beam.sdk.nexmark.model.KnownSize; +import org.apache.beam.sdk.nexmark.model.SellerPrice; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.values.TimestampedValue; import org.joda.time.Instant; diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query7.java b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/Query7.java similarity index 91% rename from integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query7.java rename to sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/Query7.java index 217d0d4ef1c9f..71b75c35fd7ab 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query7.java +++ b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/Query7.java @@ -15,13 +15,13 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.integration.nexmark.queries; +package org.apache.beam.sdk.nexmark.queries; -import org.apache.beam.integration.nexmark.NexmarkConfiguration; -import org.apache.beam.integration.nexmark.NexmarkUtils; -import org.apache.beam.integration.nexmark.model.Bid; -import org.apache.beam.integration.nexmark.model.Event; -import org.apache.beam.integration.nexmark.model.KnownSize; +import org.apache.beam.sdk.nexmark.NexmarkConfiguration; +import org.apache.beam.sdk.nexmark.NexmarkUtils; +import org.apache.beam.sdk.nexmark.model.Bid; +import org.apache.beam.sdk.nexmark.model.Event; +import org.apache.beam.sdk.nexmark.model.KnownSize; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.Max; import org.apache.beam.sdk.transforms.ParDo; diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query7Model.java b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/Query7Model.java similarity index 93% rename from integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query7Model.java rename to sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/Query7Model.java index 0ada5e8c3e90f..401174697e9c5 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query7Model.java +++ b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/Query7Model.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.integration.nexmark.queries; +package org.apache.beam.sdk.nexmark.queries; import java.io.Serializable; import java.util.ArrayList; @@ -23,10 +23,10 @@ import java.util.Iterator; import java.util.List; -import org.apache.beam.integration.nexmark.NexmarkConfiguration; -import org.apache.beam.integration.nexmark.NexmarkUtils; -import org.apache.beam.integration.nexmark.model.Bid; -import org.apache.beam.integration.nexmark.model.Event; +import org.apache.beam.sdk.nexmark.NexmarkConfiguration; +import org.apache.beam.sdk.nexmark.NexmarkUtils; +import org.apache.beam.sdk.nexmark.model.Bid; +import org.apache.beam.sdk.nexmark.model.Event; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.values.TimestampedValue; import org.joda.time.Duration; diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query8.java b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/Query8.java similarity index 88% rename from integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query8.java rename to sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/Query8.java index 603841be6d7df..fa3dd864d2ec1 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query8.java +++ b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/Query8.java @@ -15,15 +15,15 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.integration.nexmark.queries; +package org.apache.beam.sdk.nexmark.queries; -import org.apache.beam.integration.nexmark.NexmarkConfiguration; -import org.apache.beam.integration.nexmark.NexmarkUtils; -import org.apache.beam.integration.nexmark.model.Auction; -import org.apache.beam.integration.nexmark.model.Event; -import org.apache.beam.integration.nexmark.model.IdNameReserve; -import org.apache.beam.integration.nexmark.model.KnownSize; -import org.apache.beam.integration.nexmark.model.Person; +import org.apache.beam.sdk.nexmark.NexmarkConfiguration; +import org.apache.beam.sdk.nexmark.NexmarkUtils; +import org.apache.beam.sdk.nexmark.model.Auction; +import org.apache.beam.sdk.nexmark.model.Event; +import org.apache.beam.sdk.nexmark.model.IdNameReserve; +import org.apache.beam.sdk.nexmark.model.KnownSize; +import org.apache.beam.sdk.nexmark.model.Person; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.ParDo; import org.apache.beam.sdk.transforms.join.CoGbkResult; diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query8Model.java b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/Query8Model.java similarity index 92% rename from integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query8Model.java rename to sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/Query8Model.java index 8c76bc662ced4..351cef7350e72 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query8Model.java +++ b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/Query8Model.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.integration.nexmark.queries; +package org.apache.beam.sdk.nexmark.queries; import com.google.common.collect.ArrayListMultimap; import com.google.common.collect.Multimap; @@ -24,12 +24,12 @@ import java.util.HashMap; import java.util.Iterator; import java.util.Map; -import org.apache.beam.integration.nexmark.NexmarkConfiguration; -import org.apache.beam.integration.nexmark.NexmarkUtils; -import org.apache.beam.integration.nexmark.model.Auction; -import org.apache.beam.integration.nexmark.model.Event; -import org.apache.beam.integration.nexmark.model.IdNameReserve; -import org.apache.beam.integration.nexmark.model.Person; +import org.apache.beam.sdk.nexmark.NexmarkConfiguration; +import org.apache.beam.sdk.nexmark.NexmarkUtils; +import org.apache.beam.sdk.nexmark.model.Auction; +import org.apache.beam.sdk.nexmark.model.Event; +import org.apache.beam.sdk.nexmark.model.IdNameReserve; +import org.apache.beam.sdk.nexmark.model.Person; import org.apache.beam.sdk.values.TimestampedValue; import org.joda.time.Duration; import org.joda.time.Instant; diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query9.java b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/Query9.java similarity index 80% rename from integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query9.java rename to sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/Query9.java index 6dd189d11fac5..5f11e4e6a1069 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query9.java +++ b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/Query9.java @@ -15,13 +15,13 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.integration.nexmark.queries; +package org.apache.beam.sdk.nexmark.queries; -import org.apache.beam.integration.nexmark.NexmarkConfiguration; -import org.apache.beam.integration.nexmark.NexmarkUtils; -import org.apache.beam.integration.nexmark.model.AuctionBid; -import org.apache.beam.integration.nexmark.model.Event; -import org.apache.beam.integration.nexmark.model.KnownSize; +import org.apache.beam.sdk.nexmark.NexmarkConfiguration; +import org.apache.beam.sdk.nexmark.NexmarkUtils; +import org.apache.beam.sdk.nexmark.model.AuctionBid; +import org.apache.beam.sdk.nexmark.model.Event; +import org.apache.beam.sdk.nexmark.model.KnownSize; import org.apache.beam.sdk.values.PCollection; /** diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query9Model.java b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/Query9Model.java similarity index 92% rename from integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query9Model.java rename to sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/Query9Model.java index d117e2dfabb6e..48d792ed5446b 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/Query9Model.java +++ b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/Query9Model.java @@ -15,13 +15,13 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.integration.nexmark.queries; +package org.apache.beam.sdk.nexmark.queries; import java.io.Serializable; import java.util.Collection; import java.util.Iterator; -import org.apache.beam.integration.nexmark.NexmarkConfiguration; +import org.apache.beam.sdk.nexmark.NexmarkConfiguration; import org.apache.beam.sdk.values.TimestampedValue; /** diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/WinningBids.java b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/WinningBids.java similarity index 97% rename from integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/WinningBids.java rename to sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/WinningBids.java index d4ca1773047fe..816a81f0d0a90 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/WinningBids.java +++ b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/WinningBids.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.integration.nexmark.queries; +package org.apache.beam.sdk.nexmark.queries; import static com.google.common.base.Preconditions.checkState; @@ -30,13 +30,6 @@ import java.util.Map; import java.util.Objects; import java.util.TreeMap; -import org.apache.beam.integration.nexmark.NexmarkConfiguration; -import org.apache.beam.integration.nexmark.NexmarkUtils; -import org.apache.beam.integration.nexmark.model.Auction; -import org.apache.beam.integration.nexmark.model.AuctionBid; -import org.apache.beam.integration.nexmark.model.Bid; -import org.apache.beam.integration.nexmark.model.Event; -import org.apache.beam.integration.nexmark.sources.GeneratorConfig; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.CoderException; import org.apache.beam.sdk.coders.CustomCoder; @@ -44,6 +37,13 @@ import org.apache.beam.sdk.coders.VarLongCoder; import org.apache.beam.sdk.metrics.Counter; import org.apache.beam.sdk.metrics.Metrics; +import org.apache.beam.sdk.nexmark.NexmarkConfiguration; +import org.apache.beam.sdk.nexmark.NexmarkUtils; +import org.apache.beam.sdk.nexmark.model.Auction; +import org.apache.beam.sdk.nexmark.model.AuctionBid; +import org.apache.beam.sdk.nexmark.model.Bid; +import org.apache.beam.sdk.nexmark.model.Event; +import org.apache.beam.sdk.nexmark.sources.GeneratorConfig; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.ParDo; diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/WinningBidsSimulator.java b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/WinningBidsSimulator.java similarity index 94% rename from integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/WinningBidsSimulator.java rename to sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/WinningBidsSimulator.java index 9624a9d69c891..69b64c05a3458 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/WinningBidsSimulator.java +++ b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/WinningBidsSimulator.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.integration.nexmark.queries; +package org.apache.beam.sdk.nexmark.queries; import java.util.ArrayList; import java.util.Iterator; @@ -26,12 +26,12 @@ import java.util.TreeSet; import javax.annotation.Nullable; -import org.apache.beam.integration.nexmark.NexmarkConfiguration; -import org.apache.beam.integration.nexmark.NexmarkUtils; -import org.apache.beam.integration.nexmark.model.Auction; -import org.apache.beam.integration.nexmark.model.AuctionBid; -import org.apache.beam.integration.nexmark.model.Bid; -import org.apache.beam.integration.nexmark.model.Event; +import org.apache.beam.sdk.nexmark.NexmarkConfiguration; +import org.apache.beam.sdk.nexmark.NexmarkUtils; +import org.apache.beam.sdk.nexmark.model.Auction; +import org.apache.beam.sdk.nexmark.model.AuctionBid; +import org.apache.beam.sdk.nexmark.model.Bid; +import org.apache.beam.sdk.nexmark.model.Event; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.values.TimestampedValue; import org.joda.time.Instant; diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/package-info.java b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/package-info.java similarity index 94% rename from integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/package-info.java rename to sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/package-info.java index 7a56733ea85b6..2ca5a1cf53d0c 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/queries/package-info.java +++ b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/queries/package-info.java @@ -19,4 +19,4 @@ /** * Nexmark Queries. */ -package org.apache.beam.integration.nexmark.queries; +package org.apache.beam.sdk.nexmark.queries; diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/sources/BoundedEventSource.java b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/sources/BoundedEventSource.java similarity index 97% rename from integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/sources/BoundedEventSource.java rename to sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/sources/BoundedEventSource.java index 43d6690848a9f..60124bb271728 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/sources/BoundedEventSource.java +++ b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/sources/BoundedEventSource.java @@ -15,17 +15,17 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.integration.nexmark.sources; +package org.apache.beam.sdk.nexmark.sources; import java.io.IOException; import java.util.ArrayList; import java.util.List; import java.util.NoSuchElementException; import javax.annotation.Nullable; -import org.apache.beam.integration.nexmark.NexmarkUtils; -import org.apache.beam.integration.nexmark.model.Event; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.io.BoundedSource; +import org.apache.beam.sdk.nexmark.NexmarkUtils; +import org.apache.beam.sdk.nexmark.model.Event; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.values.TimestampedValue; import org.joda.time.Instant; diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/sources/Generator.java b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/sources/Generator.java similarity index 98% rename from integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/sources/Generator.java rename to sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/sources/Generator.java index f6deceb32f225..c368d72804b6d 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/sources/Generator.java +++ b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/sources/Generator.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.integration.nexmark.sources; +package org.apache.beam.sdk.nexmark.sources; import static com.google.common.base.Preconditions.checkNotNull; @@ -28,15 +28,15 @@ import java.util.List; import java.util.Objects; import java.util.Random; -import org.apache.beam.integration.nexmark.model.Auction; -import org.apache.beam.integration.nexmark.model.Bid; -import org.apache.beam.integration.nexmark.model.Event; -import org.apache.beam.integration.nexmark.model.Person; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.CoderException; import org.apache.beam.sdk.coders.CustomCoder; import org.apache.beam.sdk.coders.VarLongCoder; import org.apache.beam.sdk.io.UnboundedSource; +import org.apache.beam.sdk.nexmark.model.Auction; +import org.apache.beam.sdk.nexmark.model.Bid; +import org.apache.beam.sdk.nexmark.model.Event; +import org.apache.beam.sdk.nexmark.model.Person; import org.apache.beam.sdk.values.TimestampedValue; import org.joda.time.Instant; diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/sources/GeneratorConfig.java b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/sources/GeneratorConfig.java similarity index 97% rename from integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/sources/GeneratorConfig.java rename to sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/sources/GeneratorConfig.java index 95c276b0db0f9..42183c6a528f8 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/sources/GeneratorConfig.java +++ b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/sources/GeneratorConfig.java @@ -15,16 +15,13 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.integration.nexmark.sources; - -import static com.google.common.base.Preconditions.checkNotNull; +package org.apache.beam.sdk.nexmark.sources; import java.io.Serializable; import java.util.ArrayList; import java.util.List; - -import org.apache.beam.integration.nexmark.NexmarkConfiguration; -import org.apache.beam.integration.nexmark.model.Event; +import org.apache.beam.sdk.nexmark.NexmarkConfiguration; +import org.apache.beam.sdk.nexmark.model.Event; import org.apache.beam.sdk.values.KV; /** diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/sources/UnboundedEventSource.java b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/sources/UnboundedEventSource.java similarity index 98% rename from integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/sources/UnboundedEventSource.java rename to sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/sources/UnboundedEventSource.java index 09d945d439fd6..8f5575c08851d 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/sources/UnboundedEventSource.java +++ b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/sources/UnboundedEventSource.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.integration.nexmark.sources; +package org.apache.beam.sdk.nexmark.sources; import java.util.ArrayList; import java.util.List; @@ -24,11 +24,10 @@ import java.util.Queue; import java.util.concurrent.ThreadLocalRandom; import javax.annotation.Nullable; - -import org.apache.beam.integration.nexmark.NexmarkUtils; -import org.apache.beam.integration.nexmark.model.Event; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.io.UnboundedSource; +import org.apache.beam.sdk.nexmark.NexmarkUtils; +import org.apache.beam.sdk.nexmark.model.Event; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.values.TimestampedValue; diff --git a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/sources/package-info.java b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/sources/package-info.java similarity index 94% rename from integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/sources/package-info.java rename to sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/sources/package-info.java index ceaec9d46d69c..266af10f66e06 100644 --- a/integration/java/nexmark/src/main/java/org/apache/beam/integration/nexmark/sources/package-info.java +++ b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/sources/package-info.java @@ -19,4 +19,4 @@ /** * Nexmark Synthetic Sources. */ -package org.apache.beam.integration.nexmark.sources; +package org.apache.beam.sdk.nexmark.sources; diff --git a/integration/java/nexmark/src/main/resources/log4j.properties b/sdks/java/nexmark/src/main/resources/log4j.properties similarity index 100% rename from integration/java/nexmark/src/main/resources/log4j.properties rename to sdks/java/nexmark/src/main/resources/log4j.properties diff --git a/integration/java/nexmark/src/test/java/org/apache/beam/integration/nexmark/queries/QueryTest.java b/sdks/java/nexmark/src/test/java/org/apache/beam/sdk/nexmark/queries/QueryTest.java similarity index 96% rename from integration/java/nexmark/src/test/java/org/apache/beam/integration/nexmark/queries/QueryTest.java rename to sdks/java/nexmark/src/test/java/org/apache/beam/sdk/nexmark/queries/QueryTest.java index 64a8e4fee4f1a..d8ac05714ae07 100644 --- a/integration/java/nexmark/src/test/java/org/apache/beam/integration/nexmark/queries/QueryTest.java +++ b/sdks/java/nexmark/src/test/java/org/apache/beam/sdk/nexmark/queries/QueryTest.java @@ -15,12 +15,12 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.integration.nexmark.queries; +package org.apache.beam.sdk.nexmark.queries; -import org.apache.beam.integration.nexmark.NexmarkConfiguration; -import org.apache.beam.integration.nexmark.NexmarkUtils; -import org.apache.beam.integration.nexmark.model.KnownSize; import org.apache.beam.sdk.PipelineResult; +import org.apache.beam.sdk.nexmark.NexmarkConfiguration; +import org.apache.beam.sdk.nexmark.NexmarkUtils; +import org.apache.beam.sdk.nexmark.model.KnownSize; import org.apache.beam.sdk.testing.NeedsRunner; import org.apache.beam.sdk.testing.PAssert; import org.apache.beam.sdk.testing.TestPipeline; diff --git a/integration/java/nexmark/src/test/java/org/apache/beam/integration/nexmark/sources/BoundedEventSourceTest.java b/sdks/java/nexmark/src/test/java/org/apache/beam/sdk/nexmark/sources/BoundedEventSourceTest.java similarity index 93% rename from integration/java/nexmark/src/test/java/org/apache/beam/integration/nexmark/sources/BoundedEventSourceTest.java rename to sdks/java/nexmark/src/test/java/org/apache/beam/sdk/nexmark/sources/BoundedEventSourceTest.java index d95461a09ed6e..3590d64a9a181 100644 --- a/integration/java/nexmark/src/test/java/org/apache/beam/integration/nexmark/sources/BoundedEventSourceTest.java +++ b/sdks/java/nexmark/src/test/java/org/apache/beam/sdk/nexmark/sources/BoundedEventSourceTest.java @@ -15,13 +15,12 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.integration.nexmark.sources; +package org.apache.beam.sdk.nexmark.sources; -import org.apache.beam.integration.nexmark.NexmarkConfiguration; -import org.apache.beam.integration.nexmark.NexmarkOptions; +import org.apache.beam.sdk.nexmark.NexmarkConfiguration; +import org.apache.beam.sdk.nexmark.NexmarkOptions; import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.testing.SourceTestUtils; - import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; diff --git a/integration/java/nexmark/src/test/java/org/apache/beam/integration/nexmark/sources/GeneratorTest.java b/sdks/java/nexmark/src/test/java/org/apache/beam/sdk/nexmark/sources/GeneratorTest.java similarity index 96% rename from integration/java/nexmark/src/test/java/org/apache/beam/integration/nexmark/sources/GeneratorTest.java rename to sdks/java/nexmark/src/test/java/org/apache/beam/sdk/nexmark/sources/GeneratorTest.java index b0dff2f8f2300..9553d22de3c98 100644 --- a/integration/java/nexmark/src/test/java/org/apache/beam/integration/nexmark/sources/GeneratorTest.java +++ b/sdks/java/nexmark/src/test/java/org/apache/beam/sdk/nexmark/sources/GeneratorTest.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.integration.nexmark.sources; +package org.apache.beam.sdk.nexmark.sources; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; @@ -23,8 +23,7 @@ import java.util.ArrayList; import java.util.Iterator; import java.util.List; - -import org.apache.beam.integration.nexmark.NexmarkConfiguration; +import org.apache.beam.sdk.nexmark.NexmarkConfiguration; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; diff --git a/integration/java/nexmark/src/test/java/org/apache/beam/integration/nexmark/sources/UnboundedEventSourceTest.java b/sdks/java/nexmark/src/test/java/org/apache/beam/sdk/nexmark/sources/UnboundedEventSourceTest.java similarity index 95% rename from integration/java/nexmark/src/test/java/org/apache/beam/integration/nexmark/sources/UnboundedEventSourceTest.java rename to sdks/java/nexmark/src/test/java/org/apache/beam/sdk/nexmark/sources/UnboundedEventSourceTest.java index 1ecc33e536936..3853ede4ae25e 100644 --- a/integration/java/nexmark/src/test/java/org/apache/beam/integration/nexmark/sources/UnboundedEventSourceTest.java +++ b/sdks/java/nexmark/src/test/java/org/apache/beam/sdk/nexmark/sources/UnboundedEventSourceTest.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.integration.nexmark.sources; +package org.apache.beam.sdk.nexmark.sources; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; @@ -25,14 +25,12 @@ import java.util.HashSet; import java.util.Random; import java.util.Set; - -import org.apache.beam.integration.nexmark.NexmarkConfiguration; -import org.apache.beam.integration.nexmark.model.Event; import org.apache.beam.sdk.io.UnboundedSource.CheckpointMark; import org.apache.beam.sdk.io.UnboundedSource.UnboundedReader; +import org.apache.beam.sdk.nexmark.NexmarkConfiguration; +import org.apache.beam.sdk.nexmark.model.Event; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.testing.TestPipeline; - import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; diff --git a/sdks/java/pom.xml b/sdks/java/pom.xml index 3144193b9839e..f0cf8d94d6e38 100644 --- a/sdks/java/pom.xml +++ b/sdks/java/pom.xml @@ -43,6 +43,7 @@ + nexmark From 0acd72016b046640dd753d60a569403488f595a5 Mon Sep 17 00:00:00 2001 From: Etienne Chauchot Date: Fri, 11 Aug 2017 16:41:00 +0200 Subject: [PATCH 316/346] Update to Beam 2.2.0-SNAPSHOT and final touches before merge into master Clean pom, exclude nexmark packages from aggregated javadoc, put spark logs in WARN Update execution matrix in README: Flink termination of streaming pipelines is now ok as far as Nexmark is concerned Remove how to run Nexmark on Flink on Google Cloud Platform from README Update command lines in README after moving nexmark from integration to sdks module --- .../java/nexmark/BEAM_ON_FLINK_ON_GCP.md | 282 ------------------ sdks/java/javadoc/ant.xml | 1 + sdks/java/nexmark/README.md | 56 ++-- sdks/java/nexmark/pom.xml | 35 +-- .../apache/beam/sdk/nexmark/package-info.java | 2 +- .../src/main/resources/log4j.properties | 4 +- 6 files changed, 34 insertions(+), 346 deletions(-) delete mode 100644 integration/java/nexmark/BEAM_ON_FLINK_ON_GCP.md diff --git a/integration/java/nexmark/BEAM_ON_FLINK_ON_GCP.md b/integration/java/nexmark/BEAM_ON_FLINK_ON_GCP.md deleted file mode 100644 index 6a7fd3432c5db..0000000000000 --- a/integration/java/nexmark/BEAM_ON_FLINK_ON_GCP.md +++ /dev/null @@ -1,282 +0,0 @@ - - -# Running NexMark on Beam on Flink on Google Compute Platform - -Here's how to create a cluster of VMs on Google Compute Platform, deploy -Flink to them, and invoke a NexMark pipeline using the Beam-on-Flink -runner. - -These instructions are somewhat baroque and I hope they can be -simplified over time. - -## Prerequisites - -You'll need: - -* the Google Cloud SDK -* a clone of the Beam repository -* a Flink binary distribution -* a project on Google Compute Platform. - -## Establish the shell environment - -``` -# Beam root -BEAM= -# Flink root -FLINK_VER=flink-1.0.3 -FLINK= -# Google Cloud project -PROJECT= -# Google Cloud zone -ZONE= -# Cloud commands -GCLOUD= -GSUTIL= -``` - -## Establish VM names for Flink master and workers - -``` -MASTER=flink-m -NUM_WORKERS=5 -WORKERS="" -for (( i = 0; i < $NUM_WORKERS; i++ )); do - WORKERS="$WORKERS flink-w-$i" -done -ALL="$MASTER $WORKERS" -``` - -## Build Beam - -``` -( cd $BEAM && mvn clean install ) -``` - -## Bring up the cluster - -Establish project defaults and authenticate: -``` -$GCLOUD init -$GCLOUD auth login -``` - -Build Google Cloud Dataproc cluster: -``` -$GCLOUD beta dataproc clusters create \ - --project=$PROJECT \ - --zone=$ZONE \ - --bucket=nexmark \ - --scopes=cloud-platform \ - --num-workers=$NUM_WORKERS \ - --image-version=preview \ - flink -``` - -Force google_compute_engine ssh keys to be generated locally: -``` -$GCLOUD compute ssh \ - --project=$PROJECT \ - --zone=$ZONE \ - $MASTER \ - --command 'exit' -``` - -Open ports on the VMs: -``` -$GCLOUD compute firewall-rules create allow-monitoring --allow tcp:8080-8081 -$GCLOUD compute firewall-rules create allow-debug --allow tcp:5555 -``` - -Establish keys on master and workers -**CAUTION:** This will leave your private key on your master VM. -Better would be to create a key just for inter-worker ssh. -``` -for m in $ALL; do - echo "*** $m ***" - $GCLOUD beta compute scp \ - --project=$PROJECT \ - --zone=$ZONE \ - ~/.ssh/google_compute_engine.pub $m:~/.ssh/ -done -$GCLOUD beta compute scp \ - --project=$PROJECT \ - --zone=$ZONE \ - ~/.ssh/google_compute_engine $MASTER:~/.ssh/ -``` - -Collect IP addresses for workers: -``` -MASTER_EXT_IP=$($GCLOUD compute instances describe \ - --project=$PROJECT \ - --zone=$ZONE \ - $MASTER | grep natIP: | sed 's/[ ]*natIP:[ ]*//') -MASTER_IP=$($GCLOUD compute instances describe \ - --project=$PROJECT \ - --zone=$ZONE \ - $MASTER | grep networkIP: | sed 's/[ ]*networkIP:[ ]*//') -WORKER_IPS="" -for m in $WORKERS; do - echo "*** $m ***" - WORKER_IP=$($GCLOUD compute instances describe \ - --project=$PROJECT \ - --zone=$ZONE \ - $m | grep networkIP: | sed 's/[ ]*networkIP:[ ]*//') - WORKER_IPS="$WORKER_IPS $WORKER_IP" -done -``` - -Configure Flink: -``` -cat $FLINK/conf/flink-conf.yaml \ - | sed "s|.*\(jobmanager.rpc.address\):.*|\1: $MASTER_IP|g" \ - | sed "s|.*\(jobmanager.heap.mb\):.*|\1: 4096|g" \ - | sed "s|.*\(taskmanager.heap.mb\):.*|\1: 8192|g" \ - | sed "s|.*\(parallelism.default\):.*|\1: $(($NUM_WORKERS * 4))|g" \ - | sed "s|.*\(fs.hdfs.hadoopconf\):.*|\1: /etc/hadoop/conf|g" \ - | sed "s|.*\(taskmanager.numberOfTaskSlots\):.*|\1: 4|g" \ - | sed "s|.*\(jobmanager.web.submit.enable\):.*|\1: false|g" \ - | sed "s|.*\(env.ssh.opts\):.*||g" \ - > ~/flink-conf.yaml -cat $FLINK/conf/log4j.properties \ - | sed "s|.*\(log4j.rootLogger\)=.*|\1=ERROR, file|g" \ - > ~/log4j.properties -echo "env.ssh.opts: -i /home/$USER/.ssh/google_compute_engine -o StrictHostKeyChecking=no" >> ~/flink-conf.yaml -echo "$MASTER_IP:8081" > ~/masters -echo -n > ~/slaves -for ip in $WORKER_IPS; do - echo $ip >> ~/slaves -done -cp -f \ - ~/flink-conf.yaml \ - ~/masters ~/slaves \ - ~/log4j.properties \ - $FLINK/conf/ -``` - -Package configured Flink for distribution to workers: -``` -( cd ~/ && tar -cvzf ~/flink.tgz $FLINK/* ) -``` - -Distribute: -``` -$GSUTIL cp ~/flink.tgz gs://nexmark -for m in $ALL; do - echo "*** $m ***" - $GCLOUD compute ssh \ - --project=$PROJECT \ - --zone=$ZONE \ - $m \ - --command 'gsutil cp gs://nexmark/flink.tgz ~/ && tar -xvzf ~/flink.tgz' -done -``` - -Start the Flink cluster: -``` -$GCLOUD compute ssh \ - --project=$PROJECT \ - --zone=$ZONE \ - $MASTER \ - --command "~/$FLINK_VER/bin/start-cluster.sh" -``` - -Bring up the Flink monitoring UI: -``` -/usr/bin/google-chrome $MASTER_EXT_IP:8081 & -``` - -## Run NexMark - -Distribute the Beam + NexMark jar to all workers: -``` -$GSUTIL cp $BEAM/integration/java/target/java-integration-all-bundled-0.2.0-incubating-SNAPSHOT.jar gs://nexmark -for m in $ALL; do - echo "*** $m ***" - $GCLOUD compute ssh \ - --project=$PROJECT \ - --zone=$ZONE \ - $m \ - --command "gsutil cp gs://nexmark/java-integration-all-bundled-0.2.0-incubating-SNAPSHOT.jar ~/$FLINK_VER/lib/" -done -``` - -Create a Pubsub topic and subscription for testing: -``` -$GCLOUD alpha pubsub \ - --project=$PROJECT \ - topics create flink_test - -$GCLOUD alpha pubsub \ - --project=$PROJECT \ - subscriptions create flink_test \ - --topic flink_test \ - --ack-deadline=60 \ - --topic-project=$PROJECT -``` - -Launch! -**NOTE:** As of flink-1.0.3 this will throw a `NullPointerException` -in `org.apache.beam.sdk.io.PubsubUnboundedSink$WriterFn.startBundle`. -See Jira issue [BEAM-196](https://issues.apache.org/jira/browse/BEAM-196). - -``` -$GCLOUD compute ssh \ - --project=$PROJECT \ - --zone=$ZONE \ - $MASTER \ - --command "~/$FLINK_VER/bin/flink run \ - -c org.apache.beam.integration.nexmark.drivers.NexmarkFlinkDriver \ - ~/$FLINK_VER/lib/java-integration-all-bundled-0.2.0-incubating-SNAPSHOT.jar \ - --project=$PROJECT \ - --streaming=true \ - --query=0 \ - --sourceType=PUBSUB \ - --pubSubMode=COMBINED \ - --pubsubTopic=flink_test \ - --resourceNameMode=VERBATIM \ - --manageResources=false \ - --monitorJobs=false \ - --numEventGenerators=5 \ - --firstEventRate=1000 \ - --nextEventRate=1000 \ - --isRateLimited=true \ - --numEvents=0 \ - --useWallclockEventTime=true \ - --usePubsubPublishTime=true" -``` - -## Teardown the cluster - -Stop the Flink cluster: -``` -$GCLOUD compute ssh \ - --project=$PROJECT \ - --zone=$ZONE \ - $MASTER \ - --command "~/$FLINK_VER/bin/stop-cluster.sh" -``` - -Teardown the Dataproc cluster: -``` -$GCLOUD beta dataproc clusters delete \ - --project=$PROJECT \ - flink -``` diff --git a/sdks/java/javadoc/ant.xml b/sdks/java/javadoc/ant.xml index 9a1880acaa17f..152b9ddb6bfa2 100644 --- a/sdks/java/javadoc/ant.xml +++ b/sdks/java/javadoc/ant.xml @@ -30,6 +30,7 @@ + diff --git a/sdks/java/nexmark/README.md b/sdks/java/nexmark/README.md index a9acd63c51708..f252943987dbf 100644 --- a/sdks/java/nexmark/README.md +++ b/sdks/java/nexmark/README.md @@ -17,7 +17,7 @@ under the License. --> -# NEXMark integration suite +# NEXMark test suite This is a suite of pipelines inspired by the 'continuous data stream' queries in [http://datalab.cs.pdx.edu/niagaraST/NEXMark/] @@ -139,7 +139,7 @@ Available suites are: --runner=DataflowRunner \ --tempLocation=gs://talend-imejia/nexmark/temp/ \ --stagingLocation=gs://talend-imejia/nexmark/temp/staging/ \ - --filesToStage=target/beam-integration-java-0.7.0-SNAPSHOT.jar + --filesToStage=target/beam-sdks-java-nexmark-2.1.0-SNAPSHOT.jar ## Direct specific configuration @@ -183,19 +183,19 @@ Open issues are tracked [here](https://github.com../../../../../issues): | Query | Direct | Spark | Flink | Apex | | ----: | ------ | ------------------------------------------------------------ | ---------------------------------------------------------- | ------------------------------------------------------------ | -| 0 | ok | ok | [BEAM-593](https://issues.apache.org/jira/browse/BEAM-593) | ok | -| 1 | ok | ok | [BEAM-593](https://issues.apache.org/jira/browse/BEAM-593) | ok | -| 2 | ok | ok | [BEAM-593](https://issues.apache.org/jira/browse/BEAM-593) | ok | -| 3 | ok | [BEAM-1035](https://issues.apache.org/jira/browse/BEAM-1035) | [BEAM-593](https://issues.apache.org/jira/browse/BEAM-593) | [BEAM-1037](https://issues.apache.org/jira/browse/BEAM-1037) | -| 4 | ok | ok | [BEAM-593](https://issues.apache.org/jira/browse/BEAM-593) | ok | -| 5 | ok | ok | [BEAM-593](https://issues.apache.org/jira/browse/BEAM-593) | ok | -| 6 | ok | ok | [BEAM-593](https://issues.apache.org/jira/browse/BEAM-593) | ok | -| 7 | ok | [BEAM-2112](https://issues.apache.org/jira/browse/BEAM-2112) | [BEAM-593](https://issues.apache.org/jira/browse/BEAM-593) | ok | -| 8 | ok | ok | [BEAM-593](https://issues.apache.org/jira/browse/BEAM-593) | ok | -| 9 | ok | ok | [BEAM-593](https://issues.apache.org/jira/browse/BEAM-593) | ok | -| 10 | ok | ok | [BEAM-593](https://issues.apache.org/jira/browse/BEAM-593) | ok | -| 11 | ok | ok | [BEAM-593](https://issues.apache.org/jira/browse/BEAM-593) | ok | -| 12 | ok | ok | [BEAM-593](https://issues.apache.org/jira/browse/BEAM-593) | ok | +| 0 | ok | ok | ok | ok | +| 1 | ok | ok | ok | ok | +| 2 | ok | ok | ok | ok | +| 3 | ok | [BEAM-1035](https://issues.apache.org/jira/browse/BEAM-1035) | ok | [BEAM-1037](https://issues.apache.org/jira/browse/BEAM-1037) | +| 4 | ok | ok | ok | ok | +| 5 | ok | ok | ok | ok | +| 6 | ok | ok | ok | ok | +| 7 | ok | [BEAM-2112](https://issues.apache.org/jira/browse/BEAM-2112) | ok | ok | +| 8 | ok | ok | ok | ok | +| 9 | ok | ok | ok | ok | +| 10 | ok | ok | ok | ok | +| 11 | ok | ok | ok | ok | +| 12 | ok | ok | ok | ok | ## Batch / Synthetic / Cluster @@ -219,44 +219,44 @@ TODO Batch Mode - mvn exec:java -Dexec.mainClass=org.apache.beam.integration.nexmark.Main -Pdirect-runner -Dexec.args="--runner=DirectRunner --suite=SMOKE --streaming=false --manageResources=false --monitorJobs=true --enforceEncodability=true --enforceImmutability=true" + mvn exec:java -Dexec.mainClass=org.apache.beam.sdk.nexmark.Main -Pdirect-runner -Dexec.args="--runner=DirectRunner --suite=SMOKE --streaming=false --manageResources=false --monitorJobs=true --enforceEncodability=true --enforceImmutability=true" Streaming Mode - mvn exec:java -Dexec.mainClass=org.apache.beam.integration.nexmark.Main -Pdirect-runner -Dexec.args="--runner=DirectRunner --suite=SMOKE --streaming=true --manageResources=false --monitorJobs=true --enforceEncodability=true --enforceImmutability=true" + mvn exec:java -Dexec.mainClass=org.apache.beam.sdk.nexmark.Main -Pdirect-runner -Dexec.args="--runner=DirectRunner --suite=SMOKE --streaming=true --manageResources=false --monitorJobs=true --enforceEncodability=true --enforceImmutability=true" ## Running SMOKE suite on the SparkRunner (local) Batch Mode - mvn exec:java -Dexec.mainClass=org.apache.beam.integration.nexmark.Main -Pspark-runner "-Dexec.args=--runner=SparkRunner --suite=SMOKE --streamTimeout=60 --streaming=false --manageResources=false --monitorJobs=true" + mvn exec:java -Dexec.mainClass=org.apache.beam.sdk.nexmark.Main -Pspark-runner "-Dexec.args=--runner=SparkRunner --suite=SMOKE --streamTimeout=60 --streaming=false --manageResources=false --monitorJobs=true" Streaming Mode - mvn exec:java -Dexec.mainClass=org.apache.beam.integration.nexmark.Main -Pspark-runner "-Dexec.args=--runner=SparkRunner --suite=SMOKE --streamTimeout=60 --streaming=true --manageResources=false --monitorJobs=true" + mvn exec:java -Dexec.mainClass=org.apache.beam.sdk.nexmark.Main -Pspark-runner "-Dexec.args=--runner=SparkRunner --suite=SMOKE --streamTimeout=60 --streaming=true --manageResources=false --monitorJobs=true" ## Running SMOKE suite on the FlinkRunner (local) Batch Mode - mvn exec:java -Dexec.mainClass=org.apache.beam.integration.nexmark.Main -Pflink-runner "-Dexec.args=--runner=FlinkRunner --suite=SMOKE --streamTimeout=60 --streaming=false --manageResources=false --monitorJobs=true --flinkMaster=local" + mvn exec:java -Dexec.mainClass=org.apache.beam.sdk.nexmark.Main -Pflink-runner "-Dexec.args=--runner=FlinkRunner --suite=SMOKE --streamTimeout=60 --streaming=false --manageResources=false --monitorJobs=true --flinkMaster=local" Streaming Mode - mvn exec:java -Dexec.mainClass=org.apache.beam.integration.nexmark.Main -Pflink-runner "-Dexec.args=--runner=FlinkRunner --suite=SMOKE --streamTimeout=60 --streaming=true --manageResources=false --monitorJobs=true --flinkMaster=local" + mvn exec:java -Dexec.mainClass=org.apache.beam.sdk.nexmark.Main -Pflink-runner "-Dexec.args=--runner=FlinkRunner --suite=SMOKE --streamTimeout=60 --streaming=true --manageResources=false --monitorJobs=true --flinkMaster=local" ## Running SMOKE suite on the ApexRunner (local) Batch Mode - mvn exec:java -Dexec.mainClass=org.apache.beam.integration.nexmark.Main -Papex-runner "-Dexec.args=--runner=ApexRunner --suite=SMOKE --streamTimeout=60 --streaming=false --manageResources=false --monitorJobs=false" + mvn exec:java -Dexec.mainClass=org.apache.beam.sdk.nexmark.Main -Papex-runner "-Dexec.args=--runner=ApexRunner --suite=SMOKE --streamTimeout=60 --streaming=false --manageResources=false --monitorJobs=false" Streaming Mode - mvn exec:java -Dexec.mainClass=org.apache.beam.integration.nexmark.Main -Papex-runner "-Dexec.args=--runner=ApexRunner --suite=SMOKE --streamTimeout=60 --streaming=true --manageResources=false --monitorJobs=false" + mvn exec:java -Dexec.mainClass=org.apache.beam.sdk.nexmark.Main -Papex-runner "-Dexec.args=--runner=ApexRunner --suite=SMOKE --streamTimeout=60 --streaming=true --manageResources=false --monitorJobs=false" ## Running SMOKE suite on Google Cloud Dataflow @@ -269,8 +269,8 @@ Submit to Google Dataflow service ``` -java -cp integration/java/target/beam-integration-java-nexmark-bundled-2.1.0-SNAPSHOT.jar \ - org.apache.beam.integration.nexmark.Main \ +java -cp sdks/java/nexmark/target/beam-sdks-java-nexmark-bundled-2.1.0-SNAPSHOT.jar \ + org.apache.beam.sdk.nexmark.Main \ --runner=DataflowRunner --project= \ --zone= \ @@ -303,8 +303,8 @@ java -cp integration/java/target/beam-integration-java-nexmark-bundled-2.1.0-SNA ``` ``` -java -cp integration/java/target/beam-integration-java-nexmark-bundled-2.1.0-SNAPSHOT.jar \ - org.apache.beam.integration.nexmark.Main \ +java -cp sdks/java/nexmark/target/beam-sdks-java-nexmark-bundled-2.1.0-SNAPSHOT.jar \ + org.apache.beam.sdk.nexmark.Main \ --runner=DataflowRunner --project= \ --zone= \ @@ -336,5 +336,5 @@ Building package Submit to the cluster - spark-submit --master yarn-client --class org.apache.beam.integration.nexmark.Main --driver-memory 512m --executor-memory 512m --executor-cores 1 beam-integration-java-nexmark-bundled-2.1.0-SNAPSHOT.jar --runner=SparkRunner --query=0 --streamTimeout=60 --streaming=false --manageResources=false --monitorJobs=true + spark-submit --master yarn-client --class org.apache.beam.sdk.nexmark.Main --driver-memory 512m --executor-memory 512m --executor-cores 1 beam-sdks-java-nexmark-bundled-2.1.0-SNAPSHOT.jar --runner=SparkRunner --query=0 --streamTimeout=60 --streaming=false --manageResources=false --monitorJobs=true diff --git a/sdks/java/nexmark/pom.xml b/sdks/java/nexmark/pom.xml index c1b6025c5e685..67628a1283167 100644 --- a/sdks/java/nexmark/pom.xml +++ b/sdks/java/nexmark/pom.xml @@ -23,7 +23,7 @@ org.apache.beam beam-sdks-java-parent - 2.1.0-SNAPSHOT + 2.2.0-SNAPSHOT ../pom.xml @@ -35,7 +35,7 @@ direct-runner @@ -51,7 +51,6 @@ - apex-runner @@ -60,27 +59,9 @@ beam-runners-apex runtime - - - org.apache.httpcomponents - httpclient - 4.3.5 - runtime - - - commons-codec - commons-codec - - - - flink-runner @@ -92,7 +73,6 @@ - spark-runner @@ -122,7 +102,6 @@ - dataflow-runner @@ -186,23 +165,15 @@ - - - - org.jacoco - jacoco-maven-plugin - - org.apache.beam beam-sdks-java-core - org.apache.beam beam-sdks-java-io-google-cloud-platform @@ -218,7 +189,6 @@ google-api-services-bigquery - com.fasterxml.jackson.core jackson-core @@ -282,7 +252,6 @@ hamcrest-all - org.apache.beam beam-runners-direct-java diff --git a/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/package-info.java b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/package-info.java index 7500a247badff..62218a4419812 100644 --- a/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/package-info.java +++ b/sdks/java/nexmark/src/main/java/org/apache/beam/sdk/nexmark/package-info.java @@ -16,6 +16,6 @@ * limitations under the License. */ /** - * Nexmark. + * Nexmark test suite. */ package org.apache.beam.sdk.nexmark; diff --git a/sdks/java/nexmark/src/main/resources/log4j.properties b/sdks/java/nexmark/src/main/resources/log4j.properties index 7dd57b542f863..14f8acde98cbd 100644 --- a/sdks/java/nexmark/src/main/resources/log4j.properties +++ b/sdks/java/nexmark/src/main/resources/log4j.properties @@ -27,14 +27,14 @@ log4j.logger.org.apache.beam.runners.direct=WARN log4j.logger.org.apache.beam.sdk=WARN # Nexmark specific -log4j.logger.org.apache.beam.integration.nexmark=WARN +log4j.logger.org.apache.beam.sdk.nexmark=WARN # Settings to quiet third party logs that are too verbose log4j.logger.org.spark_project.jetty=WARN log4j.logger.org.spark_project.jetty.util.component.AbstractLifeCycle=ERROR # Setting to quiet spark logs, Beam logs should standout -log4j.logger.org.apache.beam.runners.spark=INFO +log4j.logger.org.apache.beam.runners.spark=WARN log4j.logger.org.apache.spark=WARN log4j.logger.org.spark-project=WARN log4j.logger.io.netty=INFO From 12c0fa68f463b52f21c666ef8cebc7235b79aedf Mon Sep 17 00:00:00 2001 From: David Cavazos Date: Fri, 30 Jun 2017 10:27:25 -0700 Subject: [PATCH 317/346] Add Python mobile gaming streaming examples --- .../examples/complete/game/game_stats.py | 387 ++++++++++++++++++ .../complete/game/hourly_team_score.py | 280 +++++++------ .../examples/complete/game/leader_board.py | 344 ++++++++++++++++ .../examples/complete/game/user_score.py | 179 +++----- 4 files changed, 932 insertions(+), 258 deletions(-) create mode 100644 sdks/python/apache_beam/examples/complete/game/game_stats.py create mode 100644 sdks/python/apache_beam/examples/complete/game/leader_board.py diff --git a/sdks/python/apache_beam/examples/complete/game/game_stats.py b/sdks/python/apache_beam/examples/complete/game/game_stats.py new file mode 100644 index 0000000000000..41813235e70e9 --- /dev/null +++ b/sdks/python/apache_beam/examples/complete/game/game_stats.py @@ -0,0 +1,387 @@ +# +# 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. +# + +"""Fourth in a series of four pipelines that tell a story in a 'gaming' domain. + +New concepts: session windows and finding session duration; use of both +singleton and non-singleton side inputs. + +This pipeline builds on the {@link LeaderBoard} functionality, and adds some +"business intelligence" analysis: abuse detection and usage patterns. The +pipeline derives the Mean user score sum for a window, and uses that information +to identify likely spammers/robots. (The robots have a higher click rate than +the human users). The 'robot' users are then filtered out when calculating the +team scores. + +Additionally, user sessions are tracked: that is, we find bursts of user +activity using session windows. Then, the mean session duration information is +recorded in the context of subsequent fixed windowing. (This could be used to +tell us what games are giving us greater user retention). + +Run injector.Injector to generate pubsub data for this pipeline. The Injector +documentation provides more detail on how to do this. The injector is currently +implemented in Java only, it can be used from the Java SDK. + +The PubSub topic you specify should be the same topic to which the Injector is +publishing. + +To run the Java injector: +/examples/java8$ mvn compile exec:java \ + -Dexec.mainClass=org.apache.beam.examples.complete.game.injector.Injector \ + -Dexec.args="$PROJECT_ID $PUBSUB_TOPIC none" + +For a description of the usage and options, use -h or --help. + +To specify a different runner: + --runner YOUR_RUNNER + +NOTE: When specifying a different runner, additional runner-specific options + may have to be passed in as well + +EXAMPLES +-------- + +# DirectRunner +python game_stats.py \ + --project $PROJECT_ID \ + --topic projects/$PROJECT_ID/topics/$PUBSUB_TOPIC \ + --dataset $BIGQUERY_DATASET + +# DataflowRunner +python game_stats.py \ + --project $PROJECT_ID \ + --topic projects/$PROJECT_ID/topics/$PUBSUB_TOPIC \ + --dataset $BIGQUERY_DATASET \ + --runner DataflowRunner \ + --temp_location gs://$BUCKET/user_score/temp + +-------------------------------------------------------------------------------- +NOTE [BEAM-2354]: This example is not yet runnable by DataflowRunner. + The runner still needs support for: + * the --save_main_session flag when streaming is enabled + * combiners +-------------------------------------------------------------------------------- +""" + +from __future__ import absolute_import +from __future__ import print_function + +import argparse +import csv +import logging +import sys +import time +from datetime import datetime + +import apache_beam as beam +from apache_beam.metrics.metric import Metrics +from apache_beam.options.pipeline_options import GoogleCloudOptions +from apache_beam.options.pipeline_options import PipelineOptions +from apache_beam.options.pipeline_options import SetupOptions +from apache_beam.options.pipeline_options import StandardOptions + + +def timestamp2str(t, fmt='%Y-%m-%d %H:%M:%S.000'): + """Converts a unix timestamp into a formatted string.""" + return datetime.fromtimestamp(t).strftime(fmt) + + +class ParseGameEventFn(beam.DoFn): + """Parses the raw game event info into a Python dictionary. + + Each event line has the following format: + username,teamname,score,timestamp_in_ms,readable_time + + e.g.: + user2_AsparagusPig,AsparagusPig,10,1445230923951,2015-11-02 09:09:28.224 + + The human-readable time string is not used here. + """ + def __init__(self): + super(ParseGameEventFn, self).__init__() + self.num_parse_errors = Metrics.counter(self.__class__, 'num_parse_errors') + + def process(self, elem): + try: + row = list(csv.reader([elem]))[0] + yield { + 'user': row[0], + 'team': row[1], + 'score': int(row[2]), + 'timestamp': int(row[3]) / 1000.0, + } + except: # pylint: disable=bare-except + # Log and count parse errors + self.num_parse_errors.inc() + logging.error('Parse error on "%s"', elem) + + +class ExtractAndSumScore(beam.PTransform): + """A transform to extract key/score information and sum the scores. + The constructor argument `field` determines whether 'team' or 'user' info is + extracted. + """ + def __init__(self, field): + super(ExtractAndSumScore, self).__init__() + self.field = field + + def expand(self, pcoll): + return (pcoll + | beam.Map(lambda elem: (elem[self.field], elem['score'])) + | beam.CombinePerKey(sum)) + + +class TeamScoresDict(beam.DoFn): + """Formats the data into a dictionary of BigQuery columns with their values + + Receives a (team, score) pair, extracts the window start timestamp, and + formats everything together into a dictionary. The dictionary is in the format + {'bigquery_column': value} + """ + def process(self, team_score, window=beam.DoFn.WindowParam): + team, score = team_score + start = timestamp2str(int(window.start)) + yield { + 'team': team, + 'total_score': score, + 'window_start': start, + 'processing_time': timestamp2str(int(time.time())) + } + + +class WriteToBigQuery(beam.PTransform): + """Generate, format, and write BigQuery table row information.""" + def __init__(self, table_name, dataset, schema): + """Initializes the transform. + Args: + table_name: Name of the BigQuery table to use. + dataset: Name of the dataset to use. + schema: Dictionary in the format {'column_name': 'bigquery_type'} + """ + super(WriteToBigQuery, self).__init__() + self.table_name = table_name + self.dataset = dataset + self.schema = schema + + def get_schema(self): + """Build the output table schema.""" + return ', '.join( + '%s:%s' % (col, self.schema[col]) for col in self.schema) + + def get_table(self, pipeline): + """Utility to construct an output table reference.""" + project = pipeline.options.view_as(GoogleCloudOptions).project + return '%s:%s.%s' % (project, self.dataset, self.table_name) + + def expand(self, pcoll): + table = self.get_table(pcoll.pipeline) + return ( + pcoll + | 'ConvertToRow' >> beam.Map( + lambda elem: {col: elem[col] for col in self.schema}) + | beam.io.Write(beam.io.BigQuerySink( + table, + schema=self.get_schema(), + create_disposition=beam.io.BigQueryDisposition.CREATE_IF_NEEDED, + write_disposition=beam.io.BigQueryDisposition.WRITE_APPEND))) + + +class CalculateSpammyUsers(beam.PTransform): + """Filter out all but those users with a high clickrate, which we will + consider as 'spammy' uesrs. + + We do this by finding the mean total score per user, then using that + information as a side input to filter out all but those user scores that are + larger than (mean * SCORE_WEIGHT). + """ + SCORE_WEIGHT = 2.5 + + def expand(self, user_scores): + # Get the sum of scores for each user. + sum_scores = ( + user_scores + | 'SumUsersScores' >> beam.CombinePerKey(sum)) + + # Extract the score from each element, and use it to find the global mean. + global_mean_score = ( + sum_scores + | beam.Values() + | beam.CombineGlobally(beam.combiners.MeanCombineFn())\ + .as_singleton_view()) + + # Filter the user sums using the global mean. + filtered = ( + sum_scores + # Use the derived mean total score (global_mean_score) as a side input. + | 'ProcessAndFilter' >> beam.Filter( + lambda (_, score), global_mean:\ + score > global_mean * self.SCORE_WEIGHT, + global_mean_score)) + return filtered + + +class UserSessionActivity(beam.DoFn): + """Calculate and output an element's session duration, in seconds.""" + def process(self, elem, window=beam.DoFn.WindowParam): + yield (window.end.micros - window.start.micros) / 1000000 + + +def run(argv=None): + """Main entry point; defines and runs the hourly_team_score pipeline.""" + parser = argparse.ArgumentParser() + + parser.add_argument('--topic', + type=str, + required=True, + help='Pub/Sub topic to read from') + parser.add_argument('--dataset', + type=str, + required=True, + help='BigQuery Dataset to write tables to. ' + 'Must already exist.') + parser.add_argument('--table_name', + type=str, + default='game_stats', + help='The BigQuery table name. Should not already exist.') + parser.add_argument('--fixed_window_duration', + type=int, + default=60, + help='Numeric value of fixed window duration for user ' + 'analysis, in minutes') + parser.add_argument('--session_gap', + type=int, + default=5, + help='Numeric value of gap between user sessions, ' + 'in minutes') + parser.add_argument('--user_activity_window_duration', + type=int, + default=30, + help='Numeric value of fixed window for finding mean of ' + 'user session duration, in minutes') + + args, pipeline_args = parser.parse_known_args(argv) + + options = PipelineOptions(pipeline_args) + + # We also require the --project option to access --dataset + if options.view_as(GoogleCloudOptions).project is None: + parser.print_usage() + print(sys.argv[0] + ': error: argument --project is required') + sys.exit(1) + + fixed_window_duration = args.fixed_window_duration * 60 + session_gap = args.session_gap * 60 + user_activity_window_duration = args.user_activity_window_duration * 60 + + # We use the save_main_session option because one or more DoFn's in this + # workflow rely on global context (e.g., a module imported at module level). + options.view_as(SetupOptions).save_main_session = True + + # Enforce that this pipeline is always run in streaming mode + options.view_as(StandardOptions).streaming = True + + with beam.Pipeline(options=options) as p: + # Read events from Pub/Sub using custom timestamps + raw_events = ( + p + | 'ReadPubSub' >> beam.io.gcp.pubsub.ReadStringsFromPubSub(args.topic) + | 'ParseGameEventFn' >> beam.ParDo(ParseGameEventFn()) + | 'AddEventTimestamps' >> beam.Map( + lambda elem: beam.window.TimestampedValue(elem, elem['timestamp']))) + + # Extract username/score pairs from the event stream + user_events = ( + raw_events + | 'ExtractUserScores' >> beam.Map( + lambda elem: (elem['user'], elem['score']))) + + # Calculate the total score per user over fixed windows, and cumulative + # updates for late data + spammers_view = ( + user_events + | 'UserFixedWindows' >> beam.WindowInto( + beam.window.FixedWindows(fixed_window_duration)) + + # Filter out everyone but those with (SCORE_WEIGHT * avg) clickrate. + # These might be robots/spammers. + | 'CalculateSpammyUsers' >> CalculateSpammyUsers() + + # Derive a view from the collection of spammer users. It will be used as + # a side input in calculating the team score sums, below + | 'CreateSpammersView' >> beam.CombineGlobally( + beam.combiners.ToDictCombineFn()).as_singleton_view()) + + # Calculate the total score per team over fixed windows, and emit cumulative + # updates for late data. Uses the side input derived above --the set of + # suspected robots-- to filter out scores from those users from the sum. + # Write the results to BigQuery. + teams_schema = { + 'team': 'STRING', + 'total_score': 'INTEGER', + 'window_start': 'STRING', + 'processing_time': 'STRING', + } + (raw_events # pylint: disable=expression-not-assigned + | 'WindowIntoFixedWindows' >> beam.WindowInto( + beam.window.FixedWindows(fixed_window_duration)) + + # Filter out the detected spammer users, using the side input derived above + | 'FilterOutSpammers' >> beam.Filter( + lambda elem, spammers: elem['user'] not in spammers, + spammers_view) + # Extract and sum teamname/score pairs from the event data. + | 'ExtractAndSumScore' >> ExtractAndSumScore('team') + | 'TeamScoresDict' >> beam.ParDo(TeamScoresDict()) + | 'WriteTeamScoreSums' >> WriteToBigQuery( + args.table_name + '_teams', args.dataset, teams_schema)) + + # Detect user sessions-- that is, a burst of activity separated by a gap + # from further activity. Find and record the mean session lengths. + # This information could help the game designers track the changing user + # engagement as their set of game changes. + sessions_schema = { + 'mean_duration': 'FLOAT', + } + (user_events # pylint: disable=expression-not-assigned + | 'WindowIntoSessions' >> beam.WindowInto( + beam.window.Sessions(session_gap), + timestamp_combiner=beam.window.TimestampCombiner.OUTPUT_AT_EOW) + + # For this use, we care only about the existence of the session, not any + # particular information aggregated over it, so we can just group by key + # and assign a "dummy value" of None. + | beam.CombinePerKey(lambda _: None) + + # Get the duration of the session + | 'UserSessionActivity' >> beam.ParDo(UserSessionActivity()) + + # Re-window to process groups of session sums according to when the + # sessions complete + | 'WindowToExtractSessionMean' >> beam.WindowInto( + beam.window.FixedWindows(user_activity_window_duration)) + + # Find the mean session duration in each window + | beam.CombineGlobally(beam.combiners.MeanCombineFn()).without_defaults() + | 'FormatAvgSessionLength' >> beam.Map( + lambda elem: {'mean_duration': float(elem)}) + | 'WriteAvgSessionLength' >> WriteToBigQuery( + args.table_name + '_sessions', args.dataset, sessions_schema)) + + +if __name__ == '__main__': + logging.getLogger().setLevel(logging.INFO) + run() diff --git a/sdks/python/apache_beam/examples/complete/game/hourly_team_score.py b/sdks/python/apache_beam/examples/complete/game/hourly_team_score.py index 9f398d9995f22..9dd8b05a5c5c6 100644 --- a/sdks/python/apache_beam/examples/complete/game/hourly_team_score.py +++ b/sdks/python/apache_beam/examples/complete/game/hourly_team_score.py @@ -18,7 +18,7 @@ """Second in a series of four pipelines that tell a story in a 'gaming' domain. In addition to the concepts introduced in `user_score`, new concepts include: -windowing and element timestamps; use of `Filter`. +windowing and element timestamps; use of `Filter`; using standalone DoFns. This pipeline processes data collected from gaming events in batch, building on `user_score` but using fixed windows. It calculates the sum of scores per team, @@ -31,10 +31,6 @@ in that we don't get results from plays at the beginning of the batch's time period until the batch is processed. -To execute this pipeline using the static example input data, specify the -`--dataset=YOUR-DATASET` flag along with other runner specific flags. (Note: -BigQuery dataset you specify must already exist.) - Optionally include the `--input` argument to specify a batch input file. To indicate a time after which the data should be filtered out, include the `--stop_min` arg. E.g., `--stop_min=2015-10-18-23-59` indicates that any data @@ -43,29 +39,62 @@ the `--start_min` arg. If you're using the default input "gs://dataflow-samples/game/gaming_data*.csv", then `--start_min=2015-11-16-16-10 --stop_min=2015-11-17-16-10` are good values. + +For a description of the usage and options, use -h or --help. + +To specify a different runner: + --runner YOUR_RUNNER + +NOTE: When specifying a different runner, additional runner-specific options + may have to be passed in as well + +EXAMPLES +-------- + +# DirectRunner +python hourly_team_score.py \ + --project $PROJECT_ID \ + --dataset $BIGQUERY_DATASET + +# DataflowRunner +python hourly_team_score.py \ + --project $PROJECT_ID \ + --dataset $BIGQUERY_DATASET \ + --runner DataflowRunner \ + --temp_location gs://$BUCKET/user_score/temp """ from __future__ import absolute_import +from __future__ import print_function import argparse -import datetime +import csv import logging +import sys +import time +from datetime import datetime import apache_beam as beam -from apache_beam import typehints -from apache_beam.io import ReadFromText -from apache_beam.metrics import Metrics -from apache_beam.transforms.window import FixedWindows -from apache_beam.transforms.window import TimestampedValue -from apache_beam.typehints import with_input_types -from apache_beam.typehints import with_output_types +from apache_beam.metrics.metric import Metrics from apache_beam.options.pipeline_options import GoogleCloudOptions from apache_beam.options.pipeline_options import PipelineOptions from apache_beam.options.pipeline_options import SetupOptions -class ParseEventFn(beam.DoFn): - """Parses the raw game event info into GameActionInfo tuples. +def str2timestamp(s, fmt='%Y-%m-%d-%H-%M'): + """Converts a string into a unix timestamp.""" + dt = datetime.strptime(s, fmt) + epoch = datetime.utcfromtimestamp(0) + return (dt - epoch).total_seconds() + + +def timestamp2str(t, fmt='%Y-%m-%d %H:%M:%S.000'): + """Converts a unix timestamp into a formatted string.""" + return datetime.fromtimestamp(t).strftime(fmt) + + +class ParseGameEventFn(beam.DoFn): + """Parses the raw game event info into a Python dictionary. Each event line has the following format: username,teamname,score,timestamp_in_ms,readable_time @@ -76,32 +105,26 @@ class ParseEventFn(beam.DoFn): The human-readable time string is not used here. """ def __init__(self): - super(ParseEventFn, self).__init__() + super(ParseGameEventFn, self).__init__() self.num_parse_errors = Metrics.counter(self.__class__, 'num_parse_errors') - def process(self, element): - components = element.split(',') + def process(self, elem): try: - user = components[0].strip() - team = components[1].strip() - score = int(components[2].strip()) - timestamp = int(components[3].strip()) - yield {'user': user, 'team': team, 'score': score, 'timestamp': timestamp} + row = list(csv.reader([elem]))[0] + yield { + 'user': row[0], + 'team': row[1], + 'score': int(row[2]), + 'timestamp': int(row[3]) / 1000.0, + } except: # pylint: disable=bare-except - # Log and count parse errors. + # Log and count parse errors self.num_parse_errors.inc() - logging.info('Parse error on %s.', element) - - -@with_input_types(ints=typehints.Iterable[int]) -@with_output_types(int) -def sum_ints(ints): - return sum(ints) + logging.error('Parse error on "%s"', elem) class ExtractAndSumScore(beam.PTransform): """A transform to extract key/score information and sum the scores. - The constructor argument `field` determines whether 'team' or 'user' info is extracted. """ @@ -111,75 +134,94 @@ def __init__(self, field): def expand(self, pcoll): return (pcoll - | beam.Map(lambda info: (info[self.field], info['score'])) - | beam.CombinePerKey(sum_ints)) + | beam.Map(lambda elem: (elem[self.field], elem['score'])) + | beam.CombinePerKey(sum)) -def configure_bigquery_write(): +class HourlyTeamScore(beam.PTransform): + def __init__(self, start_min, stop_min, window_duration): + super(HourlyTeamScore, self).__init__() + self.start_timestamp = str2timestamp(start_min) + self.stop_timestamp = str2timestamp(stop_min) + self.window_duration_in_seconds = window_duration * 60 - def window_start_format(element, window): - dt = datetime.datetime.fromtimestamp(int(window.start)) - return dt.strftime('%Y-%m-%d %H:%M:%S') + def expand(self, pcoll): + return ( + pcoll + | 'ParseGameEventFn' >> beam.ParDo(ParseGameEventFn()) - return [ - ('team', 'STRING', lambda e, w: e[0]), - ('total_score', 'INTEGER', lambda e, w: e[1]), - ('window_start', 'STRING', window_start_format), - ] + # Filter out data before and after the given times so that it is not + # included in the calculations. As we collect data in batches (say, by + # day), the batch for the day that we want to analyze could potentially + # include some late-arriving data from the previous day. If so, we want + # to weed it out. Similarly, if we include data from the following day + # (to scoop up late-arriving events from the day we're analyzing), we + # need to weed out events that fall after the time period we want to + # analyze. + | 'FilterStartTime' >> beam.Filter( + lambda elem: elem['timestamp'] > self.start_timestamp) + | 'FilterEndTime' >> beam.Filter( + lambda elem: elem['timestamp'] < self.stop_timestamp) + # Add an element timestamp based on the event log, and apply fixed + # windowing. + | 'AddEventTimestamps' >> beam.Map( + lambda elem: beam.window.TimestampedValue(elem, elem['timestamp'])) + | 'FixedWindowsTeam' >> beam.WindowInto( + beam.window.FixedWindows(self.window_duration_in_seconds)) -class WriteWindowedToBigQuery(beam.PTransform): - """Generate, format, and write BigQuery table row information. + # Extract and sum teamname/score pairs from the event data. + | 'ExtractAndSumScore' >> ExtractAndSumScore('team')) - This class may be used for writes that require access to the window - information. + +class TeamScoresDict(beam.DoFn): + """Formats the data into a dictionary of BigQuery columns with their values + + Receives a (team, score) pair, extracts the window start timestamp, and + formats everything together into a dictionary. The dictionary is in the format + {'bigquery_column': value} """ - def __init__(self, table_name, dataset, field_info): + def process(self, team_score, window=beam.DoFn.WindowParam): + team, score = team_score + start = timestamp2str(int(window.start)) + yield { + 'team': team, + 'total_score': score, + 'window_start': start, + 'processing_time': timestamp2str(int(time.time())) + } + + +class WriteToBigQuery(beam.PTransform): + """Generate, format, and write BigQuery table row information.""" + def __init__(self, table_name, dataset, schema): """Initializes the transform. - Args: table_name: Name of the BigQuery table to use. dataset: Name of the dataset to use. - field_info: List of tuples that holds information about output table field - definitions. The tuples are in the - (field_name, field_type, field_fn) format, where field_name is - the name of the field, field_type is the BigQuery type of the - field and field_fn is a lambda function to generate the field - value from the element. + schema: Dictionary in the format {'column_name': 'bigquery_type'} """ - super(WriteWindowedToBigQuery, self).__init__() + super(WriteToBigQuery, self).__init__() self.table_name = table_name self.dataset = dataset - self.field_info = field_info + self.schema = schema def get_schema(self): """Build the output table schema.""" return ', '.join( - '%s:%s' % (entry[0], entry[1]) for entry in self.field_info) + '%s:%s' % (col, self.schema[col]) for col in self.schema) def get_table(self, pipeline): """Utility to construct an output table reference.""" project = pipeline.options.view_as(GoogleCloudOptions).project return '%s:%s.%s' % (project, self.dataset, self.table_name) - class BuildRowFn(beam.DoFn): - """Convert each key/score pair into a BigQuery TableRow as specified.""" - def __init__(self, field_info): - super(WriteWindowedToBigQuery.BuildRowFn, self).__init__() - self.field_info = field_info - - def process(self, element, window=beam.DoFn.WindowParam): - row = {} - for entry in self.field_info: - row[entry[0]] = entry[2](element, window) - yield row - def expand(self, pcoll): table = self.get_table(pcoll.pipeline) return ( pcoll - | 'ConvertToRow' >> beam.ParDo( - WriteWindowedToBigQuery.BuildRowFn(self.field_info)) + | 'ConvertToRow' >> beam.Map( + lambda elem: {col: elem[col] for col in self.schema}) | beam.io.Write(beam.io.BigQuerySink( table, schema=self.get_schema(), @@ -187,52 +229,6 @@ def expand(self, pcoll): write_disposition=beam.io.BigQueryDisposition.WRITE_APPEND))) -def string_to_timestamp(datetime_str): - dt = datetime.datetime.strptime(datetime_str, '%Y-%m-%d-%H-%M') - epoch = datetime.datetime.utcfromtimestamp(0) - return (dt - epoch).total_seconds() * 1000.0 - - -class HourlyTeamScore(beam.PTransform): - def __init__(self, start_min, stop_min, window_duration): - super(HourlyTeamScore, self).__init__() - self.start_min = start_min - self.stop_min = stop_min - self.window_duration = window_duration - - def expand(self, pcoll): - start_min_filter = string_to_timestamp(self.start_min) - end_min_filter = string_to_timestamp(self.stop_min) - - return ( - pcoll - | 'ParseGameEvent' >> beam.ParDo(ParseEventFn()) - # Filter out data before and after the given times so that it is not - # included in the calculations. As we collect data in batches (say, by - # day), the batch for the day that we want to analyze could potentially - # include some late-arriving data from the previous day. If so, we want - # to weed it out. Similarly, if we include data from the following day - # (to scoop up late-arriving events from the day we're analyzing), we - # need to weed out events that fall after the time period we want to - # analyze. - | 'FilterStartTime' >> beam.Filter( - lambda element: element['timestamp'] > start_min_filter) - | 'FilterEndTime' >> beam.Filter( - lambda element: element['timestamp'] < end_min_filter) - # Add an element timestamp based on the event log, and apply fixed - # windowing. - # Convert element['timestamp'] into seconds as expected by - # TimestampedValue. - | 'AddEventTimestamps' >> beam.Map( - lambda element: TimestampedValue( - element, element['timestamp'] / 1000.0)) - # Convert window_duration into seconds as expected by FixedWindows. - | 'FixedWindowsTeam' >> beam.WindowInto(FixedWindows( - size=self.window_duration * 60)) - # Extract and sum teamname/score pairs from the event data. - | 'ExtractTeamScore' >> ExtractAndSumScore('team')) - - def run(argv=None): """Main entry point; defines and runs the hourly_team_score pipeline.""" parser = argparse.ArgumentParser() @@ -240,24 +236,23 @@ def run(argv=None): # The default maps to two large Google Cloud Storage files (each ~12GB) # holding two subsequent day's worth (roughly) of data. parser.add_argument('--input', - dest='input', - default='gs://dataflow-samples/game/gaming_data*.csv', + type=str, + default='gs://apache-beam-samples/game/gaming_data*.csv', help='Path to the data file(s) containing game data.') parser.add_argument('--dataset', - dest='dataset', + type=str, required=True, help='BigQuery Dataset to write tables to. ' - 'Must already exist.') + 'Must already exist.') parser.add_argument('--table_name', - dest='table_name', - default='hourly_team_score', + default='leader_board', help='The BigQuery table name. Should not already exist.') parser.add_argument('--window_duration', type=int, default=60, help='Numeric value of fixed window duration, in minutes') parser.add_argument('--start_min', - dest='start_min', + type=str, default='1970-01-01-00-00', help='String representation of the first minute after ' 'which to generate results in the format: ' @@ -265,7 +260,7 @@ def run(argv=None): 'prior to that minute won\'t be included in the ' 'sums.') parser.add_argument('--stop_min', - dest='stop_min', + type=str, default='2100-01-01-00-00', help='String representation of the first minute for ' 'which to generate results in the format: ' @@ -273,18 +268,33 @@ def run(argv=None): 'after to that minute won\'t be included in the ' 'sums.') - known_args, pipeline_args = parser.parse_known_args(argv) + args, pipeline_args = parser.parse_known_args(argv) + + options = PipelineOptions(pipeline_args) + + # We also require the --project option to access --dataset + if options.view_as(GoogleCloudOptions).project is None: + parser.print_usage() + print(sys.argv[0] + ': error: argument --project is required') + sys.exit(1) - pipeline_options = PipelineOptions(pipeline_args) - pipeline_options.view_as(SetupOptions).save_main_session = True - with beam.Pipeline(options=pipeline_options) as p: + # We use the save_main_session option because one or more DoFn's in this + # workflow rely on global context (e.g., a module imported at module level). + options.view_as(SetupOptions).save_main_session = True + schema = { + 'team': 'STRING', + 'total_score': 'INTEGER', + 'window_start': 'STRING', + } + with beam.Pipeline(options=options) as p: (p # pylint: disable=expression-not-assigned - | ReadFromText(known_args.input) - | HourlyTeamScore( - known_args.start_min, known_args.stop_min, known_args.window_duration) - | WriteWindowedToBigQuery( - known_args.table_name, known_args.dataset, configure_bigquery_write())) + | 'ReadInputText' >> beam.io.ReadFromText(args.input) + | 'HourlyTeamScore' >> HourlyTeamScore( + args.start_min, args.stop_min, args.window_duration) + | 'TeamScoresDict' >> beam.ParDo(TeamScoresDict()) + | 'WriteTeamScoreSums' >> WriteToBigQuery( + args.table_name, args.dataset, schema)) if __name__ == '__main__': diff --git a/sdks/python/apache_beam/examples/complete/game/leader_board.py b/sdks/python/apache_beam/examples/complete/game/leader_board.py new file mode 100644 index 0000000000000..2936bc96a31a7 --- /dev/null +++ b/sdks/python/apache_beam/examples/complete/game/leader_board.py @@ -0,0 +1,344 @@ +# +# 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. +# + +"""Third in a series of four pipelines that tell a story in a 'gaming' domain. + +Concepts include: processing unbounded data using fixed windows; use of custom +timestamps and event-time processing; generation of early/speculative results; +using AccumulationMode.ACCUMULATING to do cumulative processing of late-arriving +data. + +This pipeline processes an unbounded stream of 'game events'. The calculation of +the team scores uses fixed windowing based on event time (the time of the game +play event), not processing time (the time that an event is processed by the +pipeline). The pipeline calculates the sum of scores per team, for each window. +By default, the team scores are calculated using one-hour windows. + +In contrast-- to demo another windowing option-- the user scores are calculated +using a global window, which periodically (every ten minutes) emits cumulative +user score sums. + +In contrast to the previous pipelines in the series, which used static, finite +input data, here we're using an unbounded data source, which lets us provide +speculative results, and allows handling of late data, at much lower latency. +We can use the early/speculative results to keep a 'leaderboard' updated in +near-realtime. Our handling of late data lets us generate correct results, +e.g. for 'team prizes'. We're now outputting window results as they're +calculated, giving us much lower latency than with the previous batch examples. + +Run injector.Injector to generate pubsub data for this pipeline. The Injector +documentation provides more detail on how to do this. The injector is currently +implemented in Java only, it can be used from the Java SDK. + +The PubSub topic you specify should be the same topic to which the Injector is +publishing. + +To run the Java injector: +/examples/java8$ mvn compile exec:java \ + -Dexec.mainClass=org.apache.beam.examples.complete.game.injector.Injector \ + -Dexec.args="$PROJECT_ID $PUBSUB_TOPIC none" + +For a description of the usage and options, use -h or --help. + +To specify a different runner: + --runner YOUR_RUNNER + +NOTE: When specifying a different runner, additional runner-specific options + may have to be passed in as well + +EXAMPLES +-------- + +# DirectRunner +python leader_board.py \ + --project $PROJECT_ID \ + --topic projects/$PROJECT_ID/topics/$PUBSUB_TOPIC \ + --dataset $BIGQUERY_DATASET + +# DataflowRunner +python leader_board.py \ + --project $PROJECT_ID \ + --topic projects/$PROJECT_ID/topics/$PUBSUB_TOPIC \ + --dataset $BIGQUERY_DATASET \ + --runner DataflowRunner \ + --temp_location gs://$BUCKET/user_score/temp + +-------------------------------------------------------------------------------- +NOTE [BEAM-2354]: This example is not yet runnable by DataflowRunner. + The runner still needs support for: + * the --save_main_session flag when streaming is enabled +-------------------------------------------------------------------------------- +""" + +from __future__ import absolute_import +from __future__ import print_function + +import argparse +import csv +import logging +import sys +import time +from datetime import datetime + +import apache_beam as beam +from apache_beam.metrics.metric import Metrics +from apache_beam.options.pipeline_options import GoogleCloudOptions +from apache_beam.options.pipeline_options import PipelineOptions +from apache_beam.options.pipeline_options import SetupOptions +from apache_beam.options.pipeline_options import StandardOptions +from apache_beam.transforms import trigger + + +def timestamp2str(t, fmt='%Y-%m-%d %H:%M:%S.000'): + """Converts a unix timestamp into a formatted string.""" + return datetime.fromtimestamp(t).strftime(fmt) + + +class ParseGameEventFn(beam.DoFn): + """Parses the raw game event info into a Python dictionary. + + Each event line has the following format: + username,teamname,score,timestamp_in_ms,readable_time + + e.g.: + user2_AsparagusPig,AsparagusPig,10,1445230923951,2015-11-02 09:09:28.224 + + The human-readable time string is not used here. + """ + def __init__(self): + super(ParseGameEventFn, self).__init__() + self.num_parse_errors = Metrics.counter(self.__class__, 'num_parse_errors') + + def process(self, elem): + try: + row = list(csv.reader([elem]))[0] + yield { + 'user': row[0], + 'team': row[1], + 'score': int(row[2]), + 'timestamp': int(row[3]) / 1000.0, + } + except: # pylint: disable=bare-except + # Log and count parse errors + self.num_parse_errors.inc() + logging.error('Parse error on "%s"', elem) + + +class ExtractAndSumScore(beam.PTransform): + """A transform to extract key/score information and sum the scores. + The constructor argument `field` determines whether 'team' or 'user' info is + extracted. + """ + def __init__(self, field): + super(ExtractAndSumScore, self).__init__() + self.field = field + + def expand(self, pcoll): + return (pcoll + | beam.Map(lambda elem: (elem[self.field], elem['score'])) + | beam.CombinePerKey(sum)) + + +class TeamScoresDict(beam.DoFn): + """Formats the data into a dictionary of BigQuery columns with their values + + Receives a (team, score) pair, extracts the window start timestamp, and + formats everything together into a dictionary. The dictionary is in the format + {'bigquery_column': value} + """ + def process(self, team_score, window=beam.DoFn.WindowParam): + team, score = team_score + start = timestamp2str(int(window.start)) + yield { + 'team': team, + 'total_score': score, + 'window_start': start, + 'processing_time': timestamp2str(int(time.time())) + } + + +class WriteToBigQuery(beam.PTransform): + """Generate, format, and write BigQuery table row information.""" + def __init__(self, table_name, dataset, schema): + """Initializes the transform. + Args: + table_name: Name of the BigQuery table to use. + dataset: Name of the dataset to use. + schema: Dictionary in the format {'column_name': 'bigquery_type'} + """ + super(WriteToBigQuery, self).__init__() + self.table_name = table_name + self.dataset = dataset + self.schema = schema + + def get_schema(self): + """Build the output table schema.""" + return ', '.join( + '%s:%s' % (col, self.schema[col]) for col in self.schema) + + def get_table(self, pipeline): + """Utility to construct an output table reference.""" + project = pipeline.options.view_as(GoogleCloudOptions).project + return '%s:%s.%s' % (project, self.dataset, self.table_name) + + def expand(self, pcoll): + table = self.get_table(pcoll.pipeline) + return ( + pcoll + | 'ConvertToRow' >> beam.Map( + lambda elem: {col: elem[col] for col in self.schema}) + | beam.io.Write(beam.io.BigQuerySink( + table, + schema=self.get_schema(), + create_disposition=beam.io.BigQueryDisposition.CREATE_IF_NEEDED, + write_disposition=beam.io.BigQueryDisposition.WRITE_APPEND))) + + +class CalculateTeamScores(beam.PTransform): + """Calculates scores for each team within the configured window duration. + + Extract team/score pairs from the event stream, using hour-long windows by + default. + """ + def __init__(self, team_window_duration, allowed_lateness): + super(CalculateTeamScores, self).__init__() + self.team_window_duration = team_window_duration * 60 + self.allowed_lateness_seconds = allowed_lateness * 60 + + def expand(self, pcoll): + # NOTE: the behavior does not exactly match the Java example + # TODO: allowed_lateness not implemented yet in FixedWindows + # TODO: AfterProcessingTime not implemented yet, replace AfterCount + return ( + pcoll + # We will get early (speculative) results as well as cumulative + # processing of late data. + | 'LeaderboardTeamFixedWindows' >> beam.WindowInto( + beam.window.FixedWindows(self.team_window_duration), + trigger=trigger.AfterWatermark(trigger.AfterCount(10), + trigger.AfterCount(20)), + accumulation_mode=trigger.AccumulationMode.ACCUMULATING) + # Extract and sum teamname/score pairs from the event data. + | 'ExtractAndSumScore' >> ExtractAndSumScore('team')) + + +class CalculateUserScores(beam.PTransform): + """Extract user/score pairs from the event stream using processing time, via + global windowing. Get periodic updates on all users' running scores. + """ + def __init__(self, allowed_lateness): + super(CalculateUserScores, self).__init__() + self.allowed_lateness_seconds = allowed_lateness * 60 + + def expand(self, pcoll): + # NOTE: the behavior does not exactly match the Java example + # TODO: allowed_lateness not implemented yet in FixedWindows + # TODO: AfterProcessingTime not implemented yet, replace AfterCount + return ( + pcoll + # Get periodic results every ten events. + | 'LeaderboardUserGlobalWindows' >> beam.WindowInto( + beam.window.GlobalWindows(), + trigger=trigger.Repeatedly(trigger.AfterCount(10)), + accumulation_mode=trigger.AccumulationMode.ACCUMULATING) + # Extract and sum username/score pairs from the event data. + | 'ExtractAndSumScore' >> ExtractAndSumScore('user')) + + +def run(argv=None): + """Main entry point; defines and runs the hourly_team_score pipeline.""" + parser = argparse.ArgumentParser() + + parser.add_argument('--topic', + type=str, + required=True, + help='Pub/Sub topic to read from') + parser.add_argument('--dataset', + type=str, + required=True, + help='BigQuery Dataset to write tables to. ' + 'Must already exist.') + parser.add_argument('--table_name', + default='leader_board', + help='The BigQuery table name. Should not already exist.') + parser.add_argument('--team_window_duration', + type=int, + default=60, + help='Numeric value of fixed window duration for team ' + 'analysis, in minutes') + parser.add_argument('--allowed_lateness', + type=int, + default=120, + help='Numeric value of allowed data lateness, in minutes') + + args, pipeline_args = parser.parse_known_args(argv) + + options = PipelineOptions(pipeline_args) + + # We also require the --project option to access --dataset + if options.view_as(GoogleCloudOptions).project is None: + parser.print_usage() + print(sys.argv[0] + ': error: argument --project is required') + sys.exit(1) + + # We use the save_main_session option because one or more DoFn's in this + # workflow rely on global context (e.g., a module imported at module level). + options.view_as(SetupOptions).save_main_session = True + + # Enforce that this pipeline is always run in streaming mode + options.view_as(StandardOptions).streaming = True + + with beam.Pipeline(options=options) as p: + # Read game events from Pub/Sub using custom timestamps, which are extracted + # from the pubsub data elements, and parse the data. + events = ( + p + | 'ReadPubSub' >> beam.io.gcp.pubsub.ReadStringsFromPubSub(args.topic) + | 'ParseGameEventFn' >> beam.ParDo(ParseGameEventFn()) + | 'AddEventTimestamps' >> beam.Map( + lambda elem: beam.window.TimestampedValue(elem, elem['timestamp']))) + + # Get team scores and write the results to BigQuery + teams_schema = { + 'team': 'STRING', + 'total_score': 'INTEGER', + 'window_start': 'STRING', + 'processing_time': 'STRING', + } + (events # pylint: disable=expression-not-assigned + | 'CalculateTeamScores' >> CalculateTeamScores( + args.team_window_duration, args.allowed_lateness) + | 'TeamScoresDict' >> beam.ParDo(TeamScoresDict()) + | 'WriteTeamScoreSums' >> WriteToBigQuery( + args.table_name + '_teams', args.dataset, teams_schema)) + + # Get user scores and write the results to BigQuery + users_schema = { + 'user': 'STRING', + 'total_score': 'INTEGER', + } + (events # pylint: disable=expression-not-assigned + | 'CalculateUserScores' >> CalculateUserScores(args.allowed_lateness) + | 'FormatUserScoreSums' >> beam.Map( + lambda (user, score): {'user': user, 'total_score': score}) + | 'WriteUserScoreSums' >> WriteToBigQuery( + args.table_name + '_users', args.dataset, users_schema)) + + +if __name__ == '__main__': + logging.getLogger().setLevel(logging.INFO) + run() diff --git a/sdks/python/apache_beam/examples/complete/game/user_score.py b/sdks/python/apache_beam/examples/complete/game/user_score.py index c9f273814a85f..ee78d63f73ab5 100644 --- a/sdks/python/apache_beam/examples/complete/game/user_score.py +++ b/sdks/python/apache_beam/examples/complete/game/user_score.py @@ -16,8 +16,9 @@ # """First in a series of four pipelines that tell a story in a 'gaming' domain. -Concepts: batch processing; reading input from Google Cloud Storage and writing -output to BigQuery; using standalone DoFns; use of the sum by key transform. +Concepts: batch processing; reading input from Google Cloud Storage or a from a +local text file, and writing output to a text file; using standalone DoFns; use +of the CombinePerKey transform. In this gaming scenario, many users play, as members of different teams, over the course of a day, and their actions are logged for processing. Some of the @@ -29,32 +30,41 @@ (collected, say, for each day). The batch processing will not include any late data that arrives after the day's cutoff point. -To execute this pipeline using the static example input data, specify the -`--dataset=YOUR-DATASET` flag along with other runner specific flags. Note: -The BigQuery dataset you specify must already exist. You can simply create a new -empty BigQuery dataset if you don't have an existing one. +For a description of the usage and options, use -h or --help. -Optionally include the `--input` argument to specify a batch input file. See the -`--input` default value for an example batch data file. +To specify a different runner: + --runner YOUR_RUNNER + +NOTE: When specifying a different runner, additional runner-specific options + may have to be passed in as well + +EXAMPLES +-------- + +# DirectRunner +python user_score.py \ + --output /local/path/user_score/output + +# DataflowRunner +python user_score.py \ + --output gs://$BUCKET/user_score/output \ + --runner DataflowRunner \ + --project $PROJECT_ID \ + --temp_location gs://$BUCKET/user_score/temp """ from __future__ import absolute_import import argparse +import csv import logging import apache_beam as beam -from apache_beam import typehints -from apache_beam.io import ReadFromText -from apache_beam.metrics import Metrics -from apache_beam.typehints import with_input_types -from apache_beam.typehints import with_output_types -from apache_beam.options.pipeline_options import GoogleCloudOptions -from apache_beam.options.pipeline_options import PipelineOptions +from apache_beam.metrics.metric import Metrics -class ParseEventFn(beam.DoFn): - """Parses the raw game event info into GameActionInfo tuples. +class ParseGameEventFn(beam.DoFn): + """Parses the raw game event info into a Python dictionary. Each event line has the following format: username,teamname,score,timestamp_in_ms,readable_time @@ -65,32 +75,26 @@ class ParseEventFn(beam.DoFn): The human-readable time string is not used here. """ def __init__(self): - super(ParseEventFn, self).__init__() + super(ParseGameEventFn, self).__init__() self.num_parse_errors = Metrics.counter(self.__class__, 'num_parse_errors') - def process(self, element): - components = element.split(',') + def process(self, elem): try: - user = components[0].strip() - team = components[1].strip() - score = int(components[2].strip()) - timestamp = int(components[3].strip()) - yield {'user': user, 'team': team, 'score': score, 'timestamp': timestamp} + row = list(csv.reader([elem]))[0] + yield { + 'user': row[0], + 'team': row[1], + 'score': int(row[2]), + 'timestamp': int(row[3]) / 1000.0, + } except: # pylint: disable=bare-except - # Log and count parse errors. + # Log and count parse errors self.num_parse_errors.inc() - logging.info('Parse error on %s.', element) - - -@with_input_types(ints=typehints.Iterable[int]) -@with_output_types(int) -def sum_ints(ints): - return sum(ints) + logging.error('Parse error on "%s"', elem) class ExtractAndSumScore(beam.PTransform): """A transform to extract key/score information and sum the scores. - The constructor argument `field` determines whether 'team' or 'user' info is extracted. """ @@ -100,83 +104,17 @@ def __init__(self, field): def expand(self, pcoll): return (pcoll - | beam.Map(lambda info: (info[self.field], info['score'])) - | beam.CombinePerKey(sum_ints)) - - -def configure_bigquery_write(): - return [ - ('user', 'STRING', lambda e: e[0]), - ('total_score', 'INTEGER', lambda e: e[1]), - ] - + | beam.Map(lambda elem: (elem[self.field], elem['score'])) + | beam.CombinePerKey(sum)) -class WriteToBigQuery(beam.PTransform): - """Generate, format, and write BigQuery table row information. - - Use provided information about the field names and types, as well as lambda - functions that describe how to generate their values. - """ - - def __init__(self, table_name, dataset, field_info): - """Initializes the transform. - - Args: - table_name: Name of the BigQuery table to use. - dataset: Name of the dataset to use. - field_info: List of tuples that holds information about output table field - definitions. The tuples are in the - (field_name, field_type, field_fn) format, where field_name is - the name of the field, field_type is the BigQuery type of the - field and field_fn is a lambda function to generate the field - value from the element. - """ - super(WriteToBigQuery, self).__init__() - self.table_name = table_name - self.dataset = dataset - self.field_info = field_info - - def get_schema(self): - """Build the output table schema.""" - return ', '.join( - '%s:%s' % (entry[0], entry[1]) for entry in self.field_info) - - def get_table(self, pipeline): - """Utility to construct an output table reference.""" - project = pipeline.options.view_as(GoogleCloudOptions).project - return '%s:%s.%s' % (project, self.dataset, self.table_name) - - class BuildRowFn(beam.DoFn): - """Convert each key/score pair into a BigQuery TableRow as specified.""" - def __init__(self, field_info): - super(WriteToBigQuery.BuildRowFn, self).__init__() - self.field_info = field_info - - def process(self, element): - row = {} - for entry in self.field_info: - row[entry[0]] = entry[2](element) - yield row +class UserScore(beam.PTransform): def expand(self, pcoll): - table = self.get_table(pcoll.pipeline) return ( pcoll - | 'ConvertToRow' >> beam.ParDo( - WriteToBigQuery.BuildRowFn(self.field_info)) - | beam.io.Write(beam.io.BigQuerySink( - table, - schema=self.get_schema(), - create_disposition=beam.io.BigQueryDisposition.CREATE_IF_NEEDED, - write_disposition=beam.io.BigQueryDisposition.WRITE_APPEND))) - - -class UserScore(beam.PTransform): - def expand(self, pcoll): - return (pcoll - | 'ParseGameEvent' >> beam.ParDo(ParseEventFn()) - # Extract and sum username/score pairs from the event data. - | 'ExtractUserScore' >> ExtractAndSumScore('user')) + | 'ParseGameEventFn' >> beam.ParDo(ParseGameEventFn()) + # Extract and sum username/score pairs from the event data. + | 'ExtractAndSumScore' >> ExtractAndSumScore('user')) def run(argv=None): @@ -186,28 +124,23 @@ def run(argv=None): # The default maps to two large Google Cloud Storage files (each ~12GB) # holding two subsequent day's worth (roughly) of data. parser.add_argument('--input', - dest='input', - default='gs://dataflow-samples/game/gaming_data*.csv', + type=str, + default='gs://apache-beam-samples/game/gaming_data*.csv', help='Path to the data file(s) containing game data.') - parser.add_argument('--dataset', - dest='dataset', + parser.add_argument('--output', + type=str, required=True, - help='BigQuery Dataset to write tables to. ' - 'Must already exist.') - parser.add_argument('--table_name', - dest='table_name', - default='user_score', - help='The BigQuery table name. Should not already exist.') - known_args, pipeline_args = parser.parse_known_args(argv) + help='Path to the output file(s).') - pipeline_options = PipelineOptions(pipeline_args) - with beam.Pipeline(options=pipeline_options) as p: + args, pipeline_args = parser.parse_known_args(argv) + with beam.Pipeline(argv=pipeline_args) as p: (p # pylint: disable=expression-not-assigned - | ReadFromText(known_args.input) # Read events from a file and parse them. - | UserScore() - | WriteToBigQuery( - known_args.table_name, known_args.dataset, configure_bigquery_write())) + | 'ReadInputText' >> beam.io.ReadFromText(args.input) + | 'UserScore' >> UserScore() + | 'FormatUserScoreSums' >> beam.Map( + lambda (user, score): 'user: %s, total_score: %s' % (user, score)) + | 'WriteUserScoreSums' >> beam.io.WriteToText(args.output)) if __name__ == '__main__': From e183b24ef9d07a6e2963c16c42c9d3a60166d3b0 Mon Sep 17 00:00:00 2001 From: Yunqing Zhou Date: Thu, 17 Aug 2017 23:17:52 -0700 Subject: [PATCH 318/346] Fix a bug in AvroIO, in which a SerializableFunction is created with a context containing a un-serializable member (Schema) --- .../java/org/apache/beam/sdk/io/AvroIO.java | 21 ++++++++++++------- .../org/apache/beam/sdk/io/AvroIOTest.java | 21 +++++++++++++++++++ 2 files changed, 35 insertions(+), 7 deletions(-) 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 653b80699dd81..910d8e2bfc494 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 @@ -762,15 +762,22 @@ public TypedWrite to(ResourceId outputPrefix) { return toResource(StaticValueProvider.of(outputPrefix)); } + private static class OutputPrefixToResourceId + implements SerializableFunction { + @Override + public ResourceId apply(String input) { + return FileBasedSink.convertToFileResourceIfPossible(input); + } + } + /** Like {@link #to(String)}. */ public TypedWrite to(ValueProvider outputPrefix) { - return toResource(NestedValueProvider.of(outputPrefix, - new SerializableFunction() { - @Override - public ResourceId apply(String input) { - return FileBasedSink.convertToFileResourceIfPossible(input); - } - })); + return toResource( + NestedValueProvider.of( + outputPrefix, + // The function cannot be created as an anonymous class here since the enclosed class + // may contain unserializable members. + new OutputPrefixToResourceId())); } /** Like {@link #to(ResourceId)}. */ 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 a96b6bed2741c..d0aa02c1f07b9 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 @@ -64,6 +64,7 @@ import org.apache.beam.sdk.io.FileBasedSink.OutputFileHints; import org.apache.beam.sdk.io.fs.ResolveOptions.StandardResolveOptions; import org.apache.beam.sdk.io.fs.ResourceId; +import org.apache.beam.sdk.options.ValueProvider; import org.apache.beam.sdk.options.ValueProvider.StaticValueProvider; import org.apache.beam.sdk.testing.NeedsRunner; import org.apache.beam.sdk.testing.PAssert; @@ -214,6 +215,26 @@ public void testAvroIOWriteAndReadAndParseASingleFile() throws Throwable { readPipeline.run(); } + @Test + @Category(NeedsRunner.class) + public void testAvroIOWriteAndReadViaValueProvider() throws Throwable { + List values = + ImmutableList.of(new GenericClass(3, "hi"), new GenericClass(5, "bar")); + File outputFile = tmpFolder.newFile("output.avro"); + + ValueProvider pathProvider = StaticValueProvider.of(outputFile.getAbsolutePath()); + + writePipeline + .apply(Create.of(values)) + .apply(AvroIO.write(GenericClass.class).to(pathProvider).withoutSharding()); + writePipeline.run().waitUntilFinish(); + + PAssert.that(readPipeline.apply("Read", AvroIO.read(GenericClass.class).from(pathProvider))) + .containsInAnyOrder(values); + + readPipeline.run(); + } + @Test @Category(NeedsRunner.class) public void testAvroIOWriteAndReadMultipleFilepatterns() throws Throwable { From b3f7e2181ef32579646381573f9d147e0220d0d7 Mon Sep 17 00:00:00 2001 From: Luke Cwik Date: Thu, 17 Aug 2017 11:49:35 -0700 Subject: [PATCH 319/346] [BEAM-1347] Provide an abstraction which creates an Iterator view over the Beam Fn State API Combining this with the DataStreams.DataStreamDecoder converts the Beam Fn State API into a an input stream backed by multiple logical chunks. --- .../fn/harness/state/BeamFnStateClient.java | 16 ++- .../harness/state/StateFetchingIterators.java | 126 ++++++++++++++++++ .../beam/fn/harness/stream/DataStreams.java | 2 +- .../state/StateFetchingIteratorsTest.java | 99 ++++++++++++++ 4 files changed, 241 insertions(+), 2 deletions(-) create mode 100644 sdks/java/harness/src/main/java/org/apache/beam/fn/harness/state/StateFetchingIterators.java create mode 100644 sdks/java/harness/src/test/java/org/apache/beam/fn/harness/state/StateFetchingIteratorsTest.java diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/state/BeamFnStateClient.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/state/BeamFnStateClient.java index 81505308f6da7..682adb9fb6900 100644 --- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/state/BeamFnStateClient.java +++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/state/BeamFnStateClient.java @@ -17,9 +17,23 @@ */ package org.apache.beam.fn.harness.state; +import java.util.concurrent.CompletableFuture; +import org.apache.beam.fn.v1.BeamFnApi; +import org.apache.beam.fn.v1.BeamFnApi.StateResponse; + /** - * TODO: Define interface required for handling state calls. + * The {@link BeamFnStateClient} is able to forward state requests to a handler which returns + * a corresponding response or error if completed unsuccessfully. */ public interface BeamFnStateClient { + /** + * Consumes a state request populating a unique id returning a future to the response. + * + * @param requestBuilder A partially completed state request. The id will be populated the client. + * @param response A future containing a corresponding {@link StateResponse} for the supplied + * request. + */ + void handle(BeamFnApi.StateRequest.Builder requestBuilder, + CompletableFuture response); } diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/state/StateFetchingIterators.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/state/StateFetchingIterators.java new file mode 100644 index 0000000000000..0526183be173f --- /dev/null +++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/state/StateFetchingIterators.java @@ -0,0 +1,126 @@ +package org.apache.beam.fn.harness.state; + +import com.google.common.base.Throwables; +import com.google.protobuf.ByteString; +import java.util.Iterator; +import java.util.NoSuchElementException; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutionException; +import java.util.function.Supplier; +import org.apache.beam.fn.v1.BeamFnApi.StateGetRequest; +import org.apache.beam.fn.v1.BeamFnApi.StateRequest; +import org.apache.beam.fn.v1.BeamFnApi.StateRequest.Builder; +/* + * 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 org.apache.beam.fn.v1.BeamFnApi.StateResponse; + +/** + * Adapters which convert a a logical series of chunks using continuation tokens over the Beam + * Fn State API into an {@link Iterator} of {@link ByteString}s. + */ +public class StateFetchingIterators { + + // do not instantiate + private StateFetchingIterators() {} + + /** + * This adapter handles using the continuation token to provide iteration over all the chunks + * returned by the Beam Fn State API using the supplied state client and partially filled + * out state request containing a state key. + * + * @param beamFnStateClient A client for handling state requests. + * @param partialStateRequestBuilder A {@link StateRequest} with the + * {@link StateRequest#getStateKey()} already set. + * @return An {@code Iterator} representing all the requested data. + */ + public static Iterator usingPartialRequestWithStateKey( + BeamFnStateClient beamFnStateClient, + Supplier partialStateRequestBuilder) { + return new LazyBlockingStateFetchingIterator(beamFnStateClient, partialStateRequestBuilder); + } + + /** + * An {@link Iterator} which fetches {@link ByteString} chunks using the State API. + * + *

              This iterator will only request a chunk on first access. Also it does not eagerly + * pre-fetch any future chunks and blocks whenever required to fetch the next block. + */ + static class LazyBlockingStateFetchingIterator implements Iterator { + private enum State { READ_REQUIRED, HAS_NEXT, EOF }; + private final BeamFnStateClient beamFnStateClient; + /** Allows for the partially built state request to be memoized across many requests. */ + private final Supplier stateRequestSupplier; + private State currentState; + private ByteString continuationToken; + private ByteString next; + + LazyBlockingStateFetchingIterator( + BeamFnStateClient beamFnStateClient, + Supplier stateRequestSupplier) { + this.currentState = State.READ_REQUIRED; + this.beamFnStateClient = beamFnStateClient; + this.stateRequestSupplier = stateRequestSupplier; + this.continuationToken = ByteString.EMPTY; + } + + @Override + public boolean hasNext() { + switch (currentState) { + case EOF: + return false; + case READ_REQUIRED: + CompletableFuture stateResponseFuture = new CompletableFuture<>(); + beamFnStateClient.handle( + stateRequestSupplier.get().setGet( + StateGetRequest.newBuilder().setContinuationToken(continuationToken)), + stateResponseFuture); + StateResponse stateResponse; + try { + stateResponse = stateResponseFuture.get(); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new IllegalStateException(e); + } catch (ExecutionException e) { + if (e.getCause() == null) { + throw new IllegalStateException(e); + } + Throwables.throwIfUnchecked(e.getCause()); + throw new IllegalStateException(e.getCause()); + } + continuationToken = stateResponse.getGet().getContinuationToken(); + next = stateResponse.getGet().getData(); + currentState = State.HAS_NEXT; + return true; + case HAS_NEXT: + return true; + } + throw new IllegalStateException(String.format("Unknown state %s", currentState)); + } + + @Override + public ByteString next() { + if (!hasNext()) { + throw new NoSuchElementException(); + } + // If the continuation token is empty, that means we have reached EOF. + currentState = ByteString.EMPTY.equals(continuationToken) ? State.EOF : State.READ_REQUIRED; + return next; + } + } + +} diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/stream/DataStreams.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/stream/DataStreams.java index 69671601fbc8c..3ecd303721817 100644 --- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/stream/DataStreams.java +++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/stream/DataStreams.java @@ -153,7 +153,7 @@ public boolean hasNext() { } catch (IOException e) { throw new IllegalStateException(e); } - // fall through expected + return true; case HAS_NEXT: return true; } diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/state/StateFetchingIteratorsTest.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/state/StateFetchingIteratorsTest.java new file mode 100644 index 0000000000000..67e36e1a26855 --- /dev/null +++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/state/StateFetchingIteratorsTest.java @@ -0,0 +1,99 @@ +/* + * 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.fn.harness.state; + +import static org.junit.Assert.assertArrayEquals; + +import com.google.common.collect.Iterators; +import com.google.protobuf.ByteString; +import java.util.Iterator; +import java.util.concurrent.CompletableFuture; +import org.apache.beam.fn.harness.state.StateFetchingIterators.LazyBlockingStateFetchingIterator; +import org.apache.beam.fn.v1.BeamFnApi.StateGetResponse; +import org.apache.beam.fn.v1.BeamFnApi.StateRequest; +import org.apache.beam.fn.v1.BeamFnApi.StateResponse; +import org.junit.Test; +import org.junit.experimental.runners.Enclosed; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +/** Tests for {@link StateFetchingIterators}. */ +@RunWith(Enclosed.class) +public class StateFetchingIteratorsTest { + /** Tests for {@link StateFetchingIterators.LazyBlockingStateFetchingIterator}. */ + @RunWith(JUnit4.class) + public static class LazyBlockingStateFetchingIteratorTest { + + @Test + public void testEmpty() throws Exception { + testFetch(ByteString.EMPTY); + } + + @Test + public void testNonEmpty() throws Exception { + testFetch(ByteString.copyFromUtf8("A")); + } + + @Test + public void testWithLastByteStringBeingEmpty() throws Exception { + testFetch(ByteString.copyFromUtf8("A"), ByteString.EMPTY); + } + + @Test + public void testMulti() throws Exception { + testFetch(ByteString.copyFromUtf8("BC"), ByteString.copyFromUtf8("DEF")); + } + + @Test + public void testMultiWithEmptyByteStrings() throws Exception { + testFetch(ByteString.EMPTY, ByteString.copyFromUtf8("BC"), ByteString.EMPTY, + ByteString.EMPTY, ByteString.copyFromUtf8("DEF"), ByteString.EMPTY); + } + + private void testFetch(ByteString... expected) { + BeamFnStateClient fakeStateClient = new BeamFnStateClient() { + @Override + public void handle( + StateRequest.Builder requestBuilder, CompletableFuture response) { + ByteString continuationToken = requestBuilder.getGet().getContinuationToken(); + StateGetResponse.Builder builder = StateGetResponse.newBuilder(); + + int requestedPosition = 0; // Default position is 0 + if (!ByteString.EMPTY.equals(continuationToken)) { + requestedPosition = Integer.parseInt(continuationToken.toStringUtf8()); + } + + // Compute the new continuation token + ByteString newContinuationToken = ByteString.EMPTY; + if (requestedPosition != expected.length - 1) { + newContinuationToken = ByteString.copyFromUtf8(Integer.toString(requestedPosition + 1)); + } + response.complete(StateResponse.newBuilder() + .setId(requestBuilder.getId()) + .setGet(StateGetResponse.newBuilder() + .setData(expected[requestedPosition]) + .setContinuationToken(newContinuationToken)) + .build()); + } + }; + Iterator byteStrings = + new LazyBlockingStateFetchingIterator(fakeStateClient, StateRequest::newBuilder); + assertArrayEquals(expected, Iterators.toArray(byteStrings, Object.class)); + } + } +} From 15472b28c649381b90a0405d80012aa8523d13c5 Mon Sep 17 00:00:00 2001 From: Stas Levin Date: Sun, 20 Aug 2017 16:48:57 +0300 Subject: [PATCH 320/346] [BEAM-2671] Implemented an InputDStream that syncs up with the watermark values, this should help with streaming tests in spark-runner. --- .../beam/runners/spark/SparkRunner.java | 5 +- .../beam/runners/spark/io/CreateStream.java | 104 ++++-- .../SparkGroupAlsoByWindowViaWindowSet.java | 158 ++++++--- .../spark/stateful/SparkTimerInternals.java | 6 + .../StreamingTransformTranslator.java | 71 ++-- .../streaming/WatermarkSyncedDStream.java | 149 +++++++++ .../spark/util/GlobalWatermarkHolder.java | 302 ++++++++++++------ .../runners/spark/SparkPipelineStateTest.java | 4 +- .../streaming/CreateStreamTest.java | 33 +- .../spark/src/test/resources/log4j.properties | 11 +- 10 files changed, 633 insertions(+), 210 deletions(-) create mode 100644 runners/spark/src/main/java/org/apache/beam/runners/spark/translation/streaming/WatermarkSyncedDStream.java 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 595521fd9ff59..98ca1beb6ab2a 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 @@ -40,7 +40,7 @@ import org.apache.beam.runners.spark.translation.TransformTranslator; import org.apache.beam.runners.spark.translation.streaming.Checkpoint.CheckpointDir; import org.apache.beam.runners.spark.translation.streaming.SparkRunnerStreamingContextFactory; -import org.apache.beam.runners.spark.util.GlobalWatermarkHolder.WatermarksListener; +import org.apache.beam.runners.spark.util.GlobalWatermarkHolder.WatermarkAdvancingStreamingListener; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.PipelineRunner; import org.apache.beam.sdk.io.Read; @@ -171,7 +171,8 @@ public SparkPipelineResult run(final Pipeline pipeline) { } // register Watermarks listener to broadcast the advanced WMs. - jssc.addStreamingListener(new JavaStreamingListenerWrapper(new WatermarksListener())); + jssc.addStreamingListener( + new JavaStreamingListenerWrapper(new WatermarkAdvancingStreamingListener())); // The reason we call initAccumulators here even though it is called in // SparkRunnerStreamingContextFactory is because the factory is not called when resuming 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 d485d25b019f7..4c73d95239321 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 @@ -41,34 +41,34 @@ /** * Create an input stream from Queue. For SparkRunner tests only. * - *

              To properly compose a stream of micro-batches with their Watermarks, please keep in mind - * that eventually there a two queues here - one for batches and another for Watermarks. + *

              To properly compose a stream of micro-batches with their Watermarks, please keep in mind that + * eventually there a two queues here - one for batches and another for Watermarks. * - *

              While both queues advance according to Spark's batch-interval, there is a slight difference - * in how data is pushed into the stream compared to the advancement of Watermarks since Watermarks + *

              While both queues advance according to Spark's batch-interval, there is a slight difference in + * how data is pushed into the stream compared to the advancement of Watermarks since Watermarks * advance onBatchCompleted hook call so if you'd want to set the watermark advance for a specific - * batch it should be called before that batch. - * Also keep in mind that being a queue that is polled per batch interval, if there is a need to - * "hold" the same Watermark without advancing it it should be stated explicitly or the Watermark - * will advance as soon as it can (in the next batch completed hook). + * batch it should be called before that batch. Also keep in mind that being a queue that is polled + * per batch interval, if there is a need to "hold" the same Watermark without advancing it it + * should be stated explicitly or the Watermark will advance as soon as it can (in the next batch + * completed hook). * *

              Example 1: * - * {@code - * CreateStream.>withBatchInterval(batchDuration) - * .nextBatch( - * TimestampedValue.of("foo", endOfGlobalWindow), - * TimestampedValue.of("bar", endOfGlobalWindow)) - * .advanceNextBatchWatermarkToInfinity(); - * } - * The first batch will see the default start-of-time WM of - * {@link BoundedWindow#TIMESTAMP_MIN_VALUE} and any following batch will see - * the end-of-time WM {@link BoundedWindow#TIMESTAMP_MAX_VALUE}. + *

              {@code
              + * CreateStream.of(StringUtf8Coder.of(), batchDuration)
              + *   .nextBatch(
              + *     TimestampedValue.of("foo", endOfGlobalWindow),
              + *     TimestampedValue.of("bar", endOfGlobalWindow))
              + *   .advanceNextBatchWatermarkToInfinity();
              + * }
              + * The first batch will see the default start-of-time WM of {@link + * BoundedWindow#TIMESTAMP_MIN_VALUE} and any following batch will see the end-of-time WM {@link + * BoundedWindow#TIMESTAMP_MAX_VALUE}. * *

              Example 2: * - * {@code - * CreateStream.>withBatchInterval(batchDuration) + *

              {@code
              + * CreateStream.of(VarIntCoder.of(), batchDuration)
                *     .nextBatch(
                *         TimestampedValue.of(1, instant))
                *     .advanceWatermarkForNextBatch(instant.plus(Duration.standardMinutes(20)))
              @@ -77,32 +77,59 @@
                *     .nextBatch(
                *         TimestampedValue.of(3, instant))
                *     .advanceWatermarkForNextBatch(instant.plus(Duration.standardMinutes(30)))
              - * }
              - * The first batch will see the start-of-time WM and the second will see the advanced (+20 min.) WM.
              - * The third WM will see the WM advanced to +30 min, because this is the next advancement of the WM
              - * regardless of where it ws called in the construction of CreateStream.
              - * //TODO: write a proper Builder enforcing all those rules mentioned.
              - * @param  stream type.
              + * }
              + * + *

              + * The first batch will see the start-of-time WM and the second will see the advanced (+20 min.) + * WM. The third WM will see the WM advanced to +30 min, because this is the next advancement + * of the WM regardless of where it ws called in the construction of CreateStream. + *

              + * + * @param The type of the element in this stream. */ +//TODO: write a proper Builder enforcing all those rules mentioned. public final class CreateStream extends PTransform> { - private final Duration batchInterval; + private final Duration batchDuration; private final Queue>> batches = new LinkedList<>(); private final Deque times = new LinkedList<>(); private final Coder coder; private Instant initialSystemTime; + private final boolean forceWatermarkSync; private Instant lowWatermark = BoundedWindow.TIMESTAMP_MIN_VALUE; //for test purposes. - private CreateStream(Duration batchInterval, Instant initialSystemTime, Coder coder) { - this.batchInterval = batchInterval; + private CreateStream(Duration batchDuration, + Instant initialSystemTime, + Coder coder, + boolean forceWatermarkSync) { + this.batchDuration = batchDuration; this.initialSystemTime = initialSystemTime; this.coder = coder; + this.forceWatermarkSync = forceWatermarkSync; } - /** Set the batch interval for the stream. */ - public static CreateStream of(Coder coder, Duration batchInterval) { - return new CreateStream<>(batchInterval, new Instant(0), coder); + /** + * Creates a new Spark based stream intended for test purposes. + * + * @param batchDuration the batch duration (interval) to be used for creating this stream. + * @param coder the coder to be used for this stream. + * @param forceWatermarkSync whether this stream should be synced with the advancement of the + * watermark maintained by the + * {@link org.apache.beam.runners.spark.util.GlobalWatermarkHolder}. + */ + public static CreateStream of(Coder coder, + Duration batchDuration, + boolean forceWatermarkSync) { + return new CreateStream<>(batchDuration, new Instant(0), coder, forceWatermarkSync); + } + + /** + * Creates a new Spark based stream without forced watermark sync, intended for test purposes. + * See also {@link CreateStream#of(Coder, Duration, boolean)}. + */ + public static CreateStream of(Coder coder, Duration batchDuration) { + return of(coder, batchDuration, true); } /** @@ -112,8 +139,7 @@ public static CreateStream of(Coder coder, Duration batchInterval) { @SafeVarargs public final CreateStream nextBatch(TimestampedValue... batchElements) { // validate timestamps if timestamped elements. - for (TimestampedValue element: batchElements) { - TimestampedValue timestampedValue = (TimestampedValue) element; + for (final TimestampedValue timestampedValue: batchElements) { checkArgument( timestampedValue.getTimestamp().isBefore(BoundedWindow.TIMESTAMP_MAX_VALUE), "Elements must have timestamps before %s. Got: %s", @@ -177,7 +203,7 @@ private CreateStream advance(Instant newWatermark) { // advance the system time. Instant currentSynchronizedProcessingTime = times.peekLast() == null ? initialSystemTime : times.peekLast().getSynchronizedProcessingTime(); - Instant nextSynchronizedProcessingTime = currentSynchronizedProcessingTime.plus(batchInterval); + Instant nextSynchronizedProcessingTime = currentSynchronizedProcessingTime.plus(batchDuration); checkArgument( nextSynchronizedProcessingTime.isAfter(currentSynchronizedProcessingTime), "Synchronized processing time must always advance."); @@ -186,6 +212,10 @@ private CreateStream advance(Instant newWatermark) { return this; } + public long getBatchDuration() { + return batchDuration.getMillis(); + } + /** Get the underlying queue representing the mock stream of micro-batches. */ public Queue>> getBatches() { return batches; @@ -199,6 +229,10 @@ public Queue getTimes() { return times; } + public boolean isForceWatermarkSync() { + return forceWatermarkSync; + } + @Override public PCollection expand(PBegin input) { return PCollection.createPrimitiveOutputInternal( diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/stateful/SparkGroupAlsoByWindowViaWindowSet.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/stateful/SparkGroupAlsoByWindowViaWindowSet.java index 1263618aa2bfd..52f7376c7414b 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/stateful/SparkGroupAlsoByWindowViaWindowSet.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/stateful/SparkGroupAlsoByWindowViaWindowSet.java @@ -17,12 +17,15 @@ */ package org.apache.beam.runners.spark.stateful; +import com.google.common.base.Joiner; import com.google.common.collect.AbstractIterator; +import com.google.common.collect.Lists; import com.google.common.collect.Table; import java.io.Serializable; import java.util.ArrayList; import java.util.Collection; import java.util.List; +import java.util.Map; import org.apache.beam.runners.core.GroupAlsoByWindowsAggregators; import org.apache.beam.runners.core.GroupByKeyViaGroupByKeyOnly.GroupAlsoByWindow; import org.apache.beam.runners.core.LateDataUtils; @@ -46,6 +49,7 @@ 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.coders.VarLongCoder; import org.apache.beam.sdk.metrics.MetricName; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; @@ -61,6 +65,7 @@ import org.apache.spark.api.java.function.FlatMapFunction; import org.apache.spark.api.java.function.Function; import org.apache.spark.streaming.Duration; +import org.apache.spark.streaming.Time; import org.apache.spark.streaming.api.java.JavaDStream; import org.apache.spark.streaming.api.java.JavaPairDStream; import org.apache.spark.streaming.dstream.DStream; @@ -104,12 +109,13 @@ private abstract static class SerializableFunction1 public static JavaDStream>>> groupAlsoByWindow( - final JavaDStream>>>> inputDStream, - final Coder keyCoder, - final Coder> wvCoder, - final WindowingStrategy windowingStrategy, - final SerializablePipelineOptions options, - final List sourceIds) { + final JavaDStream>>>> inputDStream, + final Coder keyCoder, + final Coder> wvCoder, + final WindowingStrategy windowingStrategy, + final SerializablePipelineOptions options, + final List sourceIds, + final String transformFullName) { final long batchDurationMillis = options.get().as(SparkPipelineOptions.class).getBatchIntervalMillis(); @@ -140,30 +146,44 @@ JavaDStream>>> groupAlsoByWindow( DStream>*/ byte[]>> pairDStream = inputDStream .transformToPair( - new Function< + new org.apache.spark.api.java.function.Function2< JavaRDD>>>>, - JavaPairRDD>() { + Time, JavaPairRDD>() { // we use mapPartitions with the RDD API because its the only available API // that allows to preserve partitioning. @Override public JavaPairRDD call( - JavaRDD>>>> rdd) + JavaRDD>>>> rdd, + final Time time) throws Exception { return rdd.mapPartitions( - TranslationUtils.functionToFlatMapFunction( - WindowingHelpers - .>>>unwindowFunction()), - true) - .mapPartitionsToPair( - TranslationUtils - .>>toPairFlatMapFunction(), - true) - // move to bytes representation and use coders for deserialization - // because of checkpointing. - .mapPartitionsToPair( - TranslationUtils.pairFunctionToPairFlatMapFunction( - CoderHelpers.toByteFunction(keyCoder, itrWvCoder)), - true); + TranslationUtils.functionToFlatMapFunction( + WindowingHelpers + .>>>unwindowFunction()), + true) + .mapPartitionsToPair( + TranslationUtils + .>>toPairFlatMapFunction(), + true) + .mapValues(new Function>, KV>>>() { + + @Override + public KV>> call + (Iterable> values) + throws Exception { + // add the batch timestamp for visibility (e.g., debugging) + return KV.of(time.milliseconds(), values); + } + }) + // move to bytes representation and use coders for deserialization + // because of checkpointing. + .mapPartitionsToPair( + TranslationUtils.pairFunctionToPairFlatMapFunction( + CoderHelpers.toByteFunction(keyCoder, + KvCoder.of(VarLongCoder.of(), + itrWvCoder))), + true); } }) .dstream(); @@ -219,9 +239,10 @@ public JavaPairRDD call( GroupAlsoByWindowsAggregators.DROPPED_DUE_TO_LATENESS_COUNTER)); AbstractIterator< - Tuple2>>*/ List>>> + Tuple2>>>*/ + List>>> outIter = new AbstractIterator>>*/ List>>>() { + Tuple2>>>*/ List>>>() { @Override protected Tuple2>>*/ List>> computeNext() { @@ -240,8 +261,11 @@ public JavaPairRDD call( List>> prevStateAndTimersOpt = next._3(); SparkStateInternals stateInternals; + Map watermarks = + GlobalWatermarkHolder.get(batchDurationMillis); SparkTimerInternals timerInternals = SparkTimerInternals.forStreamFromSources( - sourceIds, GlobalWatermarkHolder.get(batchDurationMillis)); + sourceIds, watermarks); + // get state(internals) per key. if (prevStateAndTimersOpt.isEmpty()) { // no previous state. @@ -271,20 +295,49 @@ public JavaPairRDD call( options.get()); outputHolder.clear(); // clear before potential use. + if (!seq.isEmpty()) { // new input for key. try { - Iterable> elementsIterable = - CoderHelpers.fromByteArray(seq.head(), itrWvCoder); - Iterable> validElements = - LateDataUtils - .dropExpiredWindows( - key, - elementsIterable, - timerInternals, - windowingStrategy, - droppedDueToLateness); - reduceFnRunner.processElements(validElements); + final KV>> keyedElements = + CoderHelpers.fromByteArray(seq.head(), + KvCoder.of(VarLongCoder.of(), itrWvCoder)); + + final Long rddTimestamp = keyedElements.getKey(); + + LOG.debug( + transformFullName + + ": processing RDD with timestamp: {}, watermarks: {}", + rddTimestamp, + watermarks); + + final Iterable> elements = keyedElements.getValue(); + + LOG.trace(transformFullName + ": input elements: {}", elements); + + /* + Incoming expired windows are filtered based on + timerInternals.currentInputWatermarkTime() and the configured allowed + lateness. Note that this is done prior to calling + timerInternals.advanceWatermark so essentially the inputWatermark is + the highWatermark of the previous batch and the lowWatermark of the + current batch. + The highWatermark of the current batch will only affect filtering + as of the next batch. + */ + final Iterable> nonExpiredElements = + Lists.newArrayList(LateDataUtils + .dropExpiredWindows( + key, + elements, + timerInternals, + windowingStrategy, + droppedDueToLateness)); + + LOG.trace(transformFullName + ": non expired input elements: {}", + elements); + + reduceFnRunner.processElements(nonExpiredElements); } catch (Exception e) { throw new RuntimeException( "Failed to process element with ReduceFnRunner", e); @@ -295,9 +348,28 @@ public JavaPairRDD call( } try { // advance the watermark to HWM to fire by timers. + LOG.debug(transformFullName + ": timerInternals before advance are {}", + timerInternals.toString()); + + // store the highWatermark as the new inputWatermark to calculate triggers timerInternals.advanceWatermark(); + + LOG.debug(transformFullName + ": timerInternals after advance are {}", + timerInternals.toString()); + // call on timers that are ready. - reduceFnRunner.onTimers(timerInternals.getTimersReadyToProcess()); + final Collection readyToProcess = + timerInternals.getTimersReadyToProcess(); + + LOG.debug(transformFullName + ": ready timers are {}", readyToProcess); + + /* + Note that at this point, the watermark has already advanced since + timerInternals.advanceWatermark() has been called and the highWatermark + is now stored as the new inputWatermark, according to which triggers are + calculated. + */ + reduceFnRunner.onTimers(readyToProcess); } catch (Exception e) { throw new RuntimeException( "Failed to process ReduceFnRunner onTimer.", e); @@ -306,10 +378,20 @@ public JavaPairRDD call( reduceFnRunner.persist(); // obtain output, if fired. List>>> outputs = outputHolder.get(); + if (!outputs.isEmpty() || !stateInternals.getState().isEmpty()) { + // empty outputs are filtered later using DStream filtering StateAndTimers updated = new StateAndTimers(stateInternals.getState(), SparkTimerInternals.serializeTimers( timerInternals.getTimers(), timerDataCoder)); + + /* + Not something we want to happen in production, but is very helpful + when debugging - TRACE. + */ + LOG.trace(transformFullName + ": output elements are {}", + Joiner.on(", ").join(outputs)); + // persist Spark's state by outputting. List serOutput = CoderHelpers.toByteArrays(outputs, wvKvIterCoder); return new Tuple2<>(encodedKey, new Tuple2<>(updated, serOutput)); diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/stateful/SparkTimerInternals.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/stateful/SparkTimerInternals.java index a68da5516da79..c9983288611d1 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/stateful/SparkTimerInternals.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/stateful/SparkTimerInternals.java @@ -188,4 +188,10 @@ public static Iterable deserializeTimers( return CoderHelpers.fromByteArrays(serTimers, timerDataCoder); } + @Override + public String toString() { + return "SparkTimerInternals{" + "highWatermark=" + highWatermark + + ", synchronizedProcessingTime=" + synchronizedProcessingTime + ", timers=" + timers + + ", inputWatermark=" + inputWatermark + '}'; + } } 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 38d6119b76da9..411480372c93f 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 @@ -82,6 +82,7 @@ import org.apache.spark.api.java.JavaPairRDD; import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.api.java.JavaSparkContext$; import org.apache.spark.api.java.function.Function; import org.apache.spark.streaming.api.java.JavaDStream; import org.apache.spark.streaming.api.java.JavaInputDStream; @@ -139,18 +140,41 @@ private static TransformEvaluator> createFromQueue() { return new TransformEvaluator>() { @Override public void evaluate(CreateStream transform, EvaluationContext context) { - Coder coder = context.getOutput(transform).getCoder(); - JavaStreamingContext jssc = context.getStreamingContext(); - Queue>> values = transform.getBatches(); - WindowedValue.FullWindowedValueCoder windowCoder = + + final Queue>> rddQueue = + buildRdds( + transform.getBatches(), + context.getStreamingContext(), + context.getOutput(transform).getCoder()); + + final JavaInputDStream> javaInputDStream = + buildInputStream(rddQueue, transform, context); + + final UnboundedDataset unboundedDataset = + new UnboundedDataset<>( + javaInputDStream, Collections.singletonList(javaInputDStream.inputDStream().id())); + + // add pre-baked Watermarks for the pre-baked batches. + GlobalWatermarkHolder.addAll( + ImmutableMap.of(unboundedDataset.getStreamSources().get(0), transform.getTimes())); + + context.putDataset(transform, unboundedDataset); + } + + private Queue>> buildRdds( + Queue>> batches, JavaStreamingContext jssc, Coder coder) { + + final WindowedValue.FullWindowedValueCoder windowCoder = WindowedValue.FullWindowedValueCoder.of(coder, GlobalWindow.Coder.INSTANCE); - // create the DStream from queue. - Queue>> rddQueue = new LinkedBlockingQueue<>(); - for (Iterable> tv : values) { - Iterable> windowedValues = + + final Queue>> rddQueue = new LinkedBlockingQueue<>(); + + for (final Iterable> timestampedValues : batches) { + final Iterable> windowedValues = Iterables.transform( - tv, + timestampedValues, new com.google.common.base.Function, WindowedValue>() { + @Override public WindowedValue apply(@Nonnull TimestampedValue timestampedValue) { return WindowedValue.of( @@ -159,22 +183,28 @@ public WindowedValue apply(@Nonnull TimestampedValue timestampedValue) { GlobalWindow.INSTANCE, PaneInfo.NO_FIRING); } - }); - JavaRDD> rdd = + }); + + final JavaRDD> rdd = jssc.sparkContext() .parallelize(CoderHelpers.toByteArrays(windowedValues, windowCoder)) .map(CoderHelpers.fromByteFunction(windowCoder)); + rddQueue.offer(rdd); } + return rddQueue; + } - JavaInputDStream> inputDStream = jssc.queueStream(rddQueue, true); - UnboundedDataset unboundedDataset = new UnboundedDataset( - inputDStream, Collections.singletonList(inputDStream.inputDStream().id())); - // add pre-baked Watermarks for the pre-baked batches. - Queue times = transform.getTimes(); - GlobalWatermarkHolder.addAll( - ImmutableMap.of(unboundedDataset.getStreamSources().get(0), times)); - context.putDataset(transform, unboundedDataset); + private JavaInputDStream> buildInputStream( + Queue>> rddQueue, + CreateStream transform, + EvaluationContext context) { + return transform.isForceWatermarkSync() + ? new JavaInputDStream<>( + new WatermarkSyncedDStream<>( + rddQueue, transform.getBatchDuration(), context.getStreamingContext().ssc()), + JavaSparkContext$.MODULE$.>fakeClassTag()) + : context.getStreamingContext().queueStream(rddQueue, true); } @Override @@ -301,7 +331,8 @@ public JavaRDD>>>> call( wvCoder, windowingStrategy, context.getSerializableOptions(), - streamSources); + streamSources, + context.getCurrentTransform().getFullName()); context.putDataset(transform, new UnboundedDataset<>(outStream, streamSources)); } diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/streaming/WatermarkSyncedDStream.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/streaming/WatermarkSyncedDStream.java new file mode 100644 index 0000000000000..e2a7b44b7bbd4 --- /dev/null +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/streaming/WatermarkSyncedDStream.java @@ -0,0 +1,149 @@ +/* + * 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.streaming; + +import static com.google.common.base.Preconditions.checkState; +import com.google.common.base.Stopwatch; +import com.google.common.util.concurrent.Uninterruptibles; +import java.util.Queue; +import java.util.concurrent.TimeUnit; +import org.apache.beam.runners.spark.util.GlobalWatermarkHolder; +import org.apache.beam.sdk.util.WindowedValue; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.JavaSparkContext$; +import org.apache.spark.rdd.RDD; +import org.apache.spark.streaming.StreamingContext; +import org.apache.spark.streaming.Time; +import org.apache.spark.streaming.dstream.InputDStream; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * An {@link InputDStream} that keeps track of the {@link GlobalWatermarkHolder} status and only + * generates RDDs when they are in sync. If an RDD for time CURRENT_BATCH_TIME is + * requested, this input source will wait until the time of the batch which set the watermark has + * caught up and the following holds: + * + * {@code + * CURRENT_BATCH_TIME - TIME_OF_BATCH_WHICH_SET_THE_WATERMARK <= BATCH_DURATION + * } + * + *

              In other words, this input source will stall and will NOT generate RDDs when the watermark is + * too far behind. Once the watermark has caught up with the current batch time, an RDD will be + * generated and emitted downstream. + * + *

              NOTE: This input source is intended for test-use only, where one needs to be able to simulate + * non-trivial scenarios under a deterministic execution even at the cost incorporating test-only + * code. Unlike tests, in production InputDStreams will not be synchronous with the + * watermark, and the watermark is allowed to lag behind in a non-deterministic manner (since at + * this point in time we are reluctant to apply complex and possibly overly synchronous mechanisms + * at large scale). + * + *

              See also BEAM-2671, BEAM-2789. + */ +class WatermarkSyncedDStream extends InputDStream> { + + private static final Logger LOG = + LoggerFactory.getLogger(WatermarkSyncedDStream.class.getCanonicalName() + "#compute"); + + private static final int SLEEP_DURATION_MILLIS = 10; + + private final Queue>> rdds; + private final Long batchDuration; + private volatile boolean isFirst = true; + + public WatermarkSyncedDStream(final Queue>> rdds, + final Long batchDuration, + final StreamingContext ssc) { + super(ssc, JavaSparkContext$.MODULE$.>fakeClassTag()); + this.rdds = rdds; + this.batchDuration = batchDuration; + } + + private void awaitWatermarkSyncWith(final long batchTime) { + while (!isFirstBatch() && watermarkOutOfSync(batchTime)) { + Uninterruptibles.sleepUninterruptibly(SLEEP_DURATION_MILLIS, TimeUnit.MILLISECONDS); + } + + checkState( + isFirstBatch() || watermarkIsOneBatchBehind(batchTime), + String.format( + "Watermark batch time:[%d] should be exactly one batch behind current batch time:[%d]", + GlobalWatermarkHolder.getLastWatermarkedBatchTime(), batchTime)); + } + + private boolean watermarkOutOfSync(final long batchTime) { + return batchTime - GlobalWatermarkHolder.getLastWatermarkedBatchTime() > batchDuration; + } + + private boolean isFirstBatch() { + return isFirst; + } + + private RDD> generateRdd() { + return rdds.size() > 0 + ? rdds.poll().rdd() + : ssc().sparkContext().emptyRDD(JavaSparkContext$.MODULE$.>fakeClassTag()); + } + + private boolean watermarkIsOneBatchBehind(final long batchTime) { + return GlobalWatermarkHolder.getLastWatermarkedBatchTime() == batchTime - batchDuration; + } + + @Override + public scala.Option>> compute(final Time validTime) { + final long batchTime = validTime.milliseconds(); + + LOG.trace("BEFORE waiting for watermark sync, " + + "LastWatermarkedBatchTime: {}, current batch time: {}", + GlobalWatermarkHolder.getLastWatermarkedBatchTime(), + batchTime); + + final Stopwatch stopwatch = Stopwatch.createStarted(); + + awaitWatermarkSyncWith(batchTime); + + stopwatch.stop(); + + LOG.info("Waited {} millis for watermarks to sync up with the current batch ({})", + stopwatch.elapsed(TimeUnit.MILLISECONDS), + batchTime); + + LOG.info("Watermarks are now: {}", GlobalWatermarkHolder.get(batchDuration)); + + LOG.trace("AFTER waiting for watermark sync, " + + "LastWatermarkedBatchTime: {}, current batch time: {}", + GlobalWatermarkHolder.getLastWatermarkedBatchTime(), + batchTime); + + final RDD> rdd = generateRdd(); + isFirst = false; + return scala.Option.apply(rdd); + } + + @Override + public void start() { + + } + + @Override + public void stop() { + + } +} diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/util/GlobalWatermarkHolder.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/util/GlobalWatermarkHolder.java index 2cb6f26f8a0f4..8ad3ca4153360 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/util/GlobalWatermarkHolder.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/util/GlobalWatermarkHolder.java @@ -41,9 +41,12 @@ import org.apache.spark.storage.BlockResult; import org.apache.spark.storage.BlockStore; import org.apache.spark.storage.StorageLevel; +import org.apache.spark.streaming.api.java.JavaBatchInfo; import org.apache.spark.streaming.api.java.JavaStreamingListener; import org.apache.spark.streaming.api.java.JavaStreamingListenerBatchCompleted; import org.joda.time.Instant; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import scala.Option; /** @@ -53,11 +56,18 @@ * and advances the watermarks according to the queue (first-in-first-out). */ public class GlobalWatermarkHolder { + + private static final Logger LOG = LoggerFactory.getLogger(GlobalWatermarkHolder.class); + private static final Map> sourceTimes = new HashMap<>(); private static final BlockId WATERMARKS_BLOCK_ID = BlockId.apply("broadcast_0WATERMARKS"); - private static volatile Map driverWatermarks = null; + // a local copy of the watermarks is stored on the driver node so that it can be + // accessed in test mode instead of fetching blocks remotely + private static volatile Map driverNodeWatermarks = null; + private static volatile LoadingCache> watermarkCache = null; + private static volatile long lastWatermarkedBatchTime = 0; public static void add(int sourceId, SparkWatermarks sparkWatermarks) { Queue timesQueue = sourceTimes.get(sourceId); @@ -79,18 +89,33 @@ public static void addAll(Map> sourceTimes) { } } + public static long getLastWatermarkedBatchTime() { + return lastWatermarkedBatchTime; + } + /** * Returns the {@link Broadcast} containing the {@link SparkWatermarks} mapped * to their sources. */ @SuppressWarnings("unchecked") public static Map get(Long cacheInterval) { - if (driverWatermarks != null) { - // if we are executing in local mode simply return the local values. - return driverWatermarks; + if (canBypassRemoteWatermarkFetching()) { + /* + driverNodeWatermarks != null => + => advance() was called + => WatermarkAdvancingStreamingListener#onBatchCompleted() was called + => we are currently running on the driver node + => we can get the watermarks from the driver local copy instead of fetching their block + remotely using block manger + /------------------------------------------------------------------------------------------/ + In test mode, the system is running inside a single JVM, and thus both driver and executors + "canBypassWatermarkBlockFetching" by using the static driverNodeWatermarks copy. + This allows tests to avoid the asynchronous nature of using the BlockManager directly. + */ + return getLocalWatermarkCopy(); } else { if (watermarkCache == null) { - initWatermarkCache(cacheInterval); + watermarkCache = createWatermarkCache(cacheInterval); } try { return watermarkCache.get("SINGLETON"); @@ -100,103 +125,178 @@ public static Map get(Long cacheInterval) { } } - private static synchronized void initWatermarkCache(Long batchDuration) { - if (watermarkCache == null) { - watermarkCache = - CacheBuilder.newBuilder() - // expire watermarks every half batch duration to ensure they update in every batch. - .expireAfterWrite(batchDuration / 2, TimeUnit.MILLISECONDS) - .build(new WatermarksLoader()); - } + private static boolean canBypassRemoteWatermarkFetching() { + return driverNodeWatermarks != null; + } + + private static synchronized LoadingCache> + createWatermarkCache(final Long batchDuration) { + return CacheBuilder.newBuilder() + // expire watermarks every half batch duration to ensure they update in every batch. + .expireAfterWrite(batchDuration / 2, TimeUnit.MILLISECONDS) + .build(new WatermarksLoader()); } /** * Advances the watermarks to the next-in-line watermarks. * SparkWatermarks are monotonically increasing. */ - @SuppressWarnings("unchecked") - public static void advance() { + public static void advance(final String batchId) { synchronized (GlobalWatermarkHolder.class) { - BlockManager blockManager = SparkEnv.get().blockManager(); + final BlockManager blockManager = SparkEnv.get().blockManager(); + final Map newWatermarks = computeNewWatermarks(blockManager); - if (sourceTimes.isEmpty()) { - return; + if (!newWatermarks.isEmpty()) { + writeRemoteWatermarkBlock(newWatermarks, blockManager); + writeLocalWatermarkCopy(newWatermarks); + } else { + LOG.info("No new watermarks could be computed upon completion of batch: {}", batchId); } + } + } + + private static void writeLocalWatermarkCopy(Map newWatermarks) { + driverNodeWatermarks = newWatermarks; + } - // update all sources' watermarks into the new broadcast. - Map newValues = new HashMap<>(); - - for (Map.Entry> en: sourceTimes.entrySet()) { - if (en.getValue().isEmpty()) { - continue; - } - Integer sourceId = en.getKey(); - Queue timesQueue = en.getValue(); - - // current state, if exists. - Instant currentLowWatermark = BoundedWindow.TIMESTAMP_MIN_VALUE; - Instant currentHighWatermark = BoundedWindow.TIMESTAMP_MIN_VALUE; - Instant currentSynchronizedProcessingTime = BoundedWindow.TIMESTAMP_MIN_VALUE; - - Option currentOption = blockManager.getRemote(WATERMARKS_BLOCK_ID); - Map current; - if (currentOption.isDefined()) { - current = (Map) currentOption.get().data().next(); - } else { - current = Maps.newHashMap(); - blockManager.putSingle( - WATERMARKS_BLOCK_ID, - current, - StorageLevel.MEMORY_ONLY(), - true); - } - - if (current.containsKey(sourceId)) { - SparkWatermarks currentTimes = current.get(sourceId); - currentLowWatermark = currentTimes.getLowWatermark(); - currentHighWatermark = currentTimes.getHighWatermark(); - currentSynchronizedProcessingTime = currentTimes.getSynchronizedProcessingTime(); - } - - SparkWatermarks next = timesQueue.poll(); - // advance watermarks monotonically. - Instant nextLowWatermark = next.getLowWatermark().isAfter(currentLowWatermark) - ? next.getLowWatermark() : currentLowWatermark; - Instant nextHighWatermark = next.getHighWatermark().isAfter(currentHighWatermark) - ? next.getHighWatermark() : currentHighWatermark; - Instant nextSynchronizedProcessingTime = next.getSynchronizedProcessingTime(); - checkState(!nextLowWatermark.isAfter(nextHighWatermark), - String.format( - "Low watermark %s cannot be later then high watermark %s", - nextLowWatermark, nextHighWatermark)); - checkState(nextSynchronizedProcessingTime.isAfter(currentSynchronizedProcessingTime), - "Synchronized processing time must advance."); - newValues.put( - sourceId, - new SparkWatermarks( - nextLowWatermark, nextHighWatermark, nextSynchronizedProcessingTime)); + private static Map getLocalWatermarkCopy() { + return driverNodeWatermarks; + } + + /** See {@link GlobalWatermarkHolder#advance(String)}. */ + public static void advance() { + advance("N/A"); + } + + /** + * Computes the next watermark values per source id. + * + * @return The new watermarks values or null if no source has reported its progress. + */ + private static Map computeNewWatermarks(BlockManager blockManager) { + + if (sourceTimes.isEmpty()) { + return new HashMap<>(); + } + + // update all sources' watermarks into the new broadcast. + final Map newValues = new HashMap<>(); + + for (final Map.Entry> watermarkInfo: sourceTimes.entrySet()) { + + if (watermarkInfo.getValue().isEmpty()) { + continue; } - // update the watermarks broadcast only if something has changed. - if (!newValues.isEmpty()) { - driverWatermarks = newValues; - blockManager.removeBlock(WATERMARKS_BLOCK_ID, true); - blockManager.putSingle( - WATERMARKS_BLOCK_ID, - newValues, - StorageLevel.MEMORY_ONLY(), - true); + final Integer sourceId = watermarkInfo.getKey(); + + // current state, if exists. + Instant currentLowWatermark = BoundedWindow.TIMESTAMP_MIN_VALUE; + Instant currentHighWatermark = BoundedWindow.TIMESTAMP_MIN_VALUE; + Instant currentSynchronizedProcessingTime = BoundedWindow.TIMESTAMP_MIN_VALUE; + + final Map currentWatermarks = initWatermarks(blockManager); + + if (currentWatermarks.containsKey(sourceId)) { + final SparkWatermarks currentTimes = currentWatermarks.get(sourceId); + currentLowWatermark = currentTimes.getLowWatermark(); + currentHighWatermark = currentTimes.getHighWatermark(); + currentSynchronizedProcessingTime = currentTimes.getSynchronizedProcessingTime(); } + + final Queue timesQueue = watermarkInfo.getValue(); + final SparkWatermarks next = timesQueue.poll(); + + // advance watermarks monotonically. + + final Instant nextLowWatermark = + next.getLowWatermark().isAfter(currentLowWatermark) + ? next.getLowWatermark() + : currentLowWatermark; + + final Instant nextHighWatermark = + next.getHighWatermark().isAfter(currentHighWatermark) + ? next.getHighWatermark() + : currentHighWatermark; + + final Instant nextSynchronizedProcessingTime = next.getSynchronizedProcessingTime(); + + checkState( + !nextLowWatermark.isAfter(nextHighWatermark), + String.format( + "Low watermark %s cannot be later then high watermark %s", + nextLowWatermark, nextHighWatermark)); + + checkState( + nextSynchronizedProcessingTime.isAfter(currentSynchronizedProcessingTime), + "Synchronized processing time must advance."); + + newValues.put( + sourceId, + new SparkWatermarks( + nextLowWatermark, nextHighWatermark, nextSynchronizedProcessingTime)); + } + + return newValues; + } + + private static void writeRemoteWatermarkBlock( + final Map newWatermarks, final BlockManager blockManager) { + blockManager.removeBlock(WATERMARKS_BLOCK_ID, true); + // if an executor tries to fetch the watermark block here, it will fail to do so since + // the watermark block has just been removed, but the new copy has not been put yet. + blockManager.putSingle(WATERMARKS_BLOCK_ID, newWatermarks, StorageLevel.MEMORY_ONLY(), true); + // if an executor tries to fetch the watermark block here, it still may fail to do so since + // the put operation might not have been executed yet + // see also https://issues.apache.org/jira/browse/BEAM-2789 + LOG.info("Put new watermark block: {}", newWatermarks); + } + + private static Map initWatermarks(final BlockManager blockManager) { + + final Map watermarks = fetchSparkWatermarks(blockManager); + + if (watermarks == null) { + final HashMap empty = Maps.newHashMap(); + blockManager.putSingle( + WATERMARKS_BLOCK_ID, + empty, + StorageLevel.MEMORY_ONLY(), + true); + return empty; + } else { + return watermarks; + } + } + + private static Map fetchSparkWatermarks(BlockManager blockManager) { + final Option blockResultOption = blockManager.getRemote(WATERMARKS_BLOCK_ID); + if (blockResultOption.isDefined()) { + return (Map) blockResultOption.get().data().next(); + } else { + return null; + } + } + + private static class WatermarksLoader extends CacheLoader> { + + @SuppressWarnings("unchecked") + @Override + public Map load(@Nonnull String key) throws Exception { + final BlockManager blockManager = SparkEnv.get().blockManager(); + final Map watermarks = fetchSparkWatermarks(blockManager); + return watermarks != null ? watermarks : Maps.newHashMap(); } } @VisibleForTesting public static synchronized void clear() { sourceTimes.clear(); - driverWatermarks = null; - SparkEnv sparkEnv = SparkEnv.get(); + lastWatermarkedBatchTime = 0; + writeLocalWatermarkCopy(null); + final SparkEnv sparkEnv = SparkEnv.get(); if (sparkEnv != null) { - BlockManager blockManager = sparkEnv.blockManager(); + final BlockManager blockManager = sparkEnv.blockManager(); blockManager.removeBlock(WATERMARKS_BLOCK_ID, true); } } @@ -242,25 +342,33 @@ public String toString() { } /** Advance the WMs onBatchCompleted event. */ - public static class WatermarksListener extends JavaStreamingListener { - @Override - public void onBatchCompleted(JavaStreamingListenerBatchCompleted batchCompleted) { - GlobalWatermarkHolder.advance(); + public static class WatermarkAdvancingStreamingListener extends JavaStreamingListener { + private static final Logger LOG = + LoggerFactory.getLogger(WatermarkAdvancingStreamingListener.class); + + private long timeOf(JavaBatchInfo info) { + return info.batchTime().milliseconds(); } - } - private static class WatermarksLoader extends CacheLoader> { + private long laterOf(long t1, long t2) { + return Math.max(t1, t2); + } - @SuppressWarnings("unchecked") @Override - public Map load(@Nonnull String key) throws Exception { - Option blockResultOption = - SparkEnv.get().blockManager().getRemote(WATERMARKS_BLOCK_ID); - if (blockResultOption.isDefined()) { - return (Map) blockResultOption.get().data().next(); - } else { - return Maps.newHashMap(); - } + public void onBatchCompleted(JavaStreamingListenerBatchCompleted batchCompleted) { + + final long currentBatchTime = timeOf(batchCompleted.batchInfo()); + + GlobalWatermarkHolder.advance(Long.toString(currentBatchTime)); + + // make sure to update the last watermarked batch time AFTER the watermarks have already + // been updated (i.e., after the call to GlobalWatermarkHolder.advance(...)) + // in addition, the watermark's block in the BlockManager is updated in an asynchronous manner + lastWatermarkedBatchTime = + laterOf(lastWatermarkedBatchTime, currentBatchTime); + + LOG.info("Batch with timestamp: {} has completed, watermarks have been updated.", + lastWatermarkedBatchTime); } } } 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 cfbad01952e7b..a5455da671709 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 @@ -73,8 +73,10 @@ public void processElement(final ProcessContext c) { } private PTransform> getValues(final SparkPipelineOptions options) { + final boolean doNotSyncWithWatermark = false; return options.isStreaming() - ? CreateStream.of(StringUtf8Coder.of(), Duration.millis(1)).nextBatch("one", "two") + ? CreateStream.of(StringUtf8Coder.of(), Duration.millis(1), doNotSyncWithWatermark) + .nextBatch("one", "two") : Create.of("one", "two"); } diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/CreateStreamTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/CreateStreamTest.java index 770e0c054a920..a432fda2e6801 100644 --- a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/CreateStreamTest.java +++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/streaming/CreateStreamTest.java @@ -163,16 +163,16 @@ public Void apply(Iterable input) { public void testDiscardingMode() throws IOException { CreateStream source = CreateStream.of(StringUtf8Coder.of(), batchDuration()) - .nextBatch( - TimestampedValue.of("firstPane", new Instant(100)), - TimestampedValue.of("alsoFirstPane", new Instant(200))) - .advanceWatermarkForNextBatch(new Instant(1001L)) - .nextBatch( - TimestampedValue.of("onTimePane", new Instant(500))) - .advanceNextBatchWatermarkToInfinity() - .nextBatch( - TimestampedValue.of("finalLatePane", new Instant(750)), - TimestampedValue.of("alsoFinalLatePane", new Instant(250))); + .nextBatch( + TimestampedValue.of("firstPane", new Instant(100)), + TimestampedValue.of("alsoFirstPane", new Instant(200))) + .advanceWatermarkForNextBatch(new Instant(1001L)) + .nextBatch( + TimestampedValue.of("onTimePane", new Instant(500))) + .advanceNextBatchWatermarkToInfinity() + .nextBatch( + TimestampedValue.of("finalLatePane", new Instant(750)), + TimestampedValue.of("alsoFinalLatePane", new Instant(250))); FixedWindows windowFn = FixedWindows.of(Duration.millis(1000L)); Duration allowedLateness = Duration.millis(5000L); @@ -212,12 +212,13 @@ public void testFirstElementLate() throws IOException { Instant lateElementTimestamp = new Instant(-1_000_000); CreateStream source = CreateStream.of(StringUtf8Coder.of(), batchDuration()) - .emptyBatch() - .advanceWatermarkForNextBatch(new Instant(0)) - .nextBatch( - TimestampedValue.of("late", lateElementTimestamp), - TimestampedValue.of("onTime", new Instant(100))) - .advanceNextBatchWatermarkToInfinity(); + .emptyBatch() + .advanceWatermarkForNextBatch(new Instant(0)) + .emptyBatch() + .nextBatch( + TimestampedValue.of("late", lateElementTimestamp), + TimestampedValue.of("onTime", new Instant(100))) + .advanceNextBatchWatermarkToInfinity(); FixedWindows windowFn = FixedWindows.of(Duration.millis(1000L)); Duration allowedLateness = Duration.millis(5000L); diff --git a/runners/spark/src/test/resources/log4j.properties b/runners/spark/src/test/resources/log4j.properties index 66e83c893ca36..010c7df91812a 100644 --- a/runners/spark/src/test/resources/log4j.properties +++ b/runners/spark/src/test/resources/log4j.properties @@ -24,7 +24,16 @@ log4j.rootLogger=ERROR, testlogger log4j.appender.testlogger=org.apache.log4j.ConsoleAppender log4j.appender.testlogger.target = System.err log4j.appender.testlogger.layout=org.apache.log4j.PatternLayout -log4j.appender.testlogger.layout.ConversionPattern=%-4r [%t] %-5p %c %x - %m%n +log4j.appender.testlogger.layout.ConversionPattern=%d [%t] %-5p %c %x - %m%n # TestSparkRunner prints general information abut test pipelines execution. log4j.logger.org.apache.beam.runners.spark.TestSparkRunner=INFO + +# in case of an emergency - uncomment (or better yet, stay calm and uncomment). +#log4j.logger.org.apache.beam=TRACE +#log4j.logger.org.apache.beam.sdk.Pipeline=INFO +#log4j.logger.org.apache.beam.sdk.coders=INFO +#log4j.logger.org.apache.beam.sdk.runners.TransformHierarchy=ERROR +#log4j.logger.org.apache.beam.runners.spark.SparkRunner$Evaluator=ERROR +#log4j.logger.org.apache.beam.runners.spark.translation.streaming.WatermarkSyncedDStream#compute=INFO +#log4j.logger.org.apache.beam.runners.spark.translation.streaming.WatermarkSyncedDStream=ERROR From 185daffa53595b3a6d900252d69132c85013aa4c Mon Sep 17 00:00:00 2001 From: Pablo Date: Tue, 22 Aug 2017 16:53:32 -0700 Subject: [PATCH 321/346] Adding IOTargetName and unittests for CounterName --- sdks/python/apache_beam/utils/counters.py | 58 +++++++++----- .../python/apache_beam/utils/counters_test.py | 78 +++++++++++++++++++ 2 files changed, 115 insertions(+), 21 deletions(-) create mode 100644 sdks/python/apache_beam/utils/counters_test.py diff --git a/sdks/python/apache_beam/utils/counters.py b/sdks/python/apache_beam/utils/counters.py index 5d029dcc03fa1..08685aae27593 100644 --- a/sdks/python/apache_beam/utils/counters.py +++ b/sdks/python/apache_beam/utils/counters.py @@ -23,40 +23,56 @@ For internal use only; no backwards-compatibility guarantees. """ +from collections import namedtuple import threading + from apache_beam.transforms import cy_combiners -class CounterName(object): +# Information identifying the IO being measured by a counter. +IOTargetName = namedtuple('IOTargetName', ['side_input_step_name', + 'side_input_index', + 'original_shuffle_step_name']) + + +def side_input_id(step_name, input_index): + """Create an IOTargetName that identifies the reading of a side input.""" + return IOTargetName(step_name, input_index, None) + + +def shuffle_id(step_name): + """Create an IOTargetName that identifies a GBK step.""" + return IOTargetName(None, None, step_name) + + +_CounterName = namedtuple('_CounterName', ['name', + 'stage_name', + 'step_name', + 'system_name', + 'namespace', + 'origin', + 'output_index', + 'io_target']) + + +class CounterName(_CounterName): """Naming information for a counter.""" SYSTEM = object() USER = object() - def __init__(self, name, stage_name=None, step_name=None, - system_name=None, namespace=None, - origin=None, output_index=None): - self.name = name - self.origin = origin or CounterName.SYSTEM - self.namespace = namespace - self.stage_name = stage_name - self.step_name = step_name - self.system_name = system_name - self.output_index = output_index - - def __hash__(self): - return hash((self.name, - self.origin, - self.namespace, - self.stage_name, - self.step_name, - self.system_name, - self.output_index)) + def __new__(cls, name, stage_name=None, step_name=None, + system_name=None, namespace=None, + origin=None, output_index=None, io_target=None): + origin = origin or CounterName.SYSTEM + return super(CounterName, cls).__new__(cls, name, stage_name, step_name, + system_name, namespace, + origin, output_index, io_target) def __str__(self): return '%s' % self._str_internal() def __repr__(self): - return '<%s at %s>' % (self._str_internal(), hex(id(self))) + return ' at %s>' % (self._str_internal(), hex(id(self))) def _str_internal(self): if self.origin == CounterName.USER: diff --git a/sdks/python/apache_beam/utils/counters_test.py b/sdks/python/apache_beam/utils/counters_test.py new file mode 100644 index 0000000000000..37cab881d8b31 --- /dev/null +++ b/sdks/python/apache_beam/utils/counters_test.py @@ -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. +# + +"""Unit tests for counters and counter names.""" + +from __future__ import absolute_import + +import unittest + +from apache_beam.utils import counters +from apache_beam.utils.counters import CounterName + + +class CounterNameTest(unittest.TestCase): + + def test_equal_objects(self): + self.assertEqual(CounterName('counter_name', + 'stage_name', + 'step_name'), + CounterName('counter_name', + 'stage_name', + 'step_name')) + self.assertNotEqual(CounterName('counter_name', + 'stage_name', + 'step_name'), + CounterName('counter_name', + 'stage_name', + 'step_nam')) + + # Testing objects with an IOTarget. + self.assertEqual(CounterName('counter_name', + 'stage_name', + 'step_name', + io_target=counters.side_input_id(1, 's9')), + CounterName('counter_name', + 'stage_name', + 'step_name', + io_target=counters.side_input_id(1, 's9'))) + self.assertNotEqual(CounterName('counter_name', + 'stage_name', + 'step_name', + io_target=counters.side_input_id(1, 's')), + CounterName('counter_name', + 'stage_name', + 'step_name', + io_target=counters.side_input_id(1, 's9'))) + + def test_hash_two_objects(self): + self.assertEqual(hash(CounterName('counter_name', + 'stage_name', + 'step_name')), + hash(CounterName('counter_name', + 'stage_name', + 'step_name'))) + self.assertNotEqual(hash(CounterName('counter_name', + 'stage_name', + 'step_name')), + hash(CounterName('counter_name', + 'stage_name', + 'step_nam'))) + + +if __name__ == '__main__': + unittest.main() From 885758247d572447dc402a3ef2c45462223e88a2 Mon Sep 17 00:00:00 2001 From: Thomas Groh Date: Thu, 24 Aug 2017 09:39:31 -0700 Subject: [PATCH 322/346] Expand SdkComponents Visibility Make the register, toComponents, and create methods public. --- .../runners/core/construction/SdkComponents.java | 13 +++++++------ 1 file changed, 7 insertions(+), 6 deletions(-) diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/SdkComponents.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/SdkComponents.java index 54d2e9d91f066..c2267dd0e2ecd 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/SdkComponents.java +++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/SdkComponents.java @@ -50,7 +50,7 @@ public class SdkComponents { // TODO: Specify environments /** Create a new {@link SdkComponents} with no components. */ - static SdkComponents create() { + public static SdkComponents create() { return new SdkComponents(); } @@ -69,7 +69,7 @@ private SdkComponents() { * *

              All of the children must already be registered within this {@link SdkComponents}. */ - String registerPTransform( + public String registerPTransform( AppliedPTransform appliedPTransform, List> children) throws IOException { String name = getApplicationName(appliedPTransform); @@ -118,7 +118,7 @@ String getExistingPTransformId(AppliedPTransform appliedPTransform) { * ID for the {@link PCollection}. Multiple registrations of the same {@link PCollection} will * return the same unique ID. */ - String registerPCollection(PCollection pCollection) throws IOException { + public String registerPCollection(PCollection pCollection) throws IOException { String existing = pCollectionIds.get(pCollection); if (existing != null) { return existing; @@ -135,7 +135,8 @@ String registerPCollection(PCollection pCollection) throws IOException { * unique ID for the {@link WindowingStrategy}. Multiple registrations of the same {@link * WindowingStrategy} will return the same unique ID. */ - String registerWindowingStrategy(WindowingStrategy windowingStrategy) throws IOException { + public String registerWindowingStrategy(WindowingStrategy windowingStrategy) + throws IOException { String existing = windowingStrategyIds.get(windowingStrategy); if (existing != null) { return existing; @@ -162,7 +163,7 @@ String registerWindowingStrategy(WindowingStrategy windowingStrategy) thro * #equals(Object)} and {@link #hashCode()} but incompatible binary formats are not considered the * same coder. */ - String registerCoder(Coder coder) throws IOException { + public String registerCoder(Coder coder) throws IOException { String existing = coderIds.get(Equivalence.identity().wrap(coder)); if (existing != null) { return existing; @@ -191,7 +192,7 @@ private String uniqify(String baseName, Set existing) { * PCollection PCollections}, and {@link PTransform PTransforms}. */ @Experimental - RunnerApi.Components toComponents() { + public RunnerApi.Components toComponents() { return componentsBuilder.build(); } } From 0f53e2adc7509cd8383341c2b2a8c0275b7f0816 Mon Sep 17 00:00:00 2001 From: David Cavazos Date: Thu, 24 Aug 2017 11:43:23 -0700 Subject: [PATCH 323/346] Added tests for python gaming examples --- .../examples/complete/game/game_stats_test.py | 81 +++++++++++++++++++ .../complete/game/leader_board_test.py | 69 ++++++++++++++++ 2 files changed, 150 insertions(+) create mode 100644 sdks/python/apache_beam/examples/complete/game/game_stats_test.py create mode 100644 sdks/python/apache_beam/examples/complete/game/leader_board_test.py diff --git a/sdks/python/apache_beam/examples/complete/game/game_stats_test.py b/sdks/python/apache_beam/examples/complete/game/game_stats_test.py new file mode 100644 index 0000000000000..971f9dc9e2991 --- /dev/null +++ b/sdks/python/apache_beam/examples/complete/game/game_stats_test.py @@ -0,0 +1,81 @@ +# +# 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. +# + +"""Test for the game_stats example.""" + +import logging +import unittest + +import apache_beam as beam +from apache_beam.examples.complete.game import game_stats +from apache_beam.testing.test_pipeline import TestPipeline +from apache_beam.testing.util import assert_that +from apache_beam.testing.util import equal_to + + +class GameStatsTest(unittest.TestCase): + + SAMPLE_DATA = [ + 'user1_team1,team1,18,1447686663000,2015-11-16 15:11:03.921', + 'user1_team1,team1,18,1447690263000,2015-11-16 16:11:03.921', + 'user2_team2,team2,2,1447690263000,2015-11-16 16:11:03.955', + 'user3_team3,team3,8,1447690263000,2015-11-16 16:11:03.955', + 'user4_team3,team3,5,1447690263000,2015-11-16 16:11:03.959', + 'user1_team1,team1,14,1447697463000,2015-11-16 18:11:03.955', + 'robot1_team1,team1,9000,1447697463000,2015-11-16 18:11:03.955', + 'robot2_team2,team2,1,1447697463000,2015-11-16 20:11:03.955', + 'robot2_team2,team2,9000,1447697463000,2015-11-16 21:11:03.955', + ] + + def create_data(self, p): + return (p + | beam.Create(GameStatsTest.SAMPLE_DATA) + | beam.ParDo(game_stats.ParseGameEventFn()) + | beam.Map(lambda elem:\ + beam.window.TimestampedValue(elem, elem['timestamp']))) + + def test_spammy_users(self): + with TestPipeline() as p: + result = ( + self.create_data(p) + | beam.Map(lambda elem: (elem['user'], elem['score'])) + | game_stats.CalculateSpammyUsers()) + assert_that(result, equal_to([ + ('robot1_team1', 9000), ('robot2_team2', 9001)])) + + def test_game_stats_sessions(self): + session_gap = 5 * 60 + user_activity_window_duration = 30 * 60 + with TestPipeline() as p: + result = ( + self.create_data(p) + | beam.Map(lambda elem: (elem['user'], elem['score'])) + | 'WindowIntoSessions' >> beam.WindowInto( + beam.window.Sessions(session_gap), + timestamp_combiner=beam.window.TimestampCombiner.OUTPUT_AT_EOW) + | beam.CombinePerKey(lambda _: None) + | beam.ParDo(game_stats.UserSessionActivity()) + | 'WindowToExtractSessionMean' >> beam.WindowInto( + beam.window.FixedWindows(user_activity_window_duration)) + | beam.CombineGlobally(beam.combiners.MeanCombineFn())\ + .without_defaults()) + assert_that(result, equal_to([300.0, 300.0, 300.0])) + + +if __name__ == '__main__': + logging.getLogger().setLevel(logging.INFO) + unittest.main() diff --git a/sdks/python/apache_beam/examples/complete/game/leader_board_test.py b/sdks/python/apache_beam/examples/complete/game/leader_board_test.py new file mode 100644 index 0000000000000..aece2649b8983 --- /dev/null +++ b/sdks/python/apache_beam/examples/complete/game/leader_board_test.py @@ -0,0 +1,69 @@ +# +# 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. +# + +"""Test for the leader_board example.""" + +import logging +import unittest + +import apache_beam as beam +from apache_beam.examples.complete.game import leader_board +from apache_beam.testing.test_pipeline import TestPipeline +from apache_beam.testing.util import assert_that +from apache_beam.testing.util import equal_to + + +class LeaderBoardTest(unittest.TestCase): + + SAMPLE_DATA = [ + 'user1_team1,team1,18,1447686663000,2015-11-16 15:11:03.921', + 'user1_team1,team1,18,1447690263000,2015-11-16 16:11:03.921', + 'user2_team2,team2,2,1447690263000,2015-11-16 16:11:03.955', + 'user3_team3,team3,8,1447690263000,2015-11-16 16:11:03.955', + 'user4_team3,team3,5,1447690263000,2015-11-16 16:11:03.959', + 'user1_team1,team1,14,1447697463000,2015-11-16 18:11:03.955', + ] + + def create_data(self, p): + return (p + | beam.Create(LeaderBoardTest.SAMPLE_DATA) + | beam.ParDo(leader_board.ParseGameEventFn()) + | beam.Map(lambda elem:\ + beam.window.TimestampedValue(elem, elem['timestamp']))) + + def test_leader_board_teams(self): + with TestPipeline() as p: + result = ( + self.create_data(p) + | leader_board.CalculateTeamScores( + team_window_duration=60, + allowed_lateness=120)) + assert_that(result, equal_to([ + ('team1', 14), ('team1', 18), ('team1', 18), ('team2', 2), + ('team3', 13)])) + + def test_leader_board_users(self): + with TestPipeline() as p: + result = ( + self.create_data(p) + | leader_board.CalculateUserScores(allowed_lateness=120)) + assert_that(result, equal_to([])) + + +if __name__ == '__main__': + logging.getLogger().setLevel(logging.INFO) + unittest.main() From ef4239ab7928bfad95a4debb1517c2547473bf8f Mon Sep 17 00:00:00 2001 From: Robert Bradshaw Date: Tue, 22 Aug 2017 10:01:40 -0700 Subject: [PATCH 324/346] Add URN registration mechanism for coders. --- sdks/python/apache_beam/coders/coders.py | 66 +++++++++++++++++++----- 1 file changed, 53 insertions(+), 13 deletions(-) diff --git a/sdks/python/apache_beam/coders/coders.py b/sdks/python/apache_beam/coders/coders.py index 7ced5a9e12be1..0ea5f7c9499d8 100644 --- a/sdks/python/apache_beam/coders/coders.py +++ b/sdks/python/apache_beam/coders/coders.py @@ -202,25 +202,65 @@ def __eq__(self, other): and self._dict_without_impl() == other._dict_without_impl()) # pylint: enable=protected-access - def to_runner_api(self, context): - """For internal use only; no backwards-compatibility guarantees. + _known_urns = {} + + @classmethod + def register_urn(cls, urn, parameter_type, fn=None): + """Registeres a urn with a constructor. + + For example, if 'beam:fn:foo' had paramter type FooPayload, one could + write `RunnerApiFn.register_urn('bean:fn:foo', FooPayload, foo_from_proto)` + where foo_from_proto took as arguments a FooPayload and a PipelineContext. + This function can also be used as a decorator rather than passing the + callable in as the final parameter. + + A corresponding to_runner_api_parameter method would be expected that + returns the tuple ('beam:fn:foo', FooPayload) """ - # TODO(BEAM-115): Use specialized URNs and components. - serialized_coder = serialize_coder(self) + def register(fn): + cls._known_urns[urn] = parameter_type, fn + return staticmethod(fn) + if fn: + # Used as a statement. + register(fn) + else: + # Used as a decorator. + return register + + def to_runner_api(self, context): + from apache_beam.portability.api import beam_runner_api_pb2 + urn, typed_param, components = self.to_runner_api_parameter(context) return beam_runner_api_pb2.Coder( spec=beam_runner_api_pb2.SdkFunctionSpec( spec=beam_runner_api_pb2.FunctionSpec( - urn=urns.PICKLED_CODER, - any_param=proto_utils.pack_Any( - google.protobuf.wrappers_pb2.BytesValue( - value=serialized_coder)), - payload=serialized_coder))) + urn=urn, + any_param=proto_utils.pack_Any(typed_param), + payload=typed_param.SerializeToString() + if typed_param is not None else None)), + component_coder_ids=[context.coders.get_id(c) for c in components]) - @staticmethod - def from_runner_api(proto, context): - """For internal use only; no backwards-compatibility guarantees. + @classmethod + def from_runner_api(cls, coder_proto, context): + """Converts from an SdkFunctionSpec to a Fn object. + + Prefer registering a urn with its parameter type and constructor. """ - return deserialize_coder(proto.spec.spec.payload) + parameter_type, constructor = cls._known_urns[coder_proto.spec.spec.urn] + return constructor( + proto_utils.parse_Bytes(coder_proto.spec.spec.payload, parameter_type), + [context.coders.get_by_id(c) for c in coder_proto.component_coder_ids], + context) + + def to_runner_api_parameter(self, context): + return ( + urns.PICKLED_CODER, + google.protobuf.wrappers_pb2.BytesValue(value=serialize_coder(self)), + ()) + + +@Coder.register_urn(urns.PICKLED_CODER, google.protobuf.wrappers_pb2.BytesValue) +def _pickle_from_runner_api_parameter(payload, components, context): + return deserialize_coder(payload.value) class StrUtf8Coder(Coder): From 9cc004fb0c32234b541cd622a0d0ab4c5c3d2389 Mon Sep 17 00:00:00 2001 From: Robert Bradshaw Date: Tue, 22 Aug 2017 10:54:21 -0700 Subject: [PATCH 325/346] Runner API encoding of common coders. --- sdks/python/apache_beam/coders/coders.py | 42 +++++++++++++++++-- .../apache_beam/coders/coders_test_common.py | 4 +- sdks/python/apache_beam/utils/urns.py | 11 ++++- 3 files changed, 52 insertions(+), 5 deletions(-) diff --git a/sdks/python/apache_beam/coders/coders.py b/sdks/python/apache_beam/coders/coders.py index 0ea5f7c9499d8..e204369b31037 100644 --- a/sdks/python/apache_beam/coders/coders.py +++ b/sdks/python/apache_beam/coders/coders.py @@ -206,9 +206,9 @@ def __eq__(self, other): @classmethod def register_urn(cls, urn, parameter_type, fn=None): - """Registeres a urn with a constructor. + """Registers a urn with a constructor. - For example, if 'beam:fn:foo' had paramter type FooPayload, one could + For example, if 'beam:fn:foo' had parameter type FooPayload, one could write `RunnerApiFn.register_urn('bean:fn:foo', FooPayload, foo_from_proto)` where foo_from_proto took as arguments a FooPayload and a PipelineContext. This function can also be used as a decorator rather than passing the @@ -228,7 +228,6 @@ def register(fn): return register def to_runner_api(self, context): - from apache_beam.portability.api import beam_runner_api_pb2 urn, typed_param, components = self.to_runner_api_parameter(context) return beam_runner_api_pb2.Coder( spec=beam_runner_api_pb2.SdkFunctionSpec( @@ -257,6 +256,22 @@ def to_runner_api_parameter(self, context): google.protobuf.wrappers_pb2.BytesValue(value=serialize_coder(self)), ()) + @staticmethod + def register_structured_urn(urn, cls): + """Register a coder that's completely defined by its urn and its + component(s), if any, which are passed to construct the instance. + """ + cls.to_runner_api_parameter = ( + lambda self, unused_context: (urn, None, self._get_component_coders())) + + # pylint: disable=unused-variable + @Coder.register_urn(urn, None) + def from_runner_api_parameter(unused_payload, components, unused_context): + if components: + return cls(*components) + else: + return cls() + @Coder.register_urn(urns.PICKLED_CODER, google.protobuf.wrappers_pb2.BytesValue) def _pickle_from_runner_api_parameter(payload, components, context): @@ -337,6 +352,9 @@ def __hash__(self): return hash(type(self)) +Coder.register_structured_urn(urns.BYTES_CODER, BytesCoder) + + class VarIntCoder(FastCoder): """Variable-length integer coder.""" @@ -353,6 +371,9 @@ def __hash__(self): return hash(type(self)) +Coder.register_structured_urn(urns.VAR_INT_CODER, VarIntCoder) + + class FloatCoder(FastCoder): """A coder used for floating-point values.""" @@ -757,6 +778,9 @@ def __hash__(self): return hash((type(self), self._elem_coder)) +Coder.register_structured_urn(urns.ITERABLE_CODER, IterableCoder) + + class GlobalWindowCoder(SingletonCoder): """Coder for global windows.""" @@ -770,6 +794,9 @@ def as_cloud_object(self): } +Coder.register_structured_urn(urns.GLOBAL_WINDOW_CODER, GlobalWindowCoder) + + class IntervalWindowCoder(FastCoder): """Coder for an window defined by a start timestamp and a duration.""" @@ -791,6 +818,9 @@ def __hash__(self): return hash(type(self)) +Coder.register_structured_urn(urns.INTERVAL_WINDOW_CODER, IntervalWindowCoder) + + class WindowedValueCoder(FastCoder): """Coder for windowed values.""" @@ -847,6 +877,9 @@ def __hash__(self): (self.wrapped_value_coder, self.timestamp_coder, self.window_coder)) +Coder.register_structured_urn(urns.WINDOWED_VALUE_CODER, WindowedValueCoder) + + class LengthPrefixCoder(FastCoder): """For internal use only; no backwards-compatibility guarantees. @@ -886,3 +919,6 @@ def __eq__(self, other): def __hash__(self): return hash((type(self), self._value_coder)) + + +Coder.register_structured_urn(urns.LENGTH_PREFIX_CODER, LengthPrefixCoder) diff --git a/sdks/python/apache_beam/coders/coders_test_common.py b/sdks/python/apache_beam/coders/coders_test_common.py index 577c53aee8ddb..8b0353d922e77 100644 --- a/sdks/python/apache_beam/coders/coders_test_common.py +++ b/sdks/python/apache_beam/coders/coders_test_common.py @@ -26,6 +26,7 @@ from apache_beam.transforms.window import GlobalWindow from apache_beam.utils.timestamp import MIN_TIMESTAMP import observable +from apache_beam.runners import pipeline_context from apache_beam.transforms import window from apache_beam.utils import timestamp from apache_beam.utils import windowed_value @@ -90,7 +91,8 @@ def check_coder(self, coder, *values): self.assertEqual(coder.get_impl().get_estimated_size_and_observables(v), (coder.get_impl().estimate_size(v), [])) copy1 = dill.loads(dill.dumps(coder)) - copy2 = dill.loads(dill.dumps(coder)) + context = pipeline_context.PipelineContext() + copy2 = coders.Coder.from_runner_api(coder.to_runner_api(context), context) for v in values: self.assertEqual(v, copy1.decode(copy2.encode(v))) if coder.is_deterministic(): diff --git a/sdks/python/apache_beam/utils/urns.py b/sdks/python/apache_beam/utils/urns.py index acf729f2dc8e1..18959be87503f 100644 --- a/sdks/python/apache_beam/utils/urns.py +++ b/sdks/python/apache_beam/utils/urns.py @@ -35,7 +35,6 @@ PICKLED_DO_FN = "beam:dofn:pickled_python:v0.1" PICKLED_DO_FN_INFO = "beam:dofn:pickled_python_info:v0.1" PICKLED_COMBINE_FN = "beam:combinefn:pickled_python:v0.1" -PICKLED_CODER = "beam:coder:pickled_python:v0.1" PICKLED_TRANSFORM = "beam:ptransform:pickled_python:v0.1" PARDO_TRANSFORM = "beam:ptransform:pardo:v0.1" @@ -50,6 +49,16 @@ PICKLED_SOURCE = "beam:source:pickled_python:v0.1" +PICKLED_CODER = "beam:coder:pickled_python:v0.1" +BYTES_CODER = "urn:beam:coders:bytes:0.1" +VAR_INT_CODER = "urn:beam:coders:varint:0.1" +INTERVAL_WINDOW_CODER = "urn:beam:coders:interval_window:0.1" +ITERABLE_CODER = "urn:beam:coders:stream:0.1" +KV_CODER = "urn:beam:coders:kv:0.1" +LENGTH_PREFIX_CODER = "urn:beam:coders:length_prefix:0.1" +GLOBAL_WINDOW_CODER = "urn:beam:coders:urn:beam:coders:global_window:0.1" +WINDOWED_VALUE_CODER = "urn:beam:coders:windowed_value:0.1" + class RunnerApiFn(object): """Abstract base class that provides urn registration utilities. From c9c2e81672676e3ec705269a94f11fb1a2596c48 Mon Sep 17 00:00:00 2001 From: Mairbek Khadikov Date: Mon, 7 Aug 2017 12:33:19 -0700 Subject: [PATCH 326/346] Adds SpannerAccessor - a utility for DoFn's that use Spanner --- .../sdk/io/gcp/spanner/AbstractSpannerFn.java | 71 ------------------- .../io/gcp/spanner/CreateTransactionFn.java | 22 ++++-- .../io/gcp/spanner/NaiveSpannerReadFn.java | 18 +++-- .../sdk/io/gcp/spanner/SpannerAccessor.java | 43 +++++++++++ .../sdk/io/gcp/spanner/SpannerConfig.java | 22 ++++++ .../io/gcp/spanner/SpannerWriteGroupFn.java | 24 ++++--- 6 files changed, 111 insertions(+), 89 deletions(-) delete mode 100644 sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/AbstractSpannerFn.java create mode 100644 sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerAccessor.java diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/AbstractSpannerFn.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/AbstractSpannerFn.java deleted file mode 100644 index 50efdea41b094..0000000000000 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/AbstractSpannerFn.java +++ /dev/null @@ -1,71 +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.spanner; - -import com.google.cloud.spanner.DatabaseClient; -import com.google.cloud.spanner.DatabaseId; -import com.google.cloud.spanner.Spanner; -import com.google.cloud.spanner.SpannerOptions; -import org.apache.beam.sdk.transforms.DoFn; -import org.apache.beam.sdk.util.ReleaseInfo; - -/** - * Abstract {@link DoFn} that manages {@link Spanner} lifecycle. Use {@link - * AbstractSpannerFn#databaseClient} to access the Cloud Spanner database client. - */ -abstract class AbstractSpannerFn extends DoFn { - // A common user agent token that indicates that this request was originated from Apache Beam. - private static final String USER_AGENT_PREFIX = "Apache_Beam_Java"; - - private transient Spanner spanner; - private transient DatabaseClient databaseClient; - - abstract SpannerConfig getSpannerConfig(); - - @Setup - public void setup() throws Exception { - SpannerConfig spannerConfig = getSpannerConfig(); - SpannerOptions.Builder builder = SpannerOptions.newBuilder(); - if (spannerConfig.getProjectId() != null) { - builder.setProjectId(spannerConfig.getProjectId().get()); - } - if (spannerConfig.getServiceFactory() != null) { - builder.setServiceFactory(spannerConfig.getServiceFactory()); - } - ReleaseInfo releaseInfo = ReleaseInfo.getReleaseInfo(); - builder.setUserAgentPrefix(USER_AGENT_PREFIX + "/" + releaseInfo.getVersion()); - SpannerOptions options = builder.build(); - spanner = options.getService(); - databaseClient = spanner.getDatabaseClient(DatabaseId - .of(options.getProjectId(), spannerConfig.getInstanceId().get(), - spannerConfig.getDatabaseId().get())); - } - - @Teardown - public void teardown() throws Exception { - if (spanner == null) { - return; - } - spanner.close(); - spanner = null; - } - - protected DatabaseClient databaseClient() { - return databaseClient; - } -} diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/CreateTransactionFn.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/CreateTransactionFn.java index da8e8b15e1adf..5574ae16d86be 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/CreateTransactionFn.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/CreateTransactionFn.java @@ -17,12 +17,14 @@ */ package org.apache.beam.sdk.io.gcp.spanner; +import com.google.cloud.spanner.DatabaseClient; import com.google.cloud.spanner.ReadOnlyTransaction; import com.google.cloud.spanner.ResultSet; import com.google.cloud.spanner.Statement; +import org.apache.beam.sdk.transforms.DoFn; /** Creates a batch transaction. */ -class CreateTransactionFn extends AbstractSpannerFn { +class CreateTransactionFn extends DoFn { private final SpannerIO.CreateTransaction config; @@ -30,10 +32,22 @@ class CreateTransactionFn extends AbstractSpannerFn { this.config = config; } + private transient SpannerAccessor spannerAccessor; + + @Setup + public void setup() throws Exception { + spannerAccessor = config.getSpannerConfig().connectToSpanner(); + } + @Teardown + public void teardown() throws Exception { + spannerAccessor.close(); + } + @ProcessElement public void processElement(ProcessContext c) throws Exception { + DatabaseClient databaseClient = spannerAccessor.getDatabaseClient(); try (ReadOnlyTransaction readOnlyTransaction = - databaseClient().readOnlyTransaction(config.getTimestampBound())) { + databaseClient.readOnlyTransaction(config.getTimestampBound())) { // Run a dummy sql statement to force the RPC and obtain the timestamp from the server. ResultSet resultSet = readOnlyTransaction.executeQuery(Statement.of("SELECT 1")); while (resultSet.next()) { @@ -44,8 +58,4 @@ public void processElement(ProcessContext c) throws Exception { } } - @Override - SpannerConfig getSpannerConfig() { - return config.getSpannerConfig(); - } } diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/NaiveSpannerReadFn.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/NaiveSpannerReadFn.java index 92b3fe37b9ca5..5dc6ead957fb3 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/NaiveSpannerReadFn.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/NaiveSpannerReadFn.java @@ -17,19 +17,22 @@ */ package org.apache.beam.sdk.io.gcp.spanner; +import com.google.cloud.spanner.DatabaseClient; import com.google.cloud.spanner.ReadOnlyTransaction; import com.google.cloud.spanner.ResultSet; import com.google.cloud.spanner.Struct; import com.google.cloud.spanner.TimestampBound; import com.google.common.annotations.VisibleForTesting; import javax.annotation.Nullable; +import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.values.PCollectionView; /** A simplest read function implementation. Parallelism support is coming. */ @VisibleForTesting -class NaiveSpannerReadFn extends AbstractSpannerFn { +class NaiveSpannerReadFn extends DoFn { private final SpannerConfig config; @Nullable private final PCollectionView transaction; + private transient SpannerAccessor spannerAccessor; NaiveSpannerReadFn(SpannerConfig config, @Nullable PCollectionView transaction) { this.config = config; @@ -40,8 +43,14 @@ class NaiveSpannerReadFn extends AbstractSpannerFn { this(config, null); } - SpannerConfig getSpannerConfig() { - return config; + + @Setup + public void setup() throws Exception { + spannerAccessor = config.connectToSpanner(); + } + @Teardown + public void teardown() throws Exception { + spannerAccessor.close(); } @ProcessElement @@ -52,8 +61,9 @@ public void processElement(ProcessContext c) throws Exception { timestampBound = TimestampBound.ofReadTimestamp(transaction.timestamp()); } ReadOperation op = c.element(); + DatabaseClient databaseClient = spannerAccessor.getDatabaseClient(); try (ReadOnlyTransaction readOnlyTransaction = - databaseClient().readOnlyTransaction(timestampBound)) { + databaseClient.readOnlyTransaction(timestampBound)) { ResultSet resultSet = execute(op, readOnlyTransaction); while (resultSet.next()) { c.output(resultSet.getCurrentRowAsStruct()); diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerAccessor.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerAccessor.java new file mode 100644 index 0000000000000..f32e661df6da4 --- /dev/null +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerAccessor.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.sdk.io.gcp.spanner; + +import com.google.cloud.spanner.DatabaseClient; +import com.google.cloud.spanner.Spanner; + +/** + * Manages lifecycle of {@link DatabaseClient} and {@link Spanner} instances. + */ +public class SpannerAccessor implements AutoCloseable { + private final Spanner spanner; + private final DatabaseClient databaseClient; + + SpannerAccessor(Spanner spanner, DatabaseClient databaseClient) { + this.spanner = spanner; + this.databaseClient = databaseClient; + } + + public DatabaseClient getDatabaseClient() { + return databaseClient; + } + + @Override + public void close() { + spanner.close(); + } +} diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerConfig.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerConfig.java index 034c38acd23ba..6646f32fe1eae 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerConfig.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerConfig.java @@ -21,6 +21,8 @@ import com.google.auto.value.AutoValue; import com.google.cloud.ServiceFactory; +import com.google.cloud.spanner.DatabaseClient; +import com.google.cloud.spanner.DatabaseId; import com.google.cloud.spanner.Spanner; import com.google.cloud.spanner.SpannerOptions; import com.google.common.annotations.VisibleForTesting; @@ -29,10 +31,13 @@ import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.options.ValueProvider; import org.apache.beam.sdk.transforms.display.DisplayData; +import org.apache.beam.sdk.util.ReleaseInfo; /** Configuration for a Cloud Spanner client. */ @AutoValue public abstract class SpannerConfig implements Serializable { + // A common user agent token that indicates that this request was originated from Apache Beam. + private static final String USER_AGENT_PREFIX = "Apache_Beam_Java"; @Nullable abstract ValueProvider getProjectId(); @@ -123,4 +128,21 @@ SpannerConfig withServiceFactory(ServiceFactory service return toBuilder().setServiceFactory(serviceFactory).build(); } + public SpannerAccessor connectToSpanner() { + SpannerOptions.Builder builder = SpannerOptions.newBuilder(); + if (getProjectId() != null) { + builder.setProjectId(getProjectId().get()); + } + if (getServiceFactory() != null) { + builder.setServiceFactory(this.getServiceFactory()); + } + ReleaseInfo releaseInfo = ReleaseInfo.getReleaseInfo(); + builder.setUserAgentPrefix(USER_AGENT_PREFIX + "/" + releaseInfo.getVersion()); + SpannerOptions options = builder.build(); + Spanner spanner = options.getService(); + DatabaseClient databaseClient = spanner.getDatabaseClient( + DatabaseId.of(options.getProjectId(), getInstanceId().get(), getDatabaseId().get())); + return new SpannerAccessor(spanner, databaseClient); + } + } diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerWriteGroupFn.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerWriteGroupFn.java index 34a11da8754f5..9343c0c698788 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerWriteGroupFn.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerWriteGroupFn.java @@ -18,6 +18,7 @@ package org.apache.beam.sdk.io.gcp.spanner; import com.google.cloud.spanner.AbortedException; +import com.google.cloud.spanner.DatabaseClient; import com.google.common.annotations.VisibleForTesting; import com.google.common.collect.Iterables; @@ -25,6 +26,7 @@ import java.util.ArrayList; import java.util.List; +import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.display.DisplayData; import org.apache.beam.sdk.util.BackOff; import org.apache.beam.sdk.util.BackOffUtils; @@ -35,7 +37,8 @@ import org.slf4j.LoggerFactory; /** Batches together and writes mutations to Google Cloud Spanner. */ -@VisibleForTesting class SpannerWriteGroupFn extends AbstractSpannerFn { +@VisibleForTesting +class SpannerWriteGroupFn extends DoFn { private static final Logger LOG = LoggerFactory.getLogger(SpannerWriteGroupFn.class); private final SpannerIO.Write spec; // Current batch of mutations to be written. @@ -48,21 +51,25 @@ .withMaxRetries(MAX_RETRIES) .withInitialBackoff(Duration.standardSeconds(5)); - @VisibleForTesting SpannerWriteGroupFn(SpannerIO.Write spec) { - this.spec = spec; - } + private transient SpannerAccessor spannerAccessor; - @Override SpannerConfig getSpannerConfig() { - return spec.getSpannerConfig(); + @VisibleForTesting + SpannerWriteGroupFn(SpannerIO.Write spec) { + this.spec = spec; } @Setup public void setup() throws Exception { - super.setup(); + spannerAccessor = spec.getSpannerConfig().connectToSpanner(); mutations = new ArrayList<>(); batchSizeBytes = 0; } + @Teardown + public void teardown() throws Exception { + spannerAccessor.close(); + } + @ProcessElement public void processElement(ProcessContext c) throws Exception { MutationGroup m = c.element(); @@ -94,10 +101,11 @@ private void flushBatch() throws AbortedException, IOException, InterruptedExcep Sleeper sleeper = Sleeper.DEFAULT; BackOff backoff = BUNDLE_WRITE_BACKOFF.backoff(); + DatabaseClient databaseClient = spannerAccessor.getDatabaseClient(); while (true) { // Batch upsert rows. try { - databaseClient().writeAtLeastOnce(Iterables.concat(mutations)); + databaseClient.writeAtLeastOnce(Iterables.concat(mutations)); // Break if the commit threw no exception. break; From 3362d1f52bd2076908d74ff6643a483468630502 Mon Sep 17 00:00:00 2001 From: Raghu Angadi Date: Thu, 24 Aug 2017 14:33:28 -0700 Subject: [PATCH 327/346] Fix min_timestamp used for KafkaIO watermark. --- .../src/main/java/org/apache/beam/sdk/io/kafka/KafkaIO.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) 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 7fb4260313c7a..dae4c1d4c1b51 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 @@ -82,6 +82,7 @@ import org.apache.beam.sdk.transforms.SerializableFunction; import org.apache.beam.sdk.transforms.SimpleFunction; import org.apache.beam.sdk.transforms.display.DisplayData; +import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PBegin; import org.apache.beam.sdk.values.PCollection; @@ -899,7 +900,7 @@ private static class UnboundedKafkaReader extends UnboundedReader Date: Mon, 22 May 2017 07:51:13 +0200 Subject: [PATCH 328/346] [BEAM-1531] Add dynamic work rebalancing support for HBaseIO --- .../org/apache/beam/sdk/io/hbase/HBaseIO.java | 81 ++++++++++++++++++- .../apache/beam/sdk/io/hbase/HBaseIOTest.java | 56 ++++++++++++- 2 files changed, 132 insertions(+), 5 deletions(-) diff --git a/sdks/java/io/hbase/src/main/java/org/apache/beam/sdk/io/hbase/HBaseIO.java b/sdks/java/io/hbase/src/main/java/org/apache/beam/sdk/io/hbase/HBaseIO.java index 2ba682639ab75..7f58cef30310c 100644 --- a/sdks/java/io/hbase/src/main/java/org/apache/beam/sdk/io/hbase/HBaseIO.java +++ b/sdks/java/io/hbase/src/main/java/org/apache/beam/sdk/io/hbase/HBaseIO.java @@ -36,6 +36,7 @@ import org.apache.beam.sdk.io.hadoop.SerializableConfiguration; import org.apache.beam.sdk.io.range.ByteKey; import org.apache.beam.sdk.io.range.ByteKeyRange; +import org.apache.beam.sdk.io.range.ByteKeyRangeTracker; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.PTransform; @@ -303,6 +304,22 @@ static class HBaseSource extends BoundedSource { this.estimatedSizeBytes = estimatedSizeBytes; } + HBaseSource withStartKey(ByteKey startKey) throws IOException { + checkNotNull(startKey, "startKey"); + Read newRead = new Read(read.serializableConfiguration, read.tableId, + new SerializableScan( + new Scan(read.serializableScan.get()).setStartRow(startKey.getBytes()))); + return new HBaseSource(newRead, estimatedSizeBytes); + } + + HBaseSource withEndKey(ByteKey endKey) throws IOException { + checkNotNull(endKey, "endKey"); + Read newRead = new Read(read.serializableConfiguration, read.tableId, + new SerializableScan( + new Scan(read.serializableScan.get()).setStopRow(endKey.getBytes()))); + return new HBaseSource(newRead, estimatedSizeBytes); + } + @Override public long getEstimatedSizeBytes(PipelineOptions pipelineOptions) throws Exception { if (estimatedSizeBytes == null) { @@ -463,19 +480,25 @@ public Coder getOutputCoder() { } private static class HBaseReader extends BoundedSource.BoundedReader { - private final HBaseSource source; + private HBaseSource source; private Connection connection; private ResultScanner scanner; private Iterator iter; private Result current; + private final ByteKeyRangeTracker rangeTracker; private long recordsReturned; HBaseReader(HBaseSource source) { this.source = source; + Scan scan = source.read.serializableScan.get(); + ByteKeyRange range = ByteKeyRange + .of(ByteKey.copyFrom(scan.getStartRow()), ByteKey.copyFrom(scan.getStopRow())); + rangeTracker = ByteKeyRangeTracker.of(range); } @Override public boolean start() throws IOException { + HBaseSource source = getCurrentSource(); Configuration configuration = source.read.serializableConfiguration.get(); String tableId = source.read.tableId; connection = ConnectionFactory.createConnection(configuration); @@ -495,9 +518,15 @@ public Result getCurrent() throws NoSuchElementException { @Override public boolean advance() throws IOException { - boolean hasRecord = iter.hasNext(); + if (!iter.hasNext()) { + return rangeTracker.markDone(); + } + final Result next = iter.next(); + boolean hasRecord = + rangeTracker.tryReturnRecordAt(true, ByteKey.copyFrom(next.getRow())) + || rangeTracker.markDone(); if (hasRecord) { - current = iter.next(); + current = next; ++recordsReturned; } return hasRecord; @@ -517,9 +546,53 @@ public void close() throws IOException { } @Override - public BoundedSource getCurrentSource() { + public synchronized HBaseSource getCurrentSource() { return source; } + + @Override + public final Double getFractionConsumed() { + return rangeTracker.getFractionConsumed(); + } + + @Override + public final long getSplitPointsConsumed() { + return rangeTracker.getSplitPointsConsumed(); + } + + @Override + @Nullable + public final synchronized HBaseSource splitAtFraction(double fraction) { + ByteKey splitKey; + try { + splitKey = rangeTracker.getRange().interpolateKey(fraction); + } catch (RuntimeException e) { + LOG.info("{}: Failed to interpolate key for fraction {}.", rangeTracker.getRange(), + fraction, e); + return null; + } + LOG.info( + "Proposing to split {} at fraction {} (key {})", rangeTracker, fraction, splitKey); + HBaseSource primary; + HBaseSource residual; + try { + primary = source.withEndKey(splitKey); + residual = source.withStartKey(splitKey); + } catch (Exception e) { + LOG.info( + "{}: Interpolating for fraction {} yielded invalid split key {}.", + rangeTracker.getRange(), + fraction, + splitKey, + e); + return null; + } + if (!rangeTracker.trySplitAtPosition(splitKey)) { + return null; + } + this.source = primary; + return residual; + } } /** diff --git a/sdks/java/io/hbase/src/test/java/org/apache/beam/sdk/io/hbase/HBaseIOTest.java b/sdks/java/io/hbase/src/test/java/org/apache/beam/sdk/io/hbase/HBaseIOTest.java index 806a27f722841..0b7f203b1c5fb 100644 --- a/sdks/java/io/hbase/src/test/java/org/apache/beam/sdk/io/hbase/HBaseIOTest.java +++ b/sdks/java/io/hbase/src/test/java/org/apache/beam/sdk/io/hbase/HBaseIOTest.java @@ -18,6 +18,9 @@ package org.apache.beam.sdk.io.hbase; import static org.apache.beam.sdk.testing.SourceTestUtils.assertSourcesEqualReferenceSource; +import static org.apache.beam.sdk.testing.SourceTestUtils.assertSplitAtFractionExhaustive; +import static org.apache.beam.sdk.testing.SourceTestUtils.assertSplitAtFractionFails; +import static org.apache.beam.sdk.testing.SourceTestUtils.assertSplitAtFractionSucceedsAndConsistent; import static org.apache.beam.sdk.transforms.display.DisplayDataMatchers.hasDisplayItem; import static org.hamcrest.Matchers.hasSize; import static org.junit.Assert.assertEquals; @@ -83,7 +86,7 @@ public class HBaseIOTest { private static HBaseTestingUtility htu; private static HBaseAdmin admin; - private static Configuration conf = HBaseConfiguration.create(); + private static final Configuration conf = HBaseConfiguration.create(); private static final byte[] COLUMN_FAMILY = Bytes.toBytes("info"); private static final byte[] COLUMN_NAME = Bytes.toBytes("name"); private static final byte[] COLUMN_EMAIL = Bytes.toBytes("email"); @@ -277,6 +280,57 @@ public void testReadingWithKeyRange() throws Exception { .withKeyRange(startRow, stopRow), 441); } + /** + * Tests dynamic work rebalancing exhaustively. + */ + @Test + public void testReadingSplitAtFractionExhaustive() throws Exception { + final String table = "TEST-FEW-ROWS-SPLIT-EXHAUSTIVE-TABLE"; + final int numRows = 7; + + createTable(table); + writeData(table, numRows); + + HBaseIO.Read read = HBaseIO.read().withConfiguration(conf).withTableId(table); + HBaseSource source = new HBaseSource(read, null /* estimatedSizeBytes */) + .withStartKey(ByteKey.of(48)).withEndKey(ByteKey.of(58)); + + assertSplitAtFractionExhaustive(source, null); + } + + /** + * Unit tests of splitAtFraction. + */ + @Test + public void testReadingSplitAtFraction() throws Exception { + final String table = "TEST-SPLIT-AT-FRACTION"; + final int numRows = 10; + + createTable(table); + writeData(table, numRows); + + HBaseIO.Read read = HBaseIO.read().withConfiguration(conf).withTableId(table); + HBaseSource source = new HBaseSource(read, null /* estimatedSizeBytes */); + + // The value k is based on the partitioning schema for the data, in this test case, + // the partitioning is HEX-based, so we start from 1/16m and the value k will be + // around 1/256, so the tests are done in approximately k ~= 0.003922 steps + double k = 0.003922; + + assertSplitAtFractionFails(source, 0, k, null /* options */); + assertSplitAtFractionFails(source, 0, 1.0, null /* options */); + // With 1 items read, all split requests past k will succeed. + assertSplitAtFractionSucceedsAndConsistent(source, 1, k, null /* options */); + assertSplitAtFractionSucceedsAndConsistent(source, 1, 0.666, null /* options */); + // With 3 items read, all split requests past 3k will succeed. + assertSplitAtFractionFails(source, 3, 2 * k, null /* options */); + assertSplitAtFractionSucceedsAndConsistent(source, 3, 3 * k, null /* options */); + assertSplitAtFractionSucceedsAndConsistent(source, 3, 4 * k, null /* options */); + // With 6 items read, all split requests past 6k will succeed. + assertSplitAtFractionFails(source, 6, 5 * k, null /* options */); + assertSplitAtFractionSucceedsAndConsistent(source, 6, 0.7, null /* options */); + } + @Test public void testReadingDisplayData() { HBaseIO.Read read = HBaseIO.read().withConfiguration(conf).withTableId("fooTable"); From 8d36a261d4e8c6569e9036a27d45c00daccd908b Mon Sep 17 00:00:00 2001 From: Luke Cwik Date: Thu, 24 Aug 2017 18:34:47 -0700 Subject: [PATCH 329/346] [BEAM-1347] Add a BagUserState implementation over the BeamFnStateClient --- .../beam/fn/harness/state/BagUserState.java | 121 ++++++++++++++++++ .../state/LazyCachingIteratorToIterable.java | 72 +++++++++++ .../fn/harness/state/BagUserStateTest.java | 106 +++++++++++++++ .../harness/state/FakeBeamFnStateClient.java | 110 ++++++++++++++++ .../LazyCachingIteratorToIterableTest.java | 76 +++++++++++ 5 files changed, 485 insertions(+) create mode 100644 sdks/java/harness/src/main/java/org/apache/beam/fn/harness/state/BagUserState.java create mode 100644 sdks/java/harness/src/main/java/org/apache/beam/fn/harness/state/LazyCachingIteratorToIterable.java create mode 100644 sdks/java/harness/src/test/java/org/apache/beam/fn/harness/state/BagUserStateTest.java create mode 100644 sdks/java/harness/src/test/java/org/apache/beam/fn/harness/state/FakeBeamFnStateClient.java create mode 100644 sdks/java/harness/src/test/java/org/apache/beam/fn/harness/state/LazyCachingIteratorToIterableTest.java diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/state/BagUserState.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/state/BagUserState.java new file mode 100644 index 0000000000000..2d7f0c82c8e0e --- /dev/null +++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/state/BagUserState.java @@ -0,0 +1,121 @@ +/* + * 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.fn.harness.state; + +import static com.google.common.base.Preconditions.checkState; + +import com.google.common.collect.Iterables; +import com.google.protobuf.ByteString; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.concurrent.CompletableFuture; +import java.util.function.Supplier; +import org.apache.beam.fn.harness.stream.DataStreams; +import org.apache.beam.fn.v1.BeamFnApi.StateAppendRequest; +import org.apache.beam.fn.v1.BeamFnApi.StateClearRequest; +import org.apache.beam.fn.v1.BeamFnApi.StateRequest.Builder; +import org.apache.beam.sdk.coders.Coder; + +/** + * An implementation of a bag user state that utilizes the Beam Fn State API to fetch, clear + * and persist values. + * + *

              Calling {@link #asyncClose()} schedules any required persistence changes. This object should + * no longer be used after it is closed. + * + *

              TODO: Move to an async persist model where persistence is signalled based upon cache + * memory pressure and its need to flush. + * + *

              TODO: Support block level caching and prefetch. + */ +public class BagUserState { + private final BeamFnStateClient beamFnStateClient; + private final String stateId; + private final Coder coder; + private final Supplier partialRequestSupplier; + private Iterable oldValues; + private ArrayList newValues; + private List unmodifiableNewValues; + private boolean isClosed; + + public BagUserState( + BeamFnStateClient beamFnStateClient, + String stateId, + Coder coder, + Supplier partialRequestSupplier) { + this.beamFnStateClient = beamFnStateClient; + this.stateId = stateId; + this.coder = coder; + this.partialRequestSupplier = partialRequestSupplier; + this.oldValues = new LazyCachingIteratorToIterable<>( + new DataStreams.DataStreamDecoder(coder, + DataStreams.inbound( + StateFetchingIterators.usingPartialRequestWithStateKey( + beamFnStateClient, + partialRequestSupplier)))); + this.newValues = new ArrayList<>(); + this.unmodifiableNewValues = Collections.unmodifiableList(newValues); + } + + public Iterable get() { + checkState(!isClosed, + "Bag user state is no longer usable because it is closed for %s", stateId); + // If we were cleared we should disregard old values. + if (oldValues == null) { + return unmodifiableNewValues; + } + return Iterables.concat(oldValues, unmodifiableNewValues); + } + + public void append(T t) { + checkState(!isClosed, + "Bag user state is no longer usable because it is closed for %s", stateId); + newValues.add(t); + } + + public void clear() { + checkState(!isClosed, + "Bag user state is no longer usable because it is closed for %s", stateId); + oldValues = null; + newValues.clear(); + } + + public void asyncClose() throws Exception { + checkState(!isClosed, + "Bag user state is no longer usable because it is closed for %s", stateId); + if (oldValues == null) { + beamFnStateClient.handle( + partialRequestSupplier.get() + .setClear(StateClearRequest.getDefaultInstance()), + new CompletableFuture<>()); + } + if (!newValues.isEmpty()) { + ByteString.Output out = ByteString.newOutput(); + for (T newValue : newValues) { + // TODO: Replace with chunking output stream + coder.encode(newValue, out); + } + beamFnStateClient.handle( + partialRequestSupplier.get() + .setAppend(StateAppendRequest.newBuilder().setData(out.toByteString())), + new CompletableFuture<>()); + } + isClosed = true; + } +} diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/state/LazyCachingIteratorToIterable.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/state/LazyCachingIteratorToIterable.java new file mode 100644 index 0000000000000..0a433171518f1 --- /dev/null +++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/state/LazyCachingIteratorToIterable.java @@ -0,0 +1,72 @@ +/* + * 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.fn.harness.state; + +import java.util.ArrayList; +import java.util.Iterator; +import java.util.List; +import java.util.NoSuchElementException; + +/** + * Converts an iterator to an iterable lazily loading values from the underlying iterator + * and caching them to support reiteration. + */ +class LazyCachingIteratorToIterable implements Iterable { + private final List cachedElements; + private final Iterator iterator; + + public LazyCachingIteratorToIterable(Iterator iterator) { + this.cachedElements = new ArrayList<>(); + this.iterator = iterator; + } + + @Override + public Iterator iterator() { + return new CachingIterator(); + } + + /** An {@link Iterator} which adds and fetched values into the cached elements list. */ + private class CachingIterator implements Iterator { + private int position = 0; + + private CachingIterator() { + } + + @Override + public boolean hasNext() { + // The order of the short circuit is important below. + return position < cachedElements.size() || iterator.hasNext(); + } + + @Override + public T next() { + if (position < cachedElements.size()) { + return cachedElements.get(position++); + } + + if (!iterator.hasNext()) { + throw new NoSuchElementException(); + } + + T rval = iterator.next(); + cachedElements.add(rval); + position += 1; + return rval; + } + } +} diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/state/BagUserStateTest.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/state/BagUserStateTest.java new file mode 100644 index 0000000000000..f3c76ac8a7c63 --- /dev/null +++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/state/BagUserStateTest.java @@ -0,0 +1,106 @@ +/* + * 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.fn.harness.state; + +import static org.junit.Assert.assertArrayEquals; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNull; + +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Iterables; +import com.google.protobuf.ByteString; +import java.io.IOException; +import org.apache.beam.fn.v1.BeamFnApi.StateKey; +import org.apache.beam.fn.v1.BeamFnApi.StateRequest; +import org.apache.beam.sdk.coders.StringUtf8Coder; +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 BagUserState}. */ +@RunWith(JUnit4.class) +public class BagUserStateTest { + @Rule public ExpectedException thrown = ExpectedException.none(); + + @Test + public void testGet() throws Exception { + FakeBeamFnStateClient fakeClient = new FakeBeamFnStateClient(ImmutableMap.of( + key("A"), encode("A1", "A2", "A3"))); + BagUserState userState = + new BagUserState<>(fakeClient, "A", StringUtf8Coder.of(), () -> requestForId("A")); + assertArrayEquals(new String[]{ "A1", "A2", "A3" }, + Iterables.toArray(userState.get(), String.class)); + + userState.asyncClose(); + thrown.expect(IllegalStateException.class); + userState.get(); + } + + @Test + public void testAppend() throws Exception { + FakeBeamFnStateClient fakeClient = new FakeBeamFnStateClient(ImmutableMap.of( + key("A"), encode("A1"))); + BagUserState userState = + new BagUserState<>(fakeClient, "A", StringUtf8Coder.of(), () -> requestForId("A")); + userState.append("A2"); + userState.append("A3"); + userState.asyncClose(); + + assertEquals(encode("A1", "A2", "A3"), fakeClient.getData().get(key("A"))); + thrown.expect(IllegalStateException.class); + userState.append("A4"); + } + + @Test + public void testClear() throws Exception { + FakeBeamFnStateClient fakeClient = new FakeBeamFnStateClient(ImmutableMap.of( + key("A"), encode("A1", "A2", "A3"))); + BagUserState userState = + new BagUserState<>(fakeClient, "A", StringUtf8Coder.of(), () -> requestForId("A")); + + userState.clear(); + userState.append("A1"); + userState.clear(); + userState.asyncClose(); + + assertNull(fakeClient.getData().get(key("A"))); + thrown.expect(IllegalStateException.class); + userState.clear(); + } + + private StateRequest.Builder requestForId(String id) { + return StateRequest.newBuilder().setStateKey( + StateKey.newBuilder().setBagUserState( + StateKey.BagUserState.newBuilder().setKey(ByteString.copyFromUtf8(id)))); + } + + private StateKey key(String id) { + return StateKey.newBuilder().setBagUserState( + StateKey.BagUserState.newBuilder().setKey(ByteString.copyFromUtf8(id))).build(); + } + + private ByteString encode(String ... values) throws IOException { + ByteString.Output out = ByteString.newOutput(); + for (String value : values) { + StringUtf8Coder.of().encode(value, out); + } + return out.toByteString(); + } +} diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/state/FakeBeamFnStateClient.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/state/FakeBeamFnStateClient.java new file mode 100644 index 0000000000000..d260207435129 --- /dev/null +++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/state/FakeBeamFnStateClient.java @@ -0,0 +1,110 @@ +/* + * 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.fn.harness.state; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotEquals; + +import com.google.protobuf.ByteString; +import java.util.Collections; +import java.util.Map; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ConcurrentHashMap; +import org.apache.beam.fn.v1.BeamFnApi.StateAppendResponse; +import org.apache.beam.fn.v1.BeamFnApi.StateClearResponse; +import org.apache.beam.fn.v1.BeamFnApi.StateGetResponse; +import org.apache.beam.fn.v1.BeamFnApi.StateKey; +import org.apache.beam.fn.v1.BeamFnApi.StateKey.TypeCase; +import org.apache.beam.fn.v1.BeamFnApi.StateRequest; +import org.apache.beam.fn.v1.BeamFnApi.StateRequest.RequestCase; +import org.apache.beam.fn.v1.BeamFnApi.StateResponse; + +/** A fake implementation of a {@link BeamFnStateClient} to aid with testing. */ +public class FakeBeamFnStateClient implements BeamFnStateClient { + private final Map data; + private int currentId; + + public FakeBeamFnStateClient(Map initialData) { + this.data = new ConcurrentHashMap<>(initialData); + } + + public Map getData() { + return Collections.unmodifiableMap(data); + } + + @Override + public void handle(StateRequest.Builder requestBuilder, + CompletableFuture responseFuture) { + // The id should never be filled out + assertEquals("", requestBuilder.getId()); + requestBuilder.setId(generateId()); + + StateRequest request = requestBuilder.build(); + StateKey key = request.getStateKey(); + StateResponse.Builder response; + + assertNotEquals(RequestCase.REQUEST_NOT_SET, request.getRequestCase()); + assertNotEquals(TypeCase.TYPE_NOT_SET, key.getTypeCase()); + // multimap side input and runner based state keys only support get requests + if (key.getTypeCase() == TypeCase.MULTIMAP_SIDE_INPUT + || key.getTypeCase() == TypeCase.RUNNER) { + assertEquals(RequestCase.GET, request.getRequestCase()); + } + + switch (request.getRequestCase()) { + case GET: + // Chunk gets into 5 byte return blocks + ByteString byteString = data.get(request.getStateKey()); + int block = 0; + if (request.getGet().getContinuationToken().size() > 0) { + block = Integer.parseInt(request.getGet().getContinuationToken().toStringUtf8()); + } + ByteString returnBlock = byteString.substring( + block * 5, Math.min(byteString.size(), (block + 1) * 5)); + ByteString continuationToken = ByteString.EMPTY; + if (byteString.size() > (block + 1) * 5) { + continuationToken = ByteString.copyFromUtf8(Integer.toString(block + 1)); + } + response = StateResponse.newBuilder().setGet(StateGetResponse.newBuilder() + .setData(returnBlock) + .setContinuationToken(continuationToken)); + break; + + case CLEAR: + data.remove(request.getStateKey()); + response = StateResponse.newBuilder().setClear(StateClearResponse.getDefaultInstance()); + break; + + case APPEND: + ByteString previousValue = data.getOrDefault(request.getStateKey(), ByteString.EMPTY); + data.put(request.getStateKey(), previousValue.concat(request.getAppend().getData())); + response = StateResponse.newBuilder().setAppend(StateAppendResponse.getDefaultInstance()); + break; + + default: + throw new IllegalStateException( + String.format("Unknown request type %s", request.getRequestCase())); + } + + responseFuture.complete(response.setId(requestBuilder.getId()).build()); + } + + private String generateId() { + return Integer.toString(++currentId); + } +} diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/state/LazyCachingIteratorToIterableTest.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/state/LazyCachingIteratorToIterableTest.java new file mode 100644 index 0000000000000..53eefb4c97bd5 --- /dev/null +++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/state/LazyCachingIteratorToIterableTest.java @@ -0,0 +1,76 @@ +/* + * 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.fn.harness.state; + +import static org.junit.Assert.assertArrayEquals; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; + +import com.google.common.collect.Iterables; +import com.google.common.collect.Iterators; +import java.util.Iterator; +import java.util.NoSuchElementException; +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 LazyCachingIteratorToIterable}. */ +@RunWith(JUnit4.class) +public class LazyCachingIteratorToIterableTest { + @Rule public ExpectedException thrown = ExpectedException.none(); + + @Test + public void testEmptyIterator() { + Iterable iterable = new LazyCachingIteratorToIterable<>(Iterators.forArray()); + assertArrayEquals(new Object[0], Iterables.toArray(iterable, Object.class)); + // iterate multiple times + assertArrayEquals(new Object[0], Iterables.toArray(iterable, Object.class)); + + thrown.expect(NoSuchElementException.class); + iterable.iterator().next(); + } + + @Test + public void testInterleavedIteration() { + Iterable iterable = + new LazyCachingIteratorToIterable<>(Iterators.forArray("A", "B", "C")); + + Iterator iterator1 = iterable.iterator(); + assertTrue(iterator1.hasNext()); + assertEquals("A", iterator1.next()); + Iterator iterator2 = iterable.iterator(); + assertTrue(iterator2.hasNext()); + assertEquals("A", iterator2.next()); + assertTrue(iterator2.hasNext()); + assertEquals("B", iterator2.next()); + assertTrue(iterator1.hasNext()); + assertEquals("B", iterator1.next()); + assertTrue(iterator1.hasNext()); + assertEquals("C", iterator1.next()); + assertFalse(iterator1.hasNext()); + assertTrue(iterator2.hasNext()); + assertEquals("C", iterator2.next()); + assertFalse(iterator2.hasNext()); + + thrown.expect(NoSuchElementException.class); + iterator1.next(); + } +} From 428bfde022dd199484c13408bb2f220164840dac Mon Sep 17 00:00:00 2001 From: Luke Cwik Date: Thu, 24 Aug 2017 18:31:50 -0700 Subject: [PATCH 330/346] [BEAM-1347] Remove unused FakeStepContext now that FnApiDoFnRunner has its own implementation --- .../beam/fn/harness/fake/FakeStepContext.java | 39 ------------------- .../beam/fn/harness/fake/package-info.java | 22 ----------- 2 files changed, 61 deletions(-) delete mode 100644 sdks/java/harness/src/main/java/org/apache/beam/fn/harness/fake/FakeStepContext.java delete mode 100644 sdks/java/harness/src/main/java/org/apache/beam/fn/harness/fake/package-info.java diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/fake/FakeStepContext.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/fake/FakeStepContext.java deleted file mode 100644 index bdf138b31523d..0000000000000 --- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/fake/FakeStepContext.java +++ /dev/null @@ -1,39 +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.fn.harness.fake; - -import org.apache.beam.runners.core.StateInternals; -import org.apache.beam.runners.core.StepContext; -import org.apache.beam.runners.core.TimerInternals; - -/** - * A fake {@link StepContext} factory that performs no-ops. - */ -public class FakeStepContext implements StepContext { - - @Override - public StateInternals stateInternals() { - throw new UnsupportedOperationException(); - } - - @Override - public TimerInternals timerInternals() { - throw new UnsupportedOperationException(); - } -} diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/fake/package-info.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/fake/package-info.java deleted file mode 100644 index cd6eb023d3fa8..0000000000000 --- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/fake/package-info.java +++ /dev/null @@ -1,22 +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. - */ - -/** - * Fake implementations of bindings used with runners-core. - */ -package org.apache.beam.fn.harness.fake; From e5bdedd23208e484f6852eda44c59fb873645e8f Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Isma=C3=ABl=20Mej=C3=ADa?= Date: Fri, 25 Aug 2017 10:43:17 +0200 Subject: [PATCH 331/346] Fix code style issues for HBaseIO --- .../io/hbase/HBaseCoderProviderRegistrar.java | 8 +- .../org/apache/beam/sdk/io/hbase/HBaseIO.java | 1090 ++++++++--------- .../beam/sdk/io/hbase/HBaseMutationCoder.java | 27 +- .../beam/sdk/io/hbase/HBaseResultCoder.java | 6 +- .../beam/sdk/io/hbase/SerializableScan.java | 37 +- .../HBaseCoderProviderRegistrarTest.java | 4 +- .../apache/beam/sdk/io/hbase/HBaseIOTest.java | 814 ++++++------ .../sdk/io/hbase/HBaseMutationCoderTest.java | 4 +- .../sdk/io/hbase/HBaseResultCoderTest.java | 4 +- .../sdk/io/hbase/SerializableScanTest.java | 6 +- 10 files changed, 987 insertions(+), 1013 deletions(-) diff --git a/sdks/java/io/hbase/src/main/java/org/apache/beam/sdk/io/hbase/HBaseCoderProviderRegistrar.java b/sdks/java/io/hbase/src/main/java/org/apache/beam/sdk/io/hbase/HBaseCoderProviderRegistrar.java index 2973d1b2dc23d..f836ebe3551fe 100644 --- a/sdks/java/io/hbase/src/main/java/org/apache/beam/sdk/io/hbase/HBaseCoderProviderRegistrar.java +++ b/sdks/java/io/hbase/src/main/java/org/apache/beam/sdk/io/hbase/HBaseCoderProviderRegistrar.java @@ -26,15 +26,13 @@ import org.apache.beam.sdk.values.TypeDescriptor; import org.apache.hadoop.hbase.client.Result; -/** - * A {@link CoderProviderRegistrar} for standard types used with {@link HBaseIO}. - */ +/** A {@link CoderProviderRegistrar} for standard types used with {@link HBaseIO}. */ @AutoService(CoderProviderRegistrar.class) public class HBaseCoderProviderRegistrar implements CoderProviderRegistrar { @Override public List getCoderProviders() { return ImmutableList.of( - HBaseMutationCoder.getCoderProvider(), - CoderProviders.forCoder(TypeDescriptor.of(Result.class), HBaseResultCoder.of())); + HBaseMutationCoder.getCoderProvider(), + CoderProviders.forCoder(TypeDescriptor.of(Result.class), HBaseResultCoder.of())); } } diff --git a/sdks/java/io/hbase/src/main/java/org/apache/beam/sdk/io/hbase/HBaseIO.java b/sdks/java/io/hbase/src/main/java/org/apache/beam/sdk/io/hbase/HBaseIO.java index 7f58cef30310c..41ced93aea318 100644 --- a/sdks/java/io/hbase/src/main/java/org/apache/beam/sdk/io/hbase/HBaseIO.java +++ b/sdks/java/io/hbase/src/main/java/org/apache/beam/sdk/io/hbase/HBaseIO.java @@ -71,19 +71,19 @@ /** * A bounded source and sink for HBase. * - *

              For more information, see the online documentation at - * HBase. + *

              For more information, see the online documentation at HBase. * *

              Reading from HBase

              * - *

              The HBase source returns a set of rows from a single table, returning a - * {@code PCollection}. + *

              The HBase source returns a set of rows from a single table, returning a {@code + * PCollection}. * - *

              To configure a HBase source, you must supply a table id and a {@link Configuration} - * to identify the HBase instance. By default, {@link HBaseIO.Read} will read all rows in the - * table. The row range to be read can optionally be restricted using with a {@link Scan} object - * or using the {@link HBaseIO.Read#withKeyRange}, and a {@link Filter} using - * {@link HBaseIO.Read#withFilter}, for example: + *

              To configure a HBase source, you must supply a table id and a {@link Configuration} to + * identify the HBase instance. By default, {@link HBaseIO.Read} will read all rows in the table. + * The row range to be read can optionally be restricted using with a {@link Scan} object or using + * the {@link HBaseIO.Read#withKeyRange}, and a {@link Filter} using {@link + * HBaseIO.Read#withFilter}, for example: * *

              {@code
                * // Scan the entire table.
              @@ -118,12 +118,12 @@
                *
                * 

              Writing to HBase

              * - *

              The HBase sink executes a set of row mutations on a single table. It takes as input a - * {@link PCollection PCollection<Mutation>}, where each {@link Mutation} represents an - * idempotent transformation on a row. + *

              The HBase sink executes a set of row mutations on a single table. It takes as input a {@link + * PCollection PCollection<Mutation>}, where each {@link Mutation} represents an idempotent + * transformation on a row. * - *

              To configure a HBase sink, you must supply a table id and a {@link Configuration} - * to identify the HBase instance, for example: + *

              To configure a HBase sink, you must supply a table id and a {@link Configuration} to identify + * the HBase instance, for example: * *

              {@code
                * Configuration configuration = ...;
              @@ -137,605 +137,605 @@
                *
                * 

              Experimental

              * - *

              The design of the API for HBaseIO is currently related to the BigtableIO one, - * it can evolve or be different in some aspects, but the idea is that users can easily migrate - * from one to the other

              . + *

              The design of the API for HBaseIO is currently related to the BigtableIO one, it can evolve or + * be different in some aspects, but the idea is that users can easily migrate from one to the other + * . */ @Experimental(Experimental.Kind.SOURCE_SINK) public class HBaseIO { - private static final Logger LOG = LoggerFactory.getLogger(HBaseIO.class); - - /** Disallow construction of utility class. */ - private HBaseIO() { + private static final Logger LOG = LoggerFactory.getLogger(HBaseIO.class); + + /** Disallow construction of utility class. */ + private HBaseIO() {} + + /** + * Creates an uninitialized {@link HBaseIO.Read}. Before use, the {@code Read} must be initialized + * with a {@link HBaseIO.Read#withConfiguration(Configuration)} that specifies the HBase instance, + * and a {@link HBaseIO.Read#withTableId tableId} that specifies which table to read. A {@link + * Filter} may also optionally be specified using {@link HBaseIO.Read#withFilter}. + */ + @Experimental + public static Read read() { + return new Read(null, "", new SerializableScan(new Scan())); + } + + /** + * A {@link PTransform} that reads from HBase. See the class-level Javadoc on {@link HBaseIO} for + * more information. + * + * @see HBaseIO + */ + public static class Read extends PTransform> { + /** + * Returns a new {@link HBaseIO.Read} that will read from the HBase instance indicated by the + * given configuration. + */ + public Read withConfiguration(Configuration configuration) { + checkNotNull(configuration, "conf"); + return new Read(new SerializableConfiguration(configuration), tableId, serializableScan); } /** - * Creates an uninitialized {@link HBaseIO.Read}. Before use, the {@code Read} must be - * initialized with a - * {@link HBaseIO.Read#withConfiguration(Configuration)} that specifies - * the HBase instance, and a {@link HBaseIO.Read#withTableId tableId} that - * specifies which table to read. A {@link Filter} may also optionally be specified using - * {@link HBaseIO.Read#withFilter}. + * Returns a new {@link HBaseIO.Read} that will read from the specified table. + * + *

              Does not modify this object. */ - @Experimental - public static Read read() { - return new Read(null, "", new SerializableScan(new Scan())); + public Read withTableId(String tableId) { + checkNotNull(tableId, "tableId"); + return new Read(serializableConfiguration, tableId, serializableScan); } /** - * A {@link PTransform} that reads from HBase. See the class-level Javadoc on - * {@link HBaseIO} for more information. + * Returns a new {@link HBaseIO.Read} that will filter the rows read from HBase using the given + * scan. * - * @see HBaseIO + *

              Does not modify this object. */ - public static class Read extends PTransform> { - /** - * Returns a new {@link HBaseIO.Read} that will read from the HBase instance - * indicated by the given configuration. - */ - public Read withConfiguration(Configuration configuration) { - checkNotNull(configuration, "conf"); - return new Read(new SerializableConfiguration(configuration), - tableId, serializableScan); - } + public Read withScan(Scan scan) { + checkNotNull(scan, "scan"); + return new Read(serializableConfiguration, tableId, new SerializableScan(scan)); + } - /** - * Returns a new {@link HBaseIO.Read} that will read from the specified table. - * - *

              Does not modify this object. - */ - public Read withTableId(String tableId) { - checkNotNull(tableId, "tableId"); - return new Read(serializableConfiguration, tableId, serializableScan); - } + /** + * Returns a new {@link HBaseIO.Read} that will filter the rows read from HBase using the given + * row filter. + * + *

              Does not modify this object. + */ + public Read withFilter(Filter filter) { + checkNotNull(filter, "filter"); + return withScan(serializableScan.get().setFilter(filter)); + } - /** - * Returns a new {@link HBaseIO.Read} that will filter the rows read from HBase - * using the given scan. - * - *

              Does not modify this object. - */ - public Read withScan(Scan scan) { - checkNotNull(scan, "scan"); - return new Read(serializableConfiguration, tableId, new SerializableScan(scan)); - } + /** + * Returns a new {@link HBaseIO.Read} that will read only rows in the specified range. + * + *

              Does not modify this object. + */ + public Read withKeyRange(ByteKeyRange keyRange) { + checkNotNull(keyRange, "keyRange"); + byte[] startRow = keyRange.getStartKey().getBytes(); + byte[] stopRow = keyRange.getEndKey().getBytes(); + return withScan(serializableScan.get().setStartRow(startRow).setStopRow(stopRow)); + } - /** - * Returns a new {@link HBaseIO.Read} that will filter the rows read from HBase - * using the given row filter. - * - *

              Does not modify this object. - */ - public Read withFilter(Filter filter) { - checkNotNull(filter, "filter"); - return withScan(serializableScan.get().setFilter(filter)); - } + /** + * Returns a new {@link HBaseIO.Read} that will read only rows in the specified range. + * + *

              Does not modify this object. + */ + public Read withKeyRange(byte[] startRow, byte[] stopRow) { + checkNotNull(startRow, "startRow"); + checkNotNull(stopRow, "stopRow"); + ByteKeyRange keyRange = + ByteKeyRange.of(ByteKey.copyFrom(startRow), ByteKey.copyFrom(stopRow)); + return withKeyRange(keyRange); + } - /** - * Returns a new {@link HBaseIO.Read} that will read only rows in the specified range. - * - *

              Does not modify this object. - */ - public Read withKeyRange(ByteKeyRange keyRange) { - checkNotNull(keyRange, "keyRange"); - byte[] startRow = keyRange.getStartKey().getBytes(); - byte[] stopRow = keyRange.getEndKey().getBytes(); - return withScan(serializableScan.get().setStartRow(startRow).setStopRow(stopRow)); - } + private Read( + SerializableConfiguration serializableConfiguration, + String tableId, + SerializableScan serializableScan) { + this.serializableConfiguration = serializableConfiguration; + this.tableId = tableId; + this.serializableScan = serializableScan; + } - /** - * Returns a new {@link HBaseIO.Read} that will read only rows in the specified range. - * - *

              Does not modify this object. - */ - public Read withKeyRange(byte[] startRow, byte[] stopRow) { - checkNotNull(startRow, "startRow"); - checkNotNull(stopRow, "stopRow"); - ByteKeyRange keyRange = - ByteKeyRange.of(ByteKey.copyFrom(startRow), ByteKey.copyFrom(stopRow)); - return withKeyRange(keyRange); - } + @Override + public PCollection expand(PBegin input) { + HBaseSource source = new HBaseSource(this, null /* estimatedSizeBytes */); + return input.getPipeline().apply(org.apache.beam.sdk.io.Read.from(source)); + } - private Read(SerializableConfiguration serializableConfiguration, String tableId, - SerializableScan serializableScan) { - this.serializableConfiguration = serializableConfiguration; - this.tableId = tableId; - this.serializableScan = serializableScan; - } + @Override + public void validate(PipelineOptions options) { + checkArgument(serializableConfiguration != null, "Configuration not provided"); + checkArgument(!tableId.isEmpty(), "Table ID not specified"); + try (Connection connection = + ConnectionFactory.createConnection(serializableConfiguration.get())) { + Admin admin = connection.getAdmin(); + checkArgument( + admin.tableExists(TableName.valueOf(tableId)), "Table %s does not exist", tableId); + } catch (IOException e) { + LOG.warn("Error checking whether table {} exists; proceeding.", tableId, e); + } + } - @Override - public PCollection expand(PBegin input) { - HBaseSource source = new HBaseSource(this, null /* estimatedSizeBytes */); - return input.getPipeline().apply(org.apache.beam.sdk.io.Read.from(source)); - } + @Override + public void populateDisplayData(DisplayData.Builder builder) { + super.populateDisplayData(builder); + builder.add(DisplayData.item("configuration", serializableConfiguration.get().toString())); + builder.add(DisplayData.item("tableId", tableId)); + builder.addIfNotNull(DisplayData.item("scan", serializableScan.get().toString())); + } - @Override - public void validate(PipelineOptions options) { - checkArgument(serializableConfiguration != null, - "Configuration not provided"); - checkArgument(!tableId.isEmpty(), "Table ID not specified"); - try (Connection connection = ConnectionFactory.createConnection( - serializableConfiguration.get())) { - Admin admin = connection.getAdmin(); - checkArgument(admin.tableExists(TableName.valueOf(tableId)), - "Table %s does not exist", tableId); - } catch (IOException e) { - LOG.warn("Error checking whether table {} exists; proceeding.", tableId, e); - } - } + public String getTableId() { + return tableId; + } - @Override - public void populateDisplayData(DisplayData.Builder builder) { - super.populateDisplayData(builder); - builder.add(DisplayData.item("configuration", - serializableConfiguration.get().toString())); - builder.add(DisplayData.item("tableId", tableId)); - builder.addIfNotNull(DisplayData.item("scan", serializableScan.get().toString())); - } + public Configuration getConfiguration() { + return serializableConfiguration.get(); + } - public String getTableId() { - return tableId; - } + /** Returns the range of keys that will be read from the table. */ + public ByteKeyRange getKeyRange() { + byte[] startRow = serializableScan.get().getStartRow(); + byte[] stopRow = serializableScan.get().getStopRow(); + return ByteKeyRange.of(ByteKey.copyFrom(startRow), ByteKey.copyFrom(stopRow)); + } - public Configuration getConfiguration() { - return serializableConfiguration.get(); - } + private final SerializableConfiguration serializableConfiguration; + private final String tableId; + private final SerializableScan serializableScan; + } - /** - * Returns the range of keys that will be read from the table. - */ - public ByteKeyRange getKeyRange() { - byte[] startRow = serializableScan.get().getStartRow(); - byte[] stopRow = serializableScan.get().getStopRow(); - return ByteKeyRange.of(ByteKey.copyFrom(startRow), ByteKey.copyFrom(stopRow)); - } + static class HBaseSource extends BoundedSource { + private final Read read; + @Nullable private Long estimatedSizeBytes; - private final SerializableConfiguration serializableConfiguration; - private final String tableId; - private final SerializableScan serializableScan; + HBaseSource(Read read, @Nullable Long estimatedSizeBytes) { + this.read = read; + this.estimatedSizeBytes = estimatedSizeBytes; } - static class HBaseSource extends BoundedSource { - private final Read read; - @Nullable private Long estimatedSizeBytes; - - HBaseSource(Read read, @Nullable Long estimatedSizeBytes) { - this.read = read; - this.estimatedSizeBytes = estimatedSizeBytes; - } + HBaseSource withStartKey(ByteKey startKey) throws IOException { + checkNotNull(startKey, "startKey"); + Read newRead = + new Read( + read.serializableConfiguration, + read.tableId, + new SerializableScan( + new Scan(read.serializableScan.get()).setStartRow(startKey.getBytes()))); + return new HBaseSource(newRead, estimatedSizeBytes); + } - HBaseSource withStartKey(ByteKey startKey) throws IOException { - checkNotNull(startKey, "startKey"); - Read newRead = new Read(read.serializableConfiguration, read.tableId, - new SerializableScan( - new Scan(read.serializableScan.get()).setStartRow(startKey.getBytes()))); - return new HBaseSource(newRead, estimatedSizeBytes); - } + HBaseSource withEndKey(ByteKey endKey) throws IOException { + checkNotNull(endKey, "endKey"); + Read newRead = + new Read( + read.serializableConfiguration, + read.tableId, + new SerializableScan( + new Scan(read.serializableScan.get()).setStopRow(endKey.getBytes()))); + return new HBaseSource(newRead, estimatedSizeBytes); + } - HBaseSource withEndKey(ByteKey endKey) throws IOException { - checkNotNull(endKey, "endKey"); - Read newRead = new Read(read.serializableConfiguration, read.tableId, - new SerializableScan( - new Scan(read.serializableScan.get()).setStopRow(endKey.getBytes()))); - return new HBaseSource(newRead, estimatedSizeBytes); - } + @Override + public long getEstimatedSizeBytes(PipelineOptions pipelineOptions) throws Exception { + if (estimatedSizeBytes == null) { + estimatedSizeBytes = estimateSizeBytes(); + LOG.debug( + "Estimated size {} bytes for table {} and scan {}", + estimatedSizeBytes, + read.tableId, + read.serializableScan.get()); + } + return estimatedSizeBytes; + } - @Override - public long getEstimatedSizeBytes(PipelineOptions pipelineOptions) throws Exception { - if (estimatedSizeBytes == null) { - estimatedSizeBytes = estimateSizeBytes(); - LOG.debug("Estimated size {} bytes for table {} and scan {}", estimatedSizeBytes, - read.tableId, read.serializableScan.get()); + /** + * This estimates the real size, it can be the compressed size depending on the HBase + * configuration. + */ + private long estimateSizeBytes() throws Exception { + // This code is based on RegionSizeCalculator in hbase-server + long estimatedSizeBytes = 0L; + Configuration configuration = this.read.serializableConfiguration.get(); + try (Connection connection = ConnectionFactory.createConnection(configuration)) { + // filter regions for the given table/scan + List regionLocations = getRegionLocations(connection); + + // builds set of regions who are part of the table scan + Set tableRegions = new TreeSet<>(Bytes.BYTES_COMPARATOR); + for (HRegionLocation regionLocation : regionLocations) { + tableRegions.add(regionLocation.getRegionInfo().getRegionName()); + } + + // calculate estimated size for the regions + Admin admin = connection.getAdmin(); + ClusterStatus clusterStatus = admin.getClusterStatus(); + Collection servers = clusterStatus.getServers(); + for (ServerName serverName : servers) { + ServerLoad serverLoad = clusterStatus.getLoad(serverName); + for (RegionLoad regionLoad : serverLoad.getRegionsLoad().values()) { + byte[] regionId = regionLoad.getName(); + if (tableRegions.contains(regionId)) { + long regionSizeBytes = regionLoad.getStorefileSizeMB() * 1_048_576L; + estimatedSizeBytes += regionSizeBytes; } - return estimatedSizeBytes; + } } + } + return estimatedSizeBytes; + } - /** - * This estimates the real size, it can be the compressed size depending on the HBase - * configuration. - */ - private long estimateSizeBytes() throws Exception { - // This code is based on RegionSizeCalculator in hbase-server - long estimatedSizeBytes = 0L; - Configuration configuration = this.read.serializableConfiguration.get(); - try (Connection connection = ConnectionFactory.createConnection(configuration)) { - // filter regions for the given table/scan - List regionLocations = getRegionLocations(connection); - - // builds set of regions who are part of the table scan - Set tableRegions = new TreeSet<>(Bytes.BYTES_COMPARATOR); - for (HRegionLocation regionLocation : regionLocations) { - tableRegions.add(regionLocation.getRegionInfo().getRegionName()); - } - - // calculate estimated size for the regions - Admin admin = connection.getAdmin(); - ClusterStatus clusterStatus = admin.getClusterStatus(); - Collection servers = clusterStatus.getServers(); - for (ServerName serverName : servers) { - ServerLoad serverLoad = clusterStatus.getLoad(serverName); - for (RegionLoad regionLoad : serverLoad.getRegionsLoad().values()) { - byte[] regionId = regionLoad.getName(); - if (tableRegions.contains(regionId)) { - long regionSizeBytes = regionLoad.getStorefileSizeMB() * 1_048_576L; - estimatedSizeBytes += regionSizeBytes; - } - } - } - } - return estimatedSizeBytes; - } + private List getRegionLocations(Connection connection) throws Exception { + final Scan scan = read.serializableScan.get(); + byte[] startRow = scan.getStartRow(); + byte[] stopRow = scan.getStopRow(); - private List getRegionLocations(Connection connection) throws Exception { - final Scan scan = read.serializableScan.get(); - byte[] startRow = scan.getStartRow(); - byte[] stopRow = scan.getStopRow(); - - final List regionLocations = new ArrayList<>(); - - final boolean scanWithNoLowerBound = startRow.length == 0; - final boolean scanWithNoUpperBound = stopRow.length == 0; - - TableName tableName = TableName.valueOf(read.tableId); - RegionLocator regionLocator = connection.getRegionLocator(tableName); - List tableRegionInfos = regionLocator.getAllRegionLocations(); - for (HRegionLocation regionLocation : tableRegionInfos) { - final byte[] startKey = regionLocation.getRegionInfo().getStartKey(); - final byte[] endKey = regionLocation.getRegionInfo().getEndKey(); - boolean isLastRegion = endKey.length == 0; - // filters regions who are part of the scan - if ((scanWithNoLowerBound - || isLastRegion || Bytes.compareTo(startRow, endKey) < 0) - && (scanWithNoUpperBound || Bytes.compareTo(stopRow, startKey) > 0)) { - regionLocations.add(regionLocation); - } - } + final List regionLocations = new ArrayList<>(); - return regionLocations; - } + final boolean scanWithNoLowerBound = startRow.length == 0; + final boolean scanWithNoUpperBound = stopRow.length == 0; - private List - splitBasedOnRegions(List regionLocations, int numSplits) - throws Exception { - final Scan scan = read.serializableScan.get(); - byte[] startRow = scan.getStartRow(); - byte[] stopRow = scan.getStopRow(); - - final List sources = new ArrayList<>(numSplits); - final boolean scanWithNoLowerBound = startRow.length == 0; - final boolean scanWithNoUpperBound = stopRow.length == 0; - - for (HRegionLocation regionLocation : regionLocations) { - final byte[] startKey = regionLocation.getRegionInfo().getStartKey(); - final byte[] endKey = regionLocation.getRegionInfo().getEndKey(); - boolean isLastRegion = endKey.length == 0; - String host = regionLocation.getHostnamePort(); - - final byte[] splitStart = (scanWithNoLowerBound - || Bytes.compareTo(startKey, startRow) >= 0) ? startKey : startRow; - final byte[] splitStop = - (scanWithNoUpperBound || Bytes.compareTo(endKey, stopRow) <= 0) - && !isLastRegion ? endKey : stopRow; - - LOG.debug("{} {} {} {} {}", sources.size(), host, read.tableId, - Bytes.toString(splitStart), Bytes.toString(splitStop)); - - // We need to create a new copy of the scan and read to add the new ranges - Scan newScan = new Scan(scan).setStartRow(splitStart).setStopRow(splitStop); - Read newRead = new Read(read.serializableConfiguration, read.tableId, - new SerializableScan(newScan)); - sources.add(new HBaseSource(newRead, estimatedSizeBytes)); - } - return sources; + TableName tableName = TableName.valueOf(read.tableId); + RegionLocator regionLocator = connection.getRegionLocator(tableName); + List tableRegionInfos = regionLocator.getAllRegionLocations(); + for (HRegionLocation regionLocation : tableRegionInfos) { + final byte[] startKey = regionLocation.getRegionInfo().getStartKey(); + final byte[] endKey = regionLocation.getRegionInfo().getEndKey(); + boolean isLastRegion = endKey.length == 0; + // filters regions who are part of the scan + if ((scanWithNoLowerBound || isLastRegion || Bytes.compareTo(startRow, endKey) < 0) + && (scanWithNoUpperBound || Bytes.compareTo(stopRow, startKey) > 0)) { + regionLocations.add(regionLocation); } + } + + return regionLocations; + } + + private List splitBasedOnRegions( + List regionLocations, int numSplits) throws Exception { + final Scan scan = read.serializableScan.get(); + byte[] startRow = scan.getStartRow(); + byte[] stopRow = scan.getStopRow(); + + final List sources = new ArrayList<>(numSplits); + final boolean scanWithNoLowerBound = startRow.length == 0; + final boolean scanWithNoUpperBound = stopRow.length == 0; + + for (HRegionLocation regionLocation : regionLocations) { + final byte[] startKey = regionLocation.getRegionInfo().getStartKey(); + final byte[] endKey = regionLocation.getRegionInfo().getEndKey(); + boolean isLastRegion = endKey.length == 0; + String host = regionLocation.getHostnamePort(); + + final byte[] splitStart = + (scanWithNoLowerBound || Bytes.compareTo(startKey, startRow) >= 0) + ? startKey + : startRow; + final byte[] splitStop = + (scanWithNoUpperBound || Bytes.compareTo(endKey, stopRow) <= 0) && !isLastRegion + ? endKey + : stopRow; + + LOG.debug( + "{} {} {} {} {}", + sources.size(), + host, + read.tableId, + Bytes.toString(splitStart), + Bytes.toString(splitStop)); + + // We need to create a new copy of the scan and read to add the new ranges + Scan newScan = new Scan(scan).setStartRow(splitStart).setStopRow(splitStop); + Read newRead = + new Read(read.serializableConfiguration, read.tableId, new SerializableScan(newScan)); + sources.add(new HBaseSource(newRead, estimatedSizeBytes)); + } + return sources; + } @Override public List> split( long desiredBundleSizeBytes, PipelineOptions options) throws Exception { - LOG.debug("desiredBundleSize {} bytes", desiredBundleSizeBytes); - long estimatedSizeBytes = getEstimatedSizeBytes(options); - int numSplits = 1; - if (estimatedSizeBytes > 0 && desiredBundleSizeBytes > 0) { - numSplits = (int) Math.ceil((double) estimatedSizeBytes / desiredBundleSizeBytes); - } - - try (Connection connection = ConnectionFactory.createConnection( - read.getConfiguration())) { - List regionLocations = getRegionLocations(connection); - int realNumSplits = - numSplits < regionLocations.size() ? regionLocations.size() : numSplits; - LOG.debug("Suggested {} bundle(s) based on size", numSplits); - LOG.debug("Suggested {} bundle(s) based on number of regions", - regionLocations.size()); - final List sources = splitBasedOnRegions(regionLocations, - realNumSplits); - LOG.debug("Split into {} bundle(s)", sources.size()); - if (numSplits >= 1) { - return sources; - } - return Collections.singletonList(this); - } - } - - @Override - public BoundedReader createReader(PipelineOptions pipelineOptions) - throws IOException { - return new HBaseReader(this); - } - - @Override - public void validate() { - read.validate(null /* input */); - } + LOG.debug("desiredBundleSize {} bytes", desiredBundleSizeBytes); + long estimatedSizeBytes = getEstimatedSizeBytes(options); + int numSplits = 1; + if (estimatedSizeBytes > 0 && desiredBundleSizeBytes > 0) { + numSplits = (int) Math.ceil((double) estimatedSizeBytes / desiredBundleSizeBytes); + } + + try (Connection connection = ConnectionFactory.createConnection(read.getConfiguration())) { + List regionLocations = getRegionLocations(connection); + int realNumSplits = numSplits < regionLocations.size() ? regionLocations.size() : numSplits; + LOG.debug("Suggested {} bundle(s) based on size", numSplits); + LOG.debug("Suggested {} bundle(s) based on number of regions", regionLocations.size()); + final List sources = splitBasedOnRegions(regionLocations, realNumSplits); + LOG.debug("Split into {} bundle(s)", sources.size()); + if (numSplits >= 1) { + return sources; + } + return Collections.singletonList(this); + } + } - @Override - public void populateDisplayData(DisplayData.Builder builder) { - read.populateDisplayData(builder); - } + @Override + public BoundedReader createReader(PipelineOptions pipelineOptions) throws IOException { + return new HBaseReader(this); + } - @Override - public Coder getOutputCoder() { - return HBaseResultCoder.of(); - } + @Override + public void validate() { + read.validate(null /* input */); } - private static class HBaseReader extends BoundedSource.BoundedReader { - private HBaseSource source; - private Connection connection; - private ResultScanner scanner; - private Iterator iter; - private Result current; - private final ByteKeyRangeTracker rangeTracker; - private long recordsReturned; - - HBaseReader(HBaseSource source) { - this.source = source; - Scan scan = source.read.serializableScan.get(); - ByteKeyRange range = ByteKeyRange - .of(ByteKey.copyFrom(scan.getStartRow()), ByteKey.copyFrom(scan.getStopRow())); - rangeTracker = ByteKeyRangeTracker.of(range); - } + @Override + public void populateDisplayData(DisplayData.Builder builder) { + read.populateDisplayData(builder); + } - @Override - public boolean start() throws IOException { - HBaseSource source = getCurrentSource(); - Configuration configuration = source.read.serializableConfiguration.get(); - String tableId = source.read.tableId; - connection = ConnectionFactory.createConnection(configuration); - TableName tableName = TableName.valueOf(tableId); - Table table = connection.getTable(tableName); - // [BEAM-2319] We have to clone the Scan because the underlying scanner may mutate it. - Scan scanClone = new Scan(source.read.serializableScan.get()); - scanner = table.getScanner(scanClone); - iter = scanner.iterator(); - return advance(); - } + @Override + public Coder getOutputCoder() { + return HBaseResultCoder.of(); + } + } + + private static class HBaseReader extends BoundedSource.BoundedReader { + private HBaseSource source; + private Connection connection; + private ResultScanner scanner; + private Iterator iter; + private Result current; + private final ByteKeyRangeTracker rangeTracker; + private long recordsReturned; + + HBaseReader(HBaseSource source) { + this.source = source; + Scan scan = source.read.serializableScan.get(); + ByteKeyRange range = + ByteKeyRange.of( + ByteKey.copyFrom(scan.getStartRow()), ByteKey.copyFrom(scan.getStopRow())); + rangeTracker = ByteKeyRangeTracker.of(range); + } - @Override - public Result getCurrent() throws NoSuchElementException { - return current; - } + @Override + public boolean start() throws IOException { + HBaseSource source = getCurrentSource(); + Configuration configuration = source.read.serializableConfiguration.get(); + String tableId = source.read.tableId; + connection = ConnectionFactory.createConnection(configuration); + TableName tableName = TableName.valueOf(tableId); + Table table = connection.getTable(tableName); + // [BEAM-2319] We have to clone the Scan because the underlying scanner may mutate it. + Scan scanClone = new Scan(source.read.serializableScan.get()); + scanner = table.getScanner(scanClone); + iter = scanner.iterator(); + return advance(); + } - @Override - public boolean advance() throws IOException { - if (!iter.hasNext()) { - return rangeTracker.markDone(); - } - final Result next = iter.next(); - boolean hasRecord = - rangeTracker.tryReturnRecordAt(true, ByteKey.copyFrom(next.getRow())) - || rangeTracker.markDone(); - if (hasRecord) { - current = next; - ++recordsReturned; - } - return hasRecord; - } + @Override + public Result getCurrent() throws NoSuchElementException { + return current; + } - @Override - public void close() throws IOException { - LOG.debug("Closing reader after reading {} records.", recordsReturned); - if (scanner != null) { - scanner.close(); - scanner = null; - } - if (connection != null) { - connection.close(); - connection = null; - } - } + @Override + public boolean advance() throws IOException { + if (!iter.hasNext()) { + return rangeTracker.markDone(); + } + final Result next = iter.next(); + boolean hasRecord = + rangeTracker.tryReturnRecordAt(true, ByteKey.copyFrom(next.getRow())) + || rangeTracker.markDone(); + if (hasRecord) { + current = next; + ++recordsReturned; + } + return hasRecord; + } - @Override - public synchronized HBaseSource getCurrentSource() { - return source; - } + @Override + public void close() throws IOException { + LOG.debug("Closing reader after reading {} records.", recordsReturned); + if (scanner != null) { + scanner.close(); + scanner = null; + } + if (connection != null) { + connection.close(); + connection = null; + } + } - @Override - public final Double getFractionConsumed() { - return rangeTracker.getFractionConsumed(); - } + @Override + public synchronized HBaseSource getCurrentSource() { + return source; + } - @Override - public final long getSplitPointsConsumed() { - return rangeTracker.getSplitPointsConsumed(); - } + @Override + public final Double getFractionConsumed() { + return rangeTracker.getFractionConsumed(); + } - @Override - @Nullable - public final synchronized HBaseSource splitAtFraction(double fraction) { - ByteKey splitKey; - try { - splitKey = rangeTracker.getRange().interpolateKey(fraction); - } catch (RuntimeException e) { - LOG.info("{}: Failed to interpolate key for fraction {}.", rangeTracker.getRange(), - fraction, e); - return null; - } - LOG.info( - "Proposing to split {} at fraction {} (key {})", rangeTracker, fraction, splitKey); - HBaseSource primary; - HBaseSource residual; - try { - primary = source.withEndKey(splitKey); - residual = source.withStartKey(splitKey); - } catch (Exception e) { - LOG.info( - "{}: Interpolating for fraction {} yielded invalid split key {}.", - rangeTracker.getRange(), - fraction, - splitKey, - e); - return null; - } - if (!rangeTracker.trySplitAtPosition(splitKey)) { - return null; - } - this.source = primary; - return residual; - } + @Override + public final long getSplitPointsConsumed() { + return rangeTracker.getSplitPointsConsumed(); } + @Override + @Nullable + public final synchronized HBaseSource splitAtFraction(double fraction) { + ByteKey splitKey; + try { + splitKey = rangeTracker.getRange().interpolateKey(fraction); + } catch (RuntimeException e) { + LOG.info( + "{}: Failed to interpolate key for fraction {}.", rangeTracker.getRange(), fraction, e); + return null; + } + LOG.info("Proposing to split {} at fraction {} (key {})", rangeTracker, fraction, splitKey); + HBaseSource primary; + HBaseSource residual; + try { + primary = source.withEndKey(splitKey); + residual = source.withStartKey(splitKey); + } catch (Exception e) { + LOG.info( + "{}: Interpolating for fraction {} yielded invalid split key {}.", + rangeTracker.getRange(), + fraction, + splitKey, + e); + return null; + } + if (!rangeTracker.trySplitAtPosition(splitKey)) { + return null; + } + this.source = primary; + return residual; + } + } + + /** + * Creates an uninitialized {@link HBaseIO.Write}. Before use, the {@code Write} must be + * initialized with a {@link HBaseIO.Write#withConfiguration(Configuration)} that specifies the + * destination HBase instance, and a {@link HBaseIO.Write#withTableId tableId} that specifies + * which table to write. + */ + public static Write write() { + return new Write(null /* SerializableConfiguration */, ""); + } + + /** + * A {@link PTransform} that writes to HBase. See the class-level Javadoc on {@link HBaseIO} for + * more information. + * + * @see HBaseIO + */ + public static class Write extends PTransform, PDone> { /** - * Creates an uninitialized {@link HBaseIO.Write}. Before use, the {@code Write} must be - * initialized with a - * {@link HBaseIO.Write#withConfiguration(Configuration)} that specifies - * the destination HBase instance, and a {@link HBaseIO.Write#withTableId tableId} - * that specifies which table to write. + * Returns a new {@link HBaseIO.Write} that will write to the HBase instance indicated by the + * given Configuration, and using any other specified customizations. + * + *

              Does not modify this object. */ - public static Write write() { - return new Write(null /* SerializableConfiguration */, ""); + public Write withConfiguration(Configuration configuration) { + checkNotNull(configuration, "conf"); + return new Write(new SerializableConfiguration(configuration), tableId); } /** - * A {@link PTransform} that writes to HBase. See the class-level Javadoc on - * {@link HBaseIO} for more information. + * Returns a new {@link HBaseIO.Write} that will write to the specified table. * - * @see HBaseIO + *

              Does not modify this object. */ - public static class Write extends PTransform, PDone> { - /** - * Returns a new {@link HBaseIO.Write} that will write to the HBase instance - * indicated by the given Configuration, and using any other specified customizations. - * - *

              Does not modify this object. - */ - public Write withConfiguration(Configuration configuration) { - checkNotNull(configuration, "conf"); - return new Write(new SerializableConfiguration(configuration), tableId); - } - - /** - * Returns a new {@link HBaseIO.Write} that will write to the specified table. - * - *

              Does not modify this object. - */ - public Write withTableId(String tableId) { - checkNotNull(tableId, "tableId"); - return new Write(serializableConfiguration, tableId); - } - - private Write(SerializableConfiguration serializableConfiguration, String tableId) { - this.serializableConfiguration = serializableConfiguration; - this.tableId = tableId; - } - - @Override - public PDone expand(PCollection input) { - input.apply(ParDo.of(new HBaseWriterFn(tableId, serializableConfiguration))); - return PDone.in(input.getPipeline()); - } - - @Override - public void validate(PipelineOptions options) { - checkArgument(serializableConfiguration != null, "Configuration not specified"); - checkArgument(!tableId.isEmpty(), "Table ID not specified"); - try (Connection connection = ConnectionFactory.createConnection( - serializableConfiguration.get())) { - Admin admin = connection.getAdmin(); - checkArgument(admin.tableExists(TableName.valueOf(tableId)), - "Table %s does not exist", tableId); - } catch (IOException e) { - LOG.warn("Error checking whether table {} exists; proceeding.", tableId, e); - } - } - - @Override - public void populateDisplayData(DisplayData.Builder builder) { - super.populateDisplayData(builder); - builder.add(DisplayData.item("configuration", - serializableConfiguration.get().toString())); - builder.add(DisplayData.item("tableId", tableId)); - } - - public String getTableId() { - return tableId; - } - - public Configuration getConfiguration() { - return serializableConfiguration.get(); - } - - private final String tableId; - private final SerializableConfiguration serializableConfiguration; - - private class HBaseWriterFn extends DoFn { - - public HBaseWriterFn(String tableId, - SerializableConfiguration serializableConfiguration) { - this.tableId = checkNotNull(tableId, "tableId"); - this.serializableConfiguration = checkNotNull(serializableConfiguration, - "serializableConfiguration"); - } - - @Setup - public void setup() throws Exception { - connection = ConnectionFactory.createConnection(serializableConfiguration.get()); - } - - @StartBundle - public void startBundle(StartBundleContext c) throws IOException { - BufferedMutatorParams params = - new BufferedMutatorParams(TableName.valueOf(tableId)); - mutator = connection.getBufferedMutator(params); - recordsWritten = 0; - } + public Write withTableId(String tableId) { + checkNotNull(tableId, "tableId"); + return new Write(serializableConfiguration, tableId); + } - @ProcessElement - public void processElement(ProcessContext c) throws Exception { - mutator.mutate(c.element()); - ++recordsWritten; - } + private Write(SerializableConfiguration serializableConfiguration, String tableId) { + this.serializableConfiguration = serializableConfiguration; + this.tableId = tableId; + } - @FinishBundle - public void finishBundle() throws Exception { - mutator.flush(); - LOG.debug("Wrote {} records", recordsWritten); - } + @Override + public PDone expand(PCollection input) { + input.apply(ParDo.of(new HBaseWriterFn(tableId, serializableConfiguration))); + return PDone.in(input.getPipeline()); + } - @Teardown - public void tearDown() throws Exception { - if (mutator != null) { - mutator.close(); - mutator = null; - } - if (connection != null) { - connection.close(); - connection = null; - } - } + @Override + public void validate(PipelineOptions options) { + checkArgument(serializableConfiguration != null, "Configuration not specified"); + checkArgument(!tableId.isEmpty(), "Table ID not specified"); + try (Connection connection = + ConnectionFactory.createConnection(serializableConfiguration.get())) { + Admin admin = connection.getAdmin(); + checkArgument( + admin.tableExists(TableName.valueOf(tableId)), "Table %s does not exist", tableId); + } catch (IOException e) { + LOG.warn("Error checking whether table {} exists; proceeding.", tableId, e); + } + } - @Override - public void populateDisplayData(DisplayData.Builder builder) { - builder.delegate(Write.this); - } + @Override + public void populateDisplayData(DisplayData.Builder builder) { + super.populateDisplayData(builder); + builder.add(DisplayData.item("configuration", serializableConfiguration.get().toString())); + builder.add(DisplayData.item("tableId", tableId)); + } - private final String tableId; - private final SerializableConfiguration serializableConfiguration; + public String getTableId() { + return tableId; + } - private Connection connection; - private BufferedMutator mutator; + public Configuration getConfiguration() { + return serializableConfiguration.get(); + } - private long recordsWritten; - } + private final String tableId; + private final SerializableConfiguration serializableConfiguration; + + private class HBaseWriterFn extends DoFn { + + public HBaseWriterFn(String tableId, SerializableConfiguration serializableConfiguration) { + this.tableId = checkNotNull(tableId, "tableId"); + this.serializableConfiguration = + checkNotNull(serializableConfiguration, "serializableConfiguration"); + } + + @Setup + public void setup() throws Exception { + connection = ConnectionFactory.createConnection(serializableConfiguration.get()); + } + + @StartBundle + public void startBundle(StartBundleContext c) throws IOException { + BufferedMutatorParams params = new BufferedMutatorParams(TableName.valueOf(tableId)); + mutator = connection.getBufferedMutator(params); + recordsWritten = 0; + } + + @ProcessElement + public void processElement(ProcessContext c) throws Exception { + mutator.mutate(c.element()); + ++recordsWritten; + } + + @FinishBundle + public void finishBundle() throws Exception { + mutator.flush(); + LOG.debug("Wrote {} records", recordsWritten); + } + + @Teardown + public void tearDown() throws Exception { + if (mutator != null) { + mutator.close(); + mutator = null; + } + if (connection != null) { + connection.close(); + connection = null; + } + } + + @Override + public void populateDisplayData(DisplayData.Builder builder) { + builder.delegate(Write.this); + } + + private final String tableId; + private final SerializableConfiguration serializableConfiguration; + + private Connection connection; + private BufferedMutator mutator; + + private long recordsWritten; } + } } diff --git a/sdks/java/io/hbase/src/main/java/org/apache/beam/sdk/io/hbase/HBaseMutationCoder.java b/sdks/java/io/hbase/src/main/java/org/apache/beam/sdk/io/hbase/HBaseMutationCoder.java index ee83114d3b41b..e7a36d5392994 100644 --- a/sdks/java/io/hbase/src/main/java/org/apache/beam/sdk/io/hbase/HBaseMutationCoder.java +++ b/sdks/java/io/hbase/src/main/java/org/apache/beam/sdk/io/hbase/HBaseMutationCoder.java @@ -71,30 +71,29 @@ private static MutationType getType(Mutation mutation) { } /** - * Returns a {@link CoderProvider} which uses the {@link HBaseMutationCoder} for - * {@link Mutation mutations}. + * Returns a {@link CoderProvider} which uses the {@link HBaseMutationCoder} for {@link Mutation + * mutations}. */ static CoderProvider getCoderProvider() { return HBASE_MUTATION_CODER_PROVIDER; } private static final CoderProvider HBASE_MUTATION_CODER_PROVIDER = - new HBaseMutationCoderProvider(); + new HBaseMutationCoderProvider(); - /** - * A {@link CoderProvider} for {@link Mutation mutations}. - */ + /** A {@link CoderProvider} for {@link Mutation mutations}. */ private static class HBaseMutationCoderProvider extends CoderProvider { @Override - public Coder coderFor(TypeDescriptor typeDescriptor, - List> componentCoders) throws CannotProvideCoderException { + public Coder coderFor( + TypeDescriptor typeDescriptor, List> componentCoders) + throws CannotProvideCoderException { if (!typeDescriptor.isSubtypeOf(HBASE_MUTATION_TYPE_DESCRIPTOR)) { throw new CannotProvideCoderException( - String.format( - "Cannot provide %s because %s is not a subclass of %s", - HBaseMutationCoder.class.getSimpleName(), - typeDescriptor, - Mutation.class.getName())); + String.format( + "Cannot provide %s because %s is not a subclass of %s", + HBaseMutationCoder.class.getSimpleName(), + typeDescriptor, + Mutation.class.getName())); } try { @@ -106,5 +105,5 @@ public Coder coderFor(TypeDescriptor typeDescriptor, } private static final TypeDescriptor HBASE_MUTATION_TYPE_DESCRIPTOR = - new TypeDescriptor() {}; + new TypeDescriptor() {}; } diff --git a/sdks/java/io/hbase/src/main/java/org/apache/beam/sdk/io/hbase/HBaseResultCoder.java b/sdks/java/io/hbase/src/main/java/org/apache/beam/sdk/io/hbase/HBaseResultCoder.java index 1d066351d1aab..bce1567059ac3 100644 --- a/sdks/java/io/hbase/src/main/java/org/apache/beam/sdk/io/hbase/HBaseResultCoder.java +++ b/sdks/java/io/hbase/src/main/java/org/apache/beam/sdk/io/hbase/HBaseResultCoder.java @@ -41,14 +41,12 @@ public static HBaseResultCoder of() { } @Override - public void encode(Result value, OutputStream outputStream) - throws IOException { + public void encode(Result value, OutputStream outputStream) throws IOException { ProtobufUtil.toResult(value).writeDelimitedTo(outputStream); } @Override - public Result decode(InputStream inputStream) - throws IOException { + public Result decode(InputStream inputStream) throws IOException { return ProtobufUtil.toResult(ClientProtos.Result.parseDelimitedFrom(inputStream)); } } diff --git a/sdks/java/io/hbase/src/main/java/org/apache/beam/sdk/io/hbase/SerializableScan.java b/sdks/java/io/hbase/src/main/java/org/apache/beam/sdk/io/hbase/SerializableScan.java index f3bc7acb40509..6ed3c51e7e5c6 100644 --- a/sdks/java/io/hbase/src/main/java/org/apache/beam/sdk/io/hbase/SerializableScan.java +++ b/sdks/java/io/hbase/src/main/java/org/apache/beam/sdk/io/hbase/SerializableScan.java @@ -25,31 +25,28 @@ import org.apache.hadoop.hbase.protobuf.ProtobufUtil; import org.apache.hadoop.hbase.protobuf.generated.ClientProtos; -/** - * This is just a wrapper class to serialize HBase {@link Scan} using Protobuf. - */ +/** This is just a wrapper class to serialize HBase {@link Scan} using Protobuf. */ class SerializableScan implements Serializable { - private transient Scan scan; + private transient Scan scan; - public SerializableScan() { - } + public SerializableScan() {} - public SerializableScan(Scan scan) { - if (scan == null) { - throw new NullPointerException("Scan must not be null."); - } - this.scan = scan; + public SerializableScan(Scan scan) { + if (scan == null) { + throw new NullPointerException("Scan must not be null."); } + this.scan = scan; + } - private void writeObject(ObjectOutputStream out) throws IOException { - ProtobufUtil.toScan(scan).writeDelimitedTo(out); - } + private void writeObject(ObjectOutputStream out) throws IOException { + ProtobufUtil.toScan(scan).writeDelimitedTo(out); + } - private void readObject(ObjectInputStream in) throws IOException { - scan = ProtobufUtil.toScan(ClientProtos.Scan.parseDelimitedFrom(in)); - } + private void readObject(ObjectInputStream in) throws IOException { + scan = ProtobufUtil.toScan(ClientProtos.Scan.parseDelimitedFrom(in)); + } - public Scan get() { - return scan; - } + public Scan get() { + return scan; + } } diff --git a/sdks/java/io/hbase/src/test/java/org/apache/beam/sdk/io/hbase/HBaseCoderProviderRegistrarTest.java b/sdks/java/io/hbase/src/test/java/org/apache/beam/sdk/io/hbase/HBaseCoderProviderRegistrarTest.java index 5b2e13861bf64..25369fc50a20e 100644 --- a/sdks/java/io/hbase/src/test/java/org/apache/beam/sdk/io/hbase/HBaseCoderProviderRegistrarTest.java +++ b/sdks/java/io/hbase/src/test/java/org/apache/beam/sdk/io/hbase/HBaseCoderProviderRegistrarTest.java @@ -26,9 +26,7 @@ import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -/** - * Tests for {@link HBaseCoderProviderRegistrar}. - */ +/** Tests for {@link HBaseCoderProviderRegistrar}. */ @RunWith(JUnit4.class) public class HBaseCoderProviderRegistrarTest { @Test diff --git a/sdks/java/io/hbase/src/test/java/org/apache/beam/sdk/io/hbase/HBaseIOTest.java b/sdks/java/io/hbase/src/test/java/org/apache/beam/sdk/io/hbase/HBaseIOTest.java index 0b7f203b1c5fb..e6f7ac4ee5e63 100644 --- a/sdks/java/io/hbase/src/test/java/org/apache/beam/sdk/io/hbase/HBaseIOTest.java +++ b/sdks/java/io/hbase/src/test/java/org/apache/beam/sdk/io/hbase/HBaseIOTest.java @@ -75,418 +75,408 @@ import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -/** - * Test HBaseIO. - */ +/** Test HBaseIO. */ @RunWith(JUnit4.class) public class HBaseIOTest { - @Rule public final transient TestPipeline p = TestPipeline.create(); - @Rule public ExpectedException thrown = ExpectedException.none(); - - private static HBaseTestingUtility htu; - private static HBaseAdmin admin; - - private static final Configuration conf = HBaseConfiguration.create(); - private static final byte[] COLUMN_FAMILY = Bytes.toBytes("info"); - private static final byte[] COLUMN_NAME = Bytes.toBytes("name"); - private static final byte[] COLUMN_EMAIL = Bytes.toBytes("email"); - - @BeforeClass - public static void beforeClass() throws Exception { - conf.setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 1); - // Try to bind the hostname to localhost to solve an issue when it is not configured or - // no DNS resolution available. - conf.setStrings("hbase.master.hostname", "localhost"); - conf.setStrings("hbase.regionserver.hostname", "localhost"); - htu = new HBaseTestingUtility(conf); - - // We don't use the full htu.startMiniCluster() to avoid starting unneeded HDFS/MR daemons - htu.startMiniZKCluster(); - MiniHBaseCluster hbm = htu.startMiniHBaseCluster(1, 4); - hbm.waitForActiveAndReadyMaster(); - - admin = htu.getHBaseAdmin(); - } - - @AfterClass - public static void afterClass() throws Exception { - if (admin != null) { - admin.close(); - admin = null; - } - if (htu != null) { - htu.shutdownMiniHBaseCluster(); - htu.shutdownMiniZKCluster(); - htu = null; - } - } - - @Test - public void testReadBuildsCorrectly() { - HBaseIO.Read read = HBaseIO.read().withConfiguration(conf).withTableId("table"); - assertEquals("table", read.getTableId()); - assertNotNull("configuration", read.getConfiguration()); - } - - @Test - public void testReadBuildsCorrectlyInDifferentOrder() { - HBaseIO.Read read = HBaseIO.read().withTableId("table").withConfiguration(conf); - assertEquals("table", read.getTableId()); - assertNotNull("configuration", read.getConfiguration()); - } - - @Test - public void testWriteBuildsCorrectly() { - HBaseIO.Write write = HBaseIO.write().withConfiguration(conf).withTableId("table"); - assertEquals("table", write.getTableId()); - assertNotNull("configuration", write.getConfiguration()); - } - - @Test - public void testWriteBuildsCorrectlyInDifferentOrder() { - HBaseIO.Write write = HBaseIO.write().withTableId("table").withConfiguration(conf); - assertEquals("table", write.getTableId()); - assertNotNull("configuration", write.getConfiguration()); - } - - @Test - public void testWriteValidationFailsMissingTable() { - HBaseIO.Write write = HBaseIO.write().withConfiguration(conf); - thrown.expect(IllegalArgumentException.class); - write.validate(null /* input */); - } - - @Test - public void testWriteValidationFailsMissingConfiguration() { - HBaseIO.Write write = HBaseIO.write().withTableId("table"); - thrown.expect(IllegalArgumentException.class); - write.validate(null /* input */); - } - - /** Tests that when reading from a non-existent table, the read fails. */ - @Test - public void testReadingFailsTableDoesNotExist() throws Exception { - final String table = "TEST-TABLE-INVALID"; - // Exception will be thrown by read.validate() when read is applied. - thrown.expect(IllegalArgumentException.class); - thrown.expectMessage(String.format("Table %s does not exist", table)); - runReadTest(HBaseIO.read().withConfiguration(conf).withTableId(table), - new ArrayList()); - } - - /** Tests that when reading from an empty table, the read succeeds. */ - @Test - public void testReadingEmptyTable() throws Exception { - final String table = "TEST-EMPTY-TABLE"; - createTable(table); - runReadTest(HBaseIO.read().withConfiguration(conf).withTableId(table), - new ArrayList()); - } - - @Test - public void testReading() throws Exception { - final String table = "TEST-MANY-ROWS-TABLE"; - final int numRows = 1001; - createTable(table); - writeData(table, numRows); - runReadTestLength(HBaseIO.read().withConfiguration(conf).withTableId(table), 1001); - } - - /** Tests reading all rows from a split table. */ - @Test - public void testReadingWithSplits() throws Exception { - final String table = "TEST-MANY-ROWS-SPLITS-TABLE"; - final int numRows = 1500; - final int numRegions = 4; - final long bytesPerRow = 100L; - - // Set up test table data and sample row keys for size estimation and splitting. - createTable(table); - writeData(table, numRows); - - HBaseIO.Read read = HBaseIO.read().withConfiguration(conf).withTableId(table); - HBaseSource source = new HBaseSource(read, null /* estimatedSizeBytes */); - List> splits = - source.split(numRows * bytesPerRow / numRegions, - null /* options */); - - // Test num splits and split equality. - assertThat(splits, hasSize(4)); - assertSourcesEqualReferenceSource(source, splits, null /* options */); - } - - /** Tests that a {@link HBaseSource} can be read twice, verifying its immutability. */ - @Test - public void testReadingSourceTwice() throws Exception { - final String table = "TEST-READING-TWICE"; - final int numRows = 10; - - // Set up test table data and sample row keys for size estimation and splitting. - createTable(table); - writeData(table, numRows); - - HBaseIO.Read read = HBaseIO.read().withConfiguration(conf).withTableId(table); - HBaseSource source = new HBaseSource(read, null /* estimatedSizeBytes */); - assertThat(SourceTestUtils.readFromSource(source, null), hasSize(numRows)); - // second read. - assertThat(SourceTestUtils.readFromSource(source, null), hasSize(numRows)); - } - - /** Tests reading all rows using a filter. */ - @Test - public void testReadingWithFilter() throws Exception { - final String table = "TEST-FILTER-TABLE"; - final int numRows = 1001; - - createTable(table); - writeData(table, numRows); - - String regex = ".*17.*"; - Filter filter = new RowFilter(CompareFilter.CompareOp.EQUAL, - new RegexStringComparator(regex)); - HBaseIO.Read read = - HBaseIO.read().withConfiguration(conf).withTableId(table).withFilter(filter); - runReadTestLength(read, 20); - } - - /** - * Tests reading all rows using key ranges. Tests a prefix [), a suffix (], and a restricted - * range [] and that some properties hold across them. - */ - @Test - public void testReadingWithKeyRange() throws Exception { - final String table = "TEST-KEY-RANGE-TABLE"; - final int numRows = 1001; - final byte[] startRow = "2".getBytes(); - final byte[] stopRow = "9".getBytes(); - final ByteKey startKey = ByteKey.copyFrom(startRow); - - createTable(table); - writeData(table, numRows); - - // Test prefix: [beginning, startKey). - final ByteKeyRange prefixRange = ByteKeyRange.ALL_KEYS.withEndKey(startKey); - runReadTestLength(HBaseIO.read().withConfiguration(conf).withTableId(table) - .withKeyRange(prefixRange), 126); - - // Test suffix: [startKey, end). - final ByteKeyRange suffixRange = ByteKeyRange.ALL_KEYS.withStartKey(startKey); - runReadTestLength(HBaseIO.read().withConfiguration(conf).withTableId(table) - .withKeyRange(suffixRange), 875); - - // Test restricted range: [startKey, endKey). - // This one tests the second signature of .withKeyRange - runReadTestLength(HBaseIO.read().withConfiguration(conf).withTableId(table) - .withKeyRange(startRow, stopRow), 441); - } - - /** - * Tests dynamic work rebalancing exhaustively. - */ - @Test - public void testReadingSplitAtFractionExhaustive() throws Exception { - final String table = "TEST-FEW-ROWS-SPLIT-EXHAUSTIVE-TABLE"; - final int numRows = 7; - - createTable(table); - writeData(table, numRows); - - HBaseIO.Read read = HBaseIO.read().withConfiguration(conf).withTableId(table); - HBaseSource source = new HBaseSource(read, null /* estimatedSizeBytes */) - .withStartKey(ByteKey.of(48)).withEndKey(ByteKey.of(58)); - - assertSplitAtFractionExhaustive(source, null); - } - - /** - * Unit tests of splitAtFraction. - */ - @Test - public void testReadingSplitAtFraction() throws Exception { - final String table = "TEST-SPLIT-AT-FRACTION"; - final int numRows = 10; - - createTable(table); - writeData(table, numRows); - - HBaseIO.Read read = HBaseIO.read().withConfiguration(conf).withTableId(table); - HBaseSource source = new HBaseSource(read, null /* estimatedSizeBytes */); - - // The value k is based on the partitioning schema for the data, in this test case, - // the partitioning is HEX-based, so we start from 1/16m and the value k will be - // around 1/256, so the tests are done in approximately k ~= 0.003922 steps - double k = 0.003922; - - assertSplitAtFractionFails(source, 0, k, null /* options */); - assertSplitAtFractionFails(source, 0, 1.0, null /* options */); - // With 1 items read, all split requests past k will succeed. - assertSplitAtFractionSucceedsAndConsistent(source, 1, k, null /* options */); - assertSplitAtFractionSucceedsAndConsistent(source, 1, 0.666, null /* options */); - // With 3 items read, all split requests past 3k will succeed. - assertSplitAtFractionFails(source, 3, 2 * k, null /* options */); - assertSplitAtFractionSucceedsAndConsistent(source, 3, 3 * k, null /* options */); - assertSplitAtFractionSucceedsAndConsistent(source, 3, 4 * k, null /* options */); - // With 6 items read, all split requests past 6k will succeed. - assertSplitAtFractionFails(source, 6, 5 * k, null /* options */); - assertSplitAtFractionSucceedsAndConsistent(source, 6, 0.7, null /* options */); - } - - @Test - public void testReadingDisplayData() { - HBaseIO.Read read = HBaseIO.read().withConfiguration(conf).withTableId("fooTable"); - DisplayData displayData = DisplayData.from(read); - assertThat(displayData, hasDisplayItem("tableId", "fooTable")); - assertThat(displayData, hasDisplayItem("configuration")); - } - - /** Tests that a record gets written to the service and messages are logged. */ - @Test - public void testWriting() throws Exception { - final String table = "table"; - final String key = "key"; - final String value = "value"; - final int numMutations = 100; - - createTable(table); - - p.apply("multiple rows", Create.of(makeMutations(key, value, numMutations))) - .apply("write", HBaseIO.write().withConfiguration(conf).withTableId(table)); - p.run().waitUntilFinish(); - - List results = readTable(table, new Scan()); - assertEquals(numMutations, results.size()); - } - - /** Tests that when writing to a non-existent table, the write fails. */ - @Test - public void testWritingFailsTableDoesNotExist() throws Exception { - final String table = "TEST-TABLE-DOES-NOT-EXIST"; - - p.apply(Create.empty(HBaseMutationCoder.of())) - .apply("write", HBaseIO.write().withConfiguration(conf).withTableId(table)); - - // Exception will be thrown by write.validate() when write is applied. - thrown.expect(IllegalArgumentException.class); - thrown.expectMessage(String.format("Table %s does not exist", table)); - p.run(); - } - - /** Tests that when writing an element fails, the write fails. */ - @Test - public void testWritingFailsBadElement() throws Exception { - final String table = "TEST-TABLE-BAD-ELEMENT"; - final String key = "KEY"; - createTable(table); - - p.apply(Create.of(makeBadMutation(key))) - .apply(HBaseIO.write().withConfiguration(conf).withTableId(table)); - - thrown.expect(Pipeline.PipelineExecutionException.class); - thrown.expectCause(Matchers.instanceOf(IllegalArgumentException.class)); - thrown.expectMessage("No columns to insert"); - p.run().waitUntilFinish(); - } - - @Test - public void testWritingDisplayData() { - HBaseIO.Write write = HBaseIO.write().withTableId("fooTable").withConfiguration(conf); - DisplayData displayData = DisplayData.from(write); - assertThat(displayData, hasDisplayItem("tableId", "fooTable")); - } - - // HBase helper methods - private static void createTable(String tableId) throws Exception { - byte[][] splitKeys = {"4".getBytes(), "8".getBytes(), "C".getBytes()}; - createTable(tableId, COLUMN_FAMILY, splitKeys); - } - - private static void createTable(String tableId, byte[] columnFamily, byte[][] splitKeys) - throws Exception { - TableName tableName = TableName.valueOf(tableId); - HTableDescriptor desc = new HTableDescriptor(tableName); - HColumnDescriptor colDef = new HColumnDescriptor(columnFamily); - desc.addFamily(colDef); - admin.createTable(desc, splitKeys); - } - - /** - * Helper function to create a table and return the rows that it created. - */ - private static void writeData(String tableId, int numRows) throws Exception { - Connection connection = admin.getConnection(); - TableName tableName = TableName.valueOf(tableId); - BufferedMutator mutator = connection.getBufferedMutator(tableName); - List mutations = makeTableData(numRows); - mutator.mutate(mutations); - mutator.flush(); - mutator.close(); - } - - private static List makeTableData(int numRows) { - List mutations = new ArrayList<>(numRows); - for (int i = 0; i < numRows; ++i) { - // We pad values in hex order 0,1, ... ,F,0, ... - String prefix = String.format("%X", i % 16); - // This 21 is to have a key longer than an input - byte[] rowKey = Bytes.toBytes( - StringUtils.leftPad("_" + String.valueOf(i), 21, prefix)); - byte[] value = Bytes.toBytes(String.valueOf(i)); - byte[] valueEmail = Bytes.toBytes(String.valueOf(i) + "@email.com"); - mutations.add(new Put(rowKey).addColumn(COLUMN_FAMILY, COLUMN_NAME, value)); - mutations.add(new Put(rowKey).addColumn(COLUMN_FAMILY, COLUMN_EMAIL, valueEmail)); - } - return mutations; - } - - private static ResultScanner scanTable(String tableId, Scan scan) throws Exception { - Connection connection = ConnectionFactory.createConnection(conf); - TableName tableName = TableName.valueOf(tableId); - Table table = connection.getTable(tableName); - return table.getScanner(scan); - } - - private static List readTable(String tableId, Scan scan) throws Exception { - ResultScanner scanner = scanTable(tableId, scan); - List results = new ArrayList<>(); - for (Result result : scanner) { - results.add(result); - } - scanner.close(); - return results; - } - - // Beam helper methods - /** Helper function to make a single row mutation to be written. */ - private static Iterable makeMutations(String key, String value, int numMutations) { - List mutations = new ArrayList<>(); - for (int i = 0; i < numMutations; i++) { - mutations.add(makeMutation(key + i, value)); - } - return mutations; - } - - private static Mutation makeMutation(String key, String value) { - return new Put(key.getBytes(StandardCharsets.UTF_8)) - .addColumn(COLUMN_FAMILY, COLUMN_NAME, Bytes.toBytes(value)) - .addColumn(COLUMN_FAMILY, COLUMN_EMAIL, Bytes.toBytes(value + "@email.com")); - } - - private static Mutation makeBadMutation(String key) { - return new Put(key.getBytes()); - } - - private void runReadTest(HBaseIO.Read read, List expected) { - final String transformId = read.getTableId() + "_" + read.getKeyRange(); - PCollection rows = p.apply("Read" + transformId, read); - PAssert.that(rows).containsInAnyOrder(expected); - p.run().waitUntilFinish(); - } - - private void runReadTestLength(HBaseIO.Read read, long numElements) { - final String transformId = read.getTableId() + "_" + read.getKeyRange(); - PCollection rows = p.apply("Read" + transformId, read); - PAssert.thatSingleton(rows.apply("Count" + transformId, - Count.globally())).isEqualTo(numElements); - p.run().waitUntilFinish(); - } + @Rule public final transient TestPipeline p = TestPipeline.create(); + @Rule public ExpectedException thrown = ExpectedException.none(); + + private static HBaseTestingUtility htu; + private static HBaseAdmin admin; + + private static final Configuration conf = HBaseConfiguration.create(); + private static final byte[] COLUMN_FAMILY = Bytes.toBytes("info"); + private static final byte[] COLUMN_NAME = Bytes.toBytes("name"); + private static final byte[] COLUMN_EMAIL = Bytes.toBytes("email"); + + @BeforeClass + public static void beforeClass() throws Exception { + conf.setInt(HConstants.HBASE_CLIENT_RETRIES_NUMBER, 1); + // Try to bind the hostname to localhost to solve an issue when it is not configured or + // no DNS resolution available. + conf.setStrings("hbase.master.hostname", "localhost"); + conf.setStrings("hbase.regionserver.hostname", "localhost"); + htu = new HBaseTestingUtility(conf); + + // We don't use the full htu.startMiniCluster() to avoid starting unneeded HDFS/MR daemons + htu.startMiniZKCluster(); + MiniHBaseCluster hbm = htu.startMiniHBaseCluster(1, 4); + hbm.waitForActiveAndReadyMaster(); + + admin = htu.getHBaseAdmin(); + } + + @AfterClass + public static void afterClass() throws Exception { + if (admin != null) { + admin.close(); + admin = null; + } + if (htu != null) { + htu.shutdownMiniHBaseCluster(); + htu.shutdownMiniZKCluster(); + htu = null; + } + } + + @Test + public void testReadBuildsCorrectly() { + HBaseIO.Read read = HBaseIO.read().withConfiguration(conf).withTableId("table"); + assertEquals("table", read.getTableId()); + assertNotNull("configuration", read.getConfiguration()); + } + + @Test + public void testReadBuildsCorrectlyInDifferentOrder() { + HBaseIO.Read read = HBaseIO.read().withTableId("table").withConfiguration(conf); + assertEquals("table", read.getTableId()); + assertNotNull("configuration", read.getConfiguration()); + } + + @Test + public void testWriteBuildsCorrectly() { + HBaseIO.Write write = HBaseIO.write().withConfiguration(conf).withTableId("table"); + assertEquals("table", write.getTableId()); + assertNotNull("configuration", write.getConfiguration()); + } + + @Test + public void testWriteBuildsCorrectlyInDifferentOrder() { + HBaseIO.Write write = HBaseIO.write().withTableId("table").withConfiguration(conf); + assertEquals("table", write.getTableId()); + assertNotNull("configuration", write.getConfiguration()); + } + + @Test + public void testWriteValidationFailsMissingTable() { + HBaseIO.Write write = HBaseIO.write().withConfiguration(conf); + thrown.expect(IllegalArgumentException.class); + write.validate(null /* input */); + } + + @Test + public void testWriteValidationFailsMissingConfiguration() { + HBaseIO.Write write = HBaseIO.write().withTableId("table"); + thrown.expect(IllegalArgumentException.class); + write.validate(null /* input */); + } + + /** Tests that when reading from a non-existent table, the read fails. */ + @Test + public void testReadingFailsTableDoesNotExist() throws Exception { + final String table = "TEST-TABLE-INVALID"; + // Exception will be thrown by read.validate() when read is applied. + thrown.expect(IllegalArgumentException.class); + thrown.expectMessage(String.format("Table %s does not exist", table)); + runReadTest(HBaseIO.read().withConfiguration(conf).withTableId(table), new ArrayList()); + } + + /** Tests that when reading from an empty table, the read succeeds. */ + @Test + public void testReadingEmptyTable() throws Exception { + final String table = "TEST-EMPTY-TABLE"; + createTable(table); + runReadTest(HBaseIO.read().withConfiguration(conf).withTableId(table), new ArrayList()); + } + + @Test + public void testReading() throws Exception { + final String table = "TEST-MANY-ROWS-TABLE"; + final int numRows = 1001; + createTable(table); + writeData(table, numRows); + runReadTestLength(HBaseIO.read().withConfiguration(conf).withTableId(table), 1001); + } + + /** Tests reading all rows from a split table. */ + @Test + public void testReadingWithSplits() throws Exception { + final String table = "TEST-MANY-ROWS-SPLITS-TABLE"; + final int numRows = 1500; + final int numRegions = 4; + final long bytesPerRow = 100L; + + // Set up test table data and sample row keys for size estimation and splitting. + createTable(table); + writeData(table, numRows); + + HBaseIO.Read read = HBaseIO.read().withConfiguration(conf).withTableId(table); + HBaseSource source = new HBaseSource(read, null /* estimatedSizeBytes */); + List> splits = + source.split(numRows * bytesPerRow / numRegions, null /* options */); + + // Test num splits and split equality. + assertThat(splits, hasSize(4)); + assertSourcesEqualReferenceSource(source, splits, null /* options */); + } + + /** Tests that a {@link HBaseSource} can be read twice, verifying its immutability. */ + @Test + public void testReadingSourceTwice() throws Exception { + final String table = "TEST-READING-TWICE"; + final int numRows = 10; + + // Set up test table data and sample row keys for size estimation and splitting. + createTable(table); + writeData(table, numRows); + + HBaseIO.Read read = HBaseIO.read().withConfiguration(conf).withTableId(table); + HBaseSource source = new HBaseSource(read, null /* estimatedSizeBytes */); + assertThat(SourceTestUtils.readFromSource(source, null), hasSize(numRows)); + // second read. + assertThat(SourceTestUtils.readFromSource(source, null), hasSize(numRows)); + } + + /** Tests reading all rows using a filter. */ + @Test + public void testReadingWithFilter() throws Exception { + final String table = "TEST-FILTER-TABLE"; + final int numRows = 1001; + + createTable(table); + writeData(table, numRows); + + String regex = ".*17.*"; + Filter filter = new RowFilter(CompareFilter.CompareOp.EQUAL, new RegexStringComparator(regex)); + HBaseIO.Read read = + HBaseIO.read().withConfiguration(conf).withTableId(table).withFilter(filter); + runReadTestLength(read, 20); + } + + /** + * Tests reading all rows using key ranges. Tests a prefix [), a suffix (], and a restricted range + * [] and that some properties hold across them. + */ + @Test + public void testReadingWithKeyRange() throws Exception { + final String table = "TEST-KEY-RANGE-TABLE"; + final int numRows = 1001; + final byte[] startRow = "2".getBytes(); + final byte[] stopRow = "9".getBytes(); + final ByteKey startKey = ByteKey.copyFrom(startRow); + + createTable(table); + writeData(table, numRows); + + // Test prefix: [beginning, startKey). + final ByteKeyRange prefixRange = ByteKeyRange.ALL_KEYS.withEndKey(startKey); + runReadTestLength( + HBaseIO.read().withConfiguration(conf).withTableId(table).withKeyRange(prefixRange), 126); + + // Test suffix: [startKey, end). + final ByteKeyRange suffixRange = ByteKeyRange.ALL_KEYS.withStartKey(startKey); + runReadTestLength( + HBaseIO.read().withConfiguration(conf).withTableId(table).withKeyRange(suffixRange), 875); + + // Test restricted range: [startKey, endKey). + // This one tests the second signature of .withKeyRange + runReadTestLength( + HBaseIO.read().withConfiguration(conf).withTableId(table).withKeyRange(startRow, stopRow), + 441); + } + + /** Tests dynamic work rebalancing exhaustively. */ + @Test + public void testReadingSplitAtFractionExhaustive() throws Exception { + final String table = "TEST-FEW-ROWS-SPLIT-EXHAUSTIVE-TABLE"; + final int numRows = 7; + + createTable(table); + writeData(table, numRows); + + HBaseIO.Read read = HBaseIO.read().withConfiguration(conf).withTableId(table); + HBaseSource source = + new HBaseSource(read, null /* estimatedSizeBytes */) + .withStartKey(ByteKey.of(48)) + .withEndKey(ByteKey.of(58)); + + assertSplitAtFractionExhaustive(source, null); + } + + /** Unit tests of splitAtFraction. */ + @Test + public void testReadingSplitAtFraction() throws Exception { + final String table = "TEST-SPLIT-AT-FRACTION"; + final int numRows = 10; + + createTable(table); + writeData(table, numRows); + + HBaseIO.Read read = HBaseIO.read().withConfiguration(conf).withTableId(table); + HBaseSource source = new HBaseSource(read, null /* estimatedSizeBytes */); + + // The value k is based on the partitioning schema for the data, in this test case, + // the partitioning is HEX-based, so we start from 1/16m and the value k will be + // around 1/256, so the tests are done in approximately k ~= 0.003922 steps + double k = 0.003922; + + assertSplitAtFractionFails(source, 0, k, null /* options */); + assertSplitAtFractionFails(source, 0, 1.0, null /* options */); + // With 1 items read, all split requests past k will succeed. + assertSplitAtFractionSucceedsAndConsistent(source, 1, k, null /* options */); + assertSplitAtFractionSucceedsAndConsistent(source, 1, 0.666, null /* options */); + // With 3 items read, all split requests past 3k will succeed. + assertSplitAtFractionFails(source, 3, 2 * k, null /* options */); + assertSplitAtFractionSucceedsAndConsistent(source, 3, 3 * k, null /* options */); + assertSplitAtFractionSucceedsAndConsistent(source, 3, 4 * k, null /* options */); + // With 6 items read, all split requests past 6k will succeed. + assertSplitAtFractionFails(source, 6, 5 * k, null /* options */); + assertSplitAtFractionSucceedsAndConsistent(source, 6, 0.7, null /* options */); + } + + @Test + public void testReadingDisplayData() { + HBaseIO.Read read = HBaseIO.read().withConfiguration(conf).withTableId("fooTable"); + DisplayData displayData = DisplayData.from(read); + assertThat(displayData, hasDisplayItem("tableId", "fooTable")); + assertThat(displayData, hasDisplayItem("configuration")); + } + + /** Tests that a record gets written to the service and messages are logged. */ + @Test + public void testWriting() throws Exception { + final String table = "table"; + final String key = "key"; + final String value = "value"; + final int numMutations = 100; + + createTable(table); + + p.apply("multiple rows", Create.of(makeMutations(key, value, numMutations))) + .apply("write", HBaseIO.write().withConfiguration(conf).withTableId(table)); + p.run().waitUntilFinish(); + + List results = readTable(table, new Scan()); + assertEquals(numMutations, results.size()); + } + + /** Tests that when writing to a non-existent table, the write fails. */ + @Test + public void testWritingFailsTableDoesNotExist() throws Exception { + final String table = "TEST-TABLE-DOES-NOT-EXIST"; + + p.apply(Create.empty(HBaseMutationCoder.of())) + .apply("write", HBaseIO.write().withConfiguration(conf).withTableId(table)); + + // Exception will be thrown by write.validate() when write is applied. + thrown.expect(IllegalArgumentException.class); + thrown.expectMessage(String.format("Table %s does not exist", table)); + p.run(); + } + + /** Tests that when writing an element fails, the write fails. */ + @Test + public void testWritingFailsBadElement() throws Exception { + final String table = "TEST-TABLE-BAD-ELEMENT"; + final String key = "KEY"; + createTable(table); + + p.apply(Create.of(makeBadMutation(key))) + .apply(HBaseIO.write().withConfiguration(conf).withTableId(table)); + + thrown.expect(Pipeline.PipelineExecutionException.class); + thrown.expectCause(Matchers.instanceOf(IllegalArgumentException.class)); + thrown.expectMessage("No columns to insert"); + p.run().waitUntilFinish(); + } + + @Test + public void testWritingDisplayData() { + HBaseIO.Write write = HBaseIO.write().withTableId("fooTable").withConfiguration(conf); + DisplayData displayData = DisplayData.from(write); + assertThat(displayData, hasDisplayItem("tableId", "fooTable")); + } + + // HBase helper methods + private static void createTable(String tableId) throws Exception { + byte[][] splitKeys = {"4".getBytes(), "8".getBytes(), "C".getBytes()}; + createTable(tableId, COLUMN_FAMILY, splitKeys); + } + + private static void createTable(String tableId, byte[] columnFamily, byte[][] splitKeys) + throws Exception { + TableName tableName = TableName.valueOf(tableId); + HTableDescriptor desc = new HTableDescriptor(tableName); + HColumnDescriptor colDef = new HColumnDescriptor(columnFamily); + desc.addFamily(colDef); + admin.createTable(desc, splitKeys); + } + + /** Helper function to create a table and return the rows that it created. */ + private static void writeData(String tableId, int numRows) throws Exception { + Connection connection = admin.getConnection(); + TableName tableName = TableName.valueOf(tableId); + BufferedMutator mutator = connection.getBufferedMutator(tableName); + List mutations = makeTableData(numRows); + mutator.mutate(mutations); + mutator.flush(); + mutator.close(); + } + + private static List makeTableData(int numRows) { + List mutations = new ArrayList<>(numRows); + for (int i = 0; i < numRows; ++i) { + // We pad values in hex order 0,1, ... ,F,0, ... + String prefix = String.format("%X", i % 16); + // This 21 is to have a key longer than an input + byte[] rowKey = Bytes.toBytes(StringUtils.leftPad("_" + String.valueOf(i), 21, prefix)); + byte[] value = Bytes.toBytes(String.valueOf(i)); + byte[] valueEmail = Bytes.toBytes(String.valueOf(i) + "@email.com"); + mutations.add(new Put(rowKey).addColumn(COLUMN_FAMILY, COLUMN_NAME, value)); + mutations.add(new Put(rowKey).addColumn(COLUMN_FAMILY, COLUMN_EMAIL, valueEmail)); + } + return mutations; + } + + private static ResultScanner scanTable(String tableId, Scan scan) throws Exception { + Connection connection = ConnectionFactory.createConnection(conf); + TableName tableName = TableName.valueOf(tableId); + Table table = connection.getTable(tableName); + return table.getScanner(scan); + } + + private static List readTable(String tableId, Scan scan) throws Exception { + ResultScanner scanner = scanTable(tableId, scan); + List results = new ArrayList<>(); + for (Result result : scanner) { + results.add(result); + } + scanner.close(); + return results; + } + + // Beam helper methods + /** Helper function to make a single row mutation to be written. */ + private static Iterable makeMutations(String key, String value, int numMutations) { + List mutations = new ArrayList<>(); + for (int i = 0; i < numMutations; i++) { + mutations.add(makeMutation(key + i, value)); + } + return mutations; + } + + private static Mutation makeMutation(String key, String value) { + return new Put(key.getBytes(StandardCharsets.UTF_8)) + .addColumn(COLUMN_FAMILY, COLUMN_NAME, Bytes.toBytes(value)) + .addColumn(COLUMN_FAMILY, COLUMN_EMAIL, Bytes.toBytes(value + "@email.com")); + } + + private static Mutation makeBadMutation(String key) { + return new Put(key.getBytes()); + } + + private void runReadTest(HBaseIO.Read read, List expected) { + final String transformId = read.getTableId() + "_" + read.getKeyRange(); + PCollection rows = p.apply("Read" + transformId, read); + PAssert.that(rows).containsInAnyOrder(expected); + p.run().waitUntilFinish(); + } + + private void runReadTestLength(HBaseIO.Read read, long numElements) { + final String transformId = read.getTableId() + "_" + read.getKeyRange(); + PCollection rows = p.apply("Read" + transformId, read); + PAssert.thatSingleton(rows.apply("Count" + transformId, Count.globally())) + .isEqualTo(numElements); + p.run().waitUntilFinish(); + } } diff --git a/sdks/java/io/hbase/src/test/java/org/apache/beam/sdk/io/hbase/HBaseMutationCoderTest.java b/sdks/java/io/hbase/src/test/java/org/apache/beam/sdk/io/hbase/HBaseMutationCoderTest.java index 5bf2d80ea4a4e..41525dc624024 100644 --- a/sdks/java/io/hbase/src/test/java/org/apache/beam/sdk/io/hbase/HBaseMutationCoderTest.java +++ b/sdks/java/io/hbase/src/test/java/org/apache/beam/sdk/io/hbase/HBaseMutationCoderTest.java @@ -28,9 +28,7 @@ import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -/** - * Tests for HBaseMutationCoder. - */ +/** Tests for HBaseMutationCoder. */ @RunWith(JUnit4.class) public class HBaseMutationCoderTest { @Rule public final ExpectedException thrown = ExpectedException.none(); diff --git a/sdks/java/io/hbase/src/test/java/org/apache/beam/sdk/io/hbase/HBaseResultCoderTest.java b/sdks/java/io/hbase/src/test/java/org/apache/beam/sdk/io/hbase/HBaseResultCoderTest.java index c6b27d6f5d246..5af5e162ed7f6 100644 --- a/sdks/java/io/hbase/src/test/java/org/apache/beam/sdk/io/hbase/HBaseResultCoderTest.java +++ b/sdks/java/io/hbase/src/test/java/org/apache/beam/sdk/io/hbase/HBaseResultCoderTest.java @@ -25,9 +25,7 @@ import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -/** - * Tests for HBaseResultCoder. - */ +/** Tests for HBaseResultCoder. */ @RunWith(JUnit4.class) public class HBaseResultCoderTest { @Rule public final ExpectedException thrown = ExpectedException.none(); diff --git a/sdks/java/io/hbase/src/test/java/org/apache/beam/sdk/io/hbase/SerializableScanTest.java b/sdks/java/io/hbase/src/test/java/org/apache/beam/sdk/io/hbase/SerializableScanTest.java index 49eb4e3c43fdf..7d2fd280240db 100644 --- a/sdks/java/io/hbase/src/test/java/org/apache/beam/sdk/io/hbase/SerializableScanTest.java +++ b/sdks/java/io/hbase/src/test/java/org/apache/beam/sdk/io/hbase/SerializableScanTest.java @@ -28,14 +28,12 @@ import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -/** - * Tests for SerializableScan. - */ +/** Tests for SerializableScan. */ @RunWith(JUnit4.class) public class SerializableScanTest { @Rule public final ExpectedException thrown = ExpectedException.none(); private static final SerializableScan DEFAULT_SERIALIZABLE_SCAN = - new SerializableScan(new Scan()); + new SerializableScan(new Scan()); @Test public void testSerializationDeserialization() throws Exception { From 2dea491cab131b830e884bd408e82e97690259d9 Mon Sep 17 00:00:00 2001 From: Sergiy Byelozyorov Date: Wed, 23 Aug 2017 19:04:31 +0200 Subject: [PATCH 332/346] Actually wait for exector service to shutdown --- .../runners/direct/direct_runner_test.py | 41 +++++++++++++++++++ .../apache_beam/runners/direct/executor.py | 1 + 2 files changed, 42 insertions(+) create mode 100644 sdks/python/apache_beam/runners/direct/direct_runner_test.py diff --git a/sdks/python/apache_beam/runners/direct/direct_runner_test.py b/sdks/python/apache_beam/runners/direct/direct_runner_test.py new file mode 100644 index 0000000000000..1c8b7855aadca --- /dev/null +++ b/sdks/python/apache_beam/runners/direct/direct_runner_test.py @@ -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 threading +import unittest + +import apache_beam as beam +from apache_beam.testing import test_pipeline + + +class DirectPipelineResultTest(unittest.TestCase): + + def test_waiting_on_result_stops_executor_threads(self): + pre_test_threads = set(t.ident for t in threading.enumerate()) + + pipeline = test_pipeline.TestPipeline() + _ = (pipeline | beam.Create([{'foo': 'bar'}])) + result = pipeline.run() + result.wait_until_finish() + + post_test_threads = set(t.ident for t in threading.enumerate()) + new_threads = post_test_threads - pre_test_threads + self.assertEqual(len(new_threads), 0) + + +if __name__ == '__main__': + unittest.main() diff --git a/sdks/python/apache_beam/runners/direct/executor.py b/sdks/python/apache_beam/runners/direct/executor.py index d465068b9648b..3e08b522d5413 100644 --- a/sdks/python/apache_beam/runners/direct/executor.py +++ b/sdks/python/apache_beam/runners/direct/executor.py @@ -415,6 +415,7 @@ def await_completion(self): raise t, v, tb finally: self.executor_service.shutdown() + self.executor_service.await_completion() def schedule_consumers(self, committed_bundle): if committed_bundle.pcollection in self.value_to_consumers: From c4d964651d6b7e73250cd2826a21b9b8ea95865e Mon Sep 17 00:00:00 2001 From: Luke Cwik Date: Fri, 25 Aug 2017 14:53:32 -0700 Subject: [PATCH 333/346] Ensure Dataflow service version matches Google API service version 1.22.0 --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 5fd12977972f3..81c80034267a7 100644 --- a/pom.xml +++ b/pom.xml @@ -112,7 +112,7 @@ v1-rev6-1.22.0 0.1.0 v2-rev8-1.22.0 - v1b3-rev198-1.20.0 + v1b3-rev198-1.22.0 0.5.160222 1.4.0 1.3.0 From 1e21f453721cb7aef0783cb73d72f6b928685515 Mon Sep 17 00:00:00 2001 From: Thomas Groh Date: Thu, 17 Aug 2017 17:45:09 -0700 Subject: [PATCH 334/346] Add Proto Definitions for the Artifact API Update the Job API to permit a "prepare" phase of executing a pipeline, where prerequisite work like staging artifacts can be performed before the job is executed. --- .../src/main/proto/beam_artifact_api.proto | 122 ++++++++++++++++++ .../src/main/proto/beam_job_api.proto | 46 +++++-- 2 files changed, 157 insertions(+), 11 deletions(-) create mode 100644 sdks/common/runner-api/src/main/proto/beam_artifact_api.proto diff --git a/sdks/common/runner-api/src/main/proto/beam_artifact_api.proto b/sdks/common/runner-api/src/main/proto/beam_artifact_api.proto new file mode 100644 index 0000000000000..6e39d88359ca4 --- /dev/null +++ b/sdks/common/runner-api/src/main/proto/beam_artifact_api.proto @@ -0,0 +1,122 @@ +/* + * 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. + */ + +/* + * Protocol Buffers describing the Artifact API, for communicating with a runner + * for artifact staging and retrieval over GRPC. + */ + +syntax = "proto3"; + +package org.apache.beam.runner_api.v1; + +option java_package = "org.apache.beam.sdk.common.runner.v1"; +option java_outer_classname = "ArtifactApi"; + +// A service to stage artifacts for use in a Job. +// +// RPCs made to an ArtifactStagingService endpoint should include some form of identification for +// the job as a header. +service ArtifactStagingService { + // Stage an artifact to be available during job execution. The first request must contain the + // name of the artifact. All future requests must contain sequential chunks of the content of + // the artifact. + rpc PutArtifact(stream PutArtifactRequest) returns (PutArtifactResponse); + + // Commit the manifest for a Job. All artifacts must have been successfully uploaded + // before this call is made. + // + // Throws error INVALID_ARGUMENT if not all of the members of the manifest are present + rpc CommitManifest(CommitManifestRequest) returns (CommitManifestResponse); +} + +// A service to retrieve artifacts for use in a Job. +service ArtifactRetrievalService { + // Get the manifest for the job + rpc GetManifest(GetManifestRequest) returns (GetManifestResponse); + + // Get an artifact staged for the job. The requested artifact must be within the manifest + rpc GetArtifact(GetArtifactRequest) returns (stream ArtifactChunk); +} + +// An artifact identifier and associated metadata. +message Artifact { + // (Required) The name of the artifact. + string name = 1; + + // (Optional) The Unix-like permissions of the artifact + int32 permissions = 2; + + // (Optional) The md5 checksum of the artifact. + string md5 = 3; +} + +// A collection of artifacts. +message Manifest { + repeated Artifact artifact = 1; +} + +// A request to get the manifest of a Job. +message GetManifestRequest {} + +// A response containing a job manifest. +message GetManifestResponse { + Manifest manifest = 1; +} + +// A request to get an artifact. The artifact must be present in the manifest for the job. +message GetArtifactRequest { + // (Required) The name of the artifact to retrieve. + string name = 1; +} + +// Part of an artifact. +message ArtifactChunk { + bytes data = 1; +} + +// A request to stage an artifact. +message PutArtifactRequest { + // (Required) + oneof content { + // The name of the artifact. The first message in a PutArtifact call must contain the name + // of the artifact. + string name = 1; + + // A chunk of the artifact. All messages after the first in a PutArtifact call must contain a + // chunk. + ArtifactChunk data = 2; + } +} + +message PutArtifactResponse { +} + +// A request to commit the manifest for a Job. All artifacts must have been successfully uploaded +// before this call is made. +message CommitManifestRequest { + // (Required) The manifest to commit. + Manifest manifest = 1; +} + +// The result of committing a manifest. +message CommitManifestResponse { + // (Required) An opaque token representing the entirety of the staged artifacts. + string staging_token = 1; +} + diff --git a/sdks/common/runner-api/src/main/proto/beam_job_api.proto b/sdks/common/runner-api/src/main/proto/beam_job_api.proto index 7be14cc90f486..8946d2a723346 100644 --- a/sdks/common/runner-api/src/main/proto/beam_job_api.proto +++ b/sdks/common/runner-api/src/main/proto/beam_job_api.proto @@ -34,36 +34,60 @@ import "google/protobuf/struct.proto"; // Job Service for running RunnerAPI pipelines service JobService { + // Prepare a job for execution. The job will not be executed until a call is made to run with the + // returned preparationId. + rpc prepare (PrepareJobRequest) returns (PrepareJobResponse); + // Submit the job for execution - rpc run (SubmitJobRequest) returns (SubmitJobResponse) {} + rpc run (RunJobRequest) returns (RunJobResponse); // Get the current state of the job - rpc getState (GetJobStateRequest) returns (GetJobStateResponse) {} + rpc getState (GetJobStateRequest) returns (GetJobStateResponse); // Cancel the job - rpc cancel (CancelJobRequest) returns (CancelJobResponse) {} + rpc cancel (CancelJobRequest) returns (CancelJobResponse); // Subscribe to a stream of state changes of the job, will immediately return the current state of the job as the first response. - rpc getStateStream (GetJobStateRequest) returns (stream GetJobStateResponse) {} + rpc getStateStream (GetJobStateRequest) returns (stream GetJobStateResponse); // Subscribe to a stream of state changes and messages from the job - rpc getMessageStream (JobMessagesRequest) returns (stream JobMessagesResponse) {} + rpc getMessageStream (JobMessagesRequest) returns (stream JobMessagesResponse); } -// Submit is a synchronus request that returns a jobId back +// Prepare is a synchronous request that returns a preparationId back // Throws error GRPC_STATUS_UNAVAILABLE if server is down -// Throws error ALREADY_EXISTS if the jobName is reused as runners are permitted to deduplicate based on the name of the job. +// Throws error ALREADY_EXISTS if the jobName is reused. Runners are permitted to deduplicate based on the name of the job. // Throws error UNKNOWN for all other issues -message SubmitJobRequest { +message PrepareJobRequest { org.apache.beam.runner_api.v1.Pipeline pipeline = 1; // (required) google.protobuf.Struct pipelineOptions = 2; // (required) string jobName = 3; // (required) } -message SubmitJobResponse { - // JobId is used as an identifier for the job in all future calls. - string jobId = 1; // (required) +message PrepareJobResponse { + // (required) The ID used to associate calls made while preparing the job. preparationId is used + // to run the job, as well as in other pre-execution APIs such as Artifact staging. + string preparationId = 1; +} + + +// Run is a synchronous request that returns a jobId back. +// Throws error GRPC_STATUS_UNAVAILABLE if server is down +// Throws error NOT_FOUND if the preparation ID does not exist +// Throws error UNKNOWN for all other issues +message RunJobRequest { + // (required) The ID provided by an earlier call to prepare. Runs the job. All prerequisite tasks + // must have been completed. + string preparationId = 1; + // (optional) If any artifacts have been staged for this job, contains the staging_token returned + // from the CommitManifestResponse. + string stagingToken = 2; +} + + +message RunJobResponse { + string jobId = 1; // (required) The ID for the executing job } From 8f71dc41b30a978095ca0e0699009e4f4445a618 Mon Sep 17 00:00:00 2001 From: Ahmet Altay Date: Wed, 23 Aug 2017 18:33:03 -0700 Subject: [PATCH 335/346] Implement cancel and wait_until_finish(duration) for DataflowRunner --- .../runners/dataflow/dataflow_runner.py | 96 ++++++++++++++++--- .../runners/dataflow/dataflow_runner_test.py | 80 +++++++++++++++- sdks/python/apache_beam/runners/runner.py | 3 + 3 files changed, 160 insertions(+), 19 deletions(-) diff --git a/sdks/python/apache_beam/runners/dataflow/dataflow_runner.py b/sdks/python/apache_beam/runners/dataflow/dataflow_runner.py index 880901e671472..813759e125312 100644 --- a/sdks/python/apache_beam/runners/dataflow/dataflow_runner.py +++ b/sdks/python/apache_beam/runners/dataflow/dataflow_runner.py @@ -87,8 +87,18 @@ def _get_unique_step_name(self): return 's%s' % self._unique_step_id @staticmethod - def poll_for_job_completion(runner, result): - """Polls for the specified job to finish running (successfully or not).""" + def poll_for_job_completion(runner, result, duration): + """Polls for the specified job to finish running (successfully or not). + + Updates the result with the new job information before returning. + + Args: + runner: DataflowRunner instance to use for polling job state. + result: DataflowPipelineResult instance used for job information. + duration (int): The time to wait (in milliseconds) for job to finish. + If it is set to :data:`None`, it will wait indefinitely until the job + is finished. + """ last_message_time = None last_message_hash = None @@ -109,6 +119,10 @@ def rank_error(msg): return 1 return 0 + if duration: + start_secs = time.time() + duration_secs = duration / 1000 + job_id = result.job_id() while True: response = runner.dataflow_client.get_job(job_id) @@ -161,6 +175,13 @@ def rank_error(msg): if not page_token: break + if duration: + passed_secs = time.time() - start_secs + if duration_secs > passed_secs: + logging.warning('Timing out on waiting for job %s after %d seconds', + job_id, passed_secs) + break + result._job = response runner.last_error_msg = last_error_msg @@ -280,7 +301,10 @@ def run(self, pipeline): self.dataflow_client = apiclient.DataflowApplicationClient( pipeline._options) - # Create the job + # Create the job description and send a request to the service. The result + # can be None if there is no need to send a request to the service (e.g. + # template creation). If a request was sent and failed then the call will + # raise an exception. result = DataflowPipelineResult( self.dataflow_client.create_job(self.job), self) @@ -838,11 +862,23 @@ class DataflowPipelineResult(PipelineResult): """Represents the state of a pipeline run on the Dataflow service.""" def __init__(self, job, runner): - """Job is a Job message from the Dataflow API.""" + """Initialize a new DataflowPipelineResult instance. + + Args: + job: Job message from the Dataflow API. Could be :data:`None` if a job + request was not sent to Dataflow service (e.g. template jobs). + runner: DataflowRunner instance. + """ self._job = job self._runner = runner self.metric_results = None + def _update_job(self): + # We need the job id to be able to update job information. There is no need + # to update the job if we are in a known terminal state. + if self.has_job and not self._is_in_terminal_state(): + self._job = self._runner.dataflow_client.get_job(self.job_id()) + def job_id(self): return self._job.id @@ -863,7 +899,12 @@ def state(self): if not self.has_job: return PipelineState.UNKNOWN + self._update_job() + values_enum = dataflow_api.Job.CurrentStateValueValuesEnum + + # TODO: Move this table to a another location. + # Ordered by the enum values. api_jobstate_map = { values_enum.JOB_STATE_UNKNOWN: PipelineState.UNKNOWN, values_enum.JOB_STATE_STOPPED: PipelineState.STOPPED, @@ -874,6 +915,8 @@ def state(self): values_enum.JOB_STATE_UPDATED: PipelineState.UPDATED, values_enum.JOB_STATE_DRAINING: PipelineState.DRAINING, values_enum.JOB_STATE_DRAINED: PipelineState.DRAINED, + values_enum.JOB_STATE_PENDING: PipelineState.PENDING, + values_enum.JOB_STATE_CANCELLING: PipelineState.CANCELLING, } return (api_jobstate_map[self._job.currentState] if self._job.currentState @@ -883,21 +926,20 @@ def _is_in_terminal_state(self): if not self.has_job: return True - return self.state in [ - PipelineState.STOPPED, PipelineState.DONE, PipelineState.FAILED, - PipelineState.CANCELLED, PipelineState.DRAINED] + values_enum = dataflow_api.Job.CurrentStateValueValuesEnum + return self._job.currentState in [ + values_enum.JOB_STATE_STOPPED, values_enum.JOB_STATE_DONE, + values_enum.JOB_STATE_FAILED, values_enum.JOB_STATE_CANCELLED, + values_enum.JOB_STATE_DRAINED] def wait_until_finish(self, duration=None): if not self._is_in_terminal_state(): if not self.has_job: raise IOError('Failed to get the Dataflow job id.') - if duration: - raise NotImplementedError( - 'DataflowRunner does not support duration argument.') thread = threading.Thread( target=DataflowRunner.poll_for_job_completion, - args=(self._runner, self)) + args=(self._runner, self, duration)) # Mark the thread as a daemon thread so a keyboard interrupt on the main # thread will terminate everything. This is also the reason we will not @@ -906,14 +948,40 @@ def wait_until_finish(self, duration=None): thread.start() while thread.isAlive(): time.sleep(5.0) - if self.state != PipelineState.DONE: - # TODO(BEAM-1290): Consider converting this to an error log based on the - # resolution of the issue. + + terminated = self._is_in_terminal_state() + assert duration or terminated, ( + 'Job did not reach to a terminal state after waiting indefinitely.') + + if terminated and self.state != PipelineState.DONE: + # TODO(BEAM-1290): Consider converting this to an error log based on + # theresolution of the issue. raise DataflowRuntimeException( 'Dataflow pipeline failed. State: %s, Error:\n%s' % (self.state, getattr(self._runner, 'last_error_msg', None)), self) return self.state + def cancel(self): + if not self.has_job: + raise IOError('Failed to get the Dataflow job id.') + + self._update_job() + + if self._is_in_terminal_state(): + logging.warning( + 'Cancel failed because job %s is already terminated in state %s.', + self.job_id(), self.state) + else: + if not self._runner.dataflow_client.modify_job_state( + self.job_id(), 'JOB_STATE_CANCELLED'): + cancel_failed_message = ( + 'Failed to cancel job %s, please go to the Developers Console to ' + 'cancel it manually.') % self.job_id() + logging.error(cancel_failed_message) + raise DataflowRuntimeException(cancel_failed_message, self) + + return self.state + def __str__(self): return '<%s %s %s>' % ( self.__class__.__name__, diff --git a/sdks/python/apache_beam/runners/dataflow/dataflow_runner_test.py b/sdks/python/apache_beam/runners/dataflow/dataflow_runner_test.py index 80414d6883daf..8e708e6fde293 100644 --- a/sdks/python/apache_beam/runners/dataflow/dataflow_runner_test.py +++ b/sdks/python/apache_beam/runners/dataflow/dataflow_runner_test.py @@ -35,6 +35,7 @@ from apache_beam.runners.dataflow.dataflow_runner import DataflowPipelineResult from apache_beam.runners.dataflow.dataflow_runner import DataflowRuntimeException from apache_beam.runners.dataflow.internal.clients import dataflow as dataflow_api +from apache_beam.runners.runner import PipelineState from apache_beam.testing.test_pipeline import TestPipeline from apache_beam.transforms.display import DisplayDataItem from apache_beam.transforms.core import _GroupByKeyOnly @@ -68,13 +69,17 @@ def test_wait_until_finish(self, patched_time_sleep): class MockDataflowRunner(object): - def __init__(self, final_state): + def __init__(self, states): self.dataflow_client = mock.MagicMock() self.job = mock.MagicMock() self.job.currentState = values_enum.JOB_STATE_UNKNOWN + self._states = states + self._next_state_index = 0 def get_job_side_effect(*args, **kwargs): - self.job.currentState = final_state + self.job.currentState = self._states[self._next_state_index] + if self._next_state_index < (len(self._states) - 1): + self._next_state_index += 1 return mock.DEFAULT self.dataflow_client.get_job = mock.MagicMock( @@ -84,14 +89,79 @@ def get_job_side_effect(*args, **kwargs): with self.assertRaisesRegexp( DataflowRuntimeException, 'Dataflow pipeline failed. State: FAILED'): - failed_runner = MockDataflowRunner(values_enum.JOB_STATE_FAILED) + failed_runner = MockDataflowRunner([values_enum.JOB_STATE_FAILED]) failed_result = DataflowPipelineResult(failed_runner.job, failed_runner) failed_result.wait_until_finish() - succeeded_runner = MockDataflowRunner(values_enum.JOB_STATE_DONE) + succeeded_runner = MockDataflowRunner([values_enum.JOB_STATE_DONE]) succeeded_result = DataflowPipelineResult( succeeded_runner.job, succeeded_runner) - succeeded_result.wait_until_finish() + result = succeeded_result.wait_until_finish() + self.assertEqual(result, PipelineState.DONE) + + @mock.patch('time.time', mock.MagicMock(side_effect=[1, 2, 3])) + def _duration_succeeded(): + duration_succeeded_runner = MockDataflowRunner( + [values_enum.JOB_STATE_RUNNING, values_enum.JOB_STATE_DONE]) + duration_succeeded_result = DataflowPipelineResult( + duration_succeeded_runner.job, duration_succeeded_runner) + result = duration_succeeded_result.wait_until_finish(5) + self.assertEqual(result, PipelineState.DONE) + _duration_succeeded() + + @mock.patch('time.time', mock.MagicMock(side_effect=[1, 10, 20])) + def _duration_timedout(): + duration_timedout_runner = MockDataflowRunner( + [values_enum.JOB_STATE_RUNNING]) + duration_timedout_result = DataflowPipelineResult( + duration_timedout_runner.job, duration_timedout_runner) + result = duration_timedout_result.wait_until_finish(5) + self.assertEqual(result, PipelineState.RUNNING) + _duration_timedout() + + @mock.patch('time.time', mock.MagicMock(side_effect=[1, 2, 3])) + def _duration_failed(): + with self.assertRaisesRegexp( + DataflowRuntimeException, 'Dataflow pipeline failed. State: FAILED'): + duration_failed_runner = MockDataflowRunner( + [values_enum.JOB_STATE_FAILED]) + duration_failed_result = DataflowPipelineResult( + duration_failed_runner.job, duration_failed_runner) + duration_failed_result.wait_until_finish(5) + _duration_failed() + + @mock.patch('time.sleep', return_value=None) + def test_cancel(self, patched_time_sleep): + values_enum = dataflow_api.Job.CurrentStateValueValuesEnum + + class MockDataflowRunner(object): + + def __init__(self, state, cancel_result): + self.dataflow_client = mock.MagicMock() + self.job = mock.MagicMock() + self.job.currentState = state + + self.dataflow_client.get_job = mock.MagicMock(return_value=self.job) + self.dataflow_client.modify_job_state = mock.MagicMock( + return_value=cancel_result) + self.dataflow_client.list_messages = mock.MagicMock( + return_value=([], None)) + + with self.assertRaisesRegexp( + DataflowRuntimeException, 'Failed to cancel job'): + failed_runner = MockDataflowRunner(values_enum.JOB_STATE_RUNNING, False) + failed_result = DataflowPipelineResult(failed_runner.job, failed_runner) + failed_result.cancel() + + succeeded_runner = MockDataflowRunner(values_enum.JOB_STATE_RUNNING, True) + succeeded_result = DataflowPipelineResult( + succeeded_runner.job, succeeded_runner) + succeeded_result.cancel() + + terminal_runner = MockDataflowRunner(values_enum.JOB_STATE_DONE, False) + terminal_result = DataflowPipelineResult( + terminal_runner.job, terminal_runner) + terminal_result.cancel() def test_create_runner(self): self.assertTrue( diff --git a/sdks/python/apache_beam/runners/runner.py b/sdks/python/apache_beam/runners/runner.py index a3c6b345f2099..43ee27bc298f8 100644 --- a/sdks/python/apache_beam/runners/runner.py +++ b/sdks/python/apache_beam/runners/runner.py @@ -298,6 +298,9 @@ class PipelineState(object): UPDATED = 'UPDATED' # replaced by another job (terminal state) DRAINING = 'DRAINING' # still processing, no longer reading data DRAINED = 'DRAINED' # draining completed (terminal state) + PENDING = 'PENDING' # the job has been created but is not yet running. + CANCELLING = 'CANCELLING' # job has been explicitly cancelled and is + # in the process of stopping class PipelineResult(object): From dba5e5ca3779f31e407e18a7d22915491b071fe9 Mon Sep 17 00:00:00 2001 From: "wtanaka.com" Date: Sun, 27 Aug 2017 10:23:21 -1000 Subject: [PATCH 336/346] Small tweak to View.asList javadoc This may help clarify https://lists.apache.org/thread.html/cd9bd1ae4b6945cd78e04b3baa7628bd43071c443a752acbb83d388d@%3Cdev.beam.apache.org%3E --- .../core/src/main/java/org/apache/beam/sdk/transforms/View.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 f6f3af57d2a7f..e463d46db8a27 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 @@ -174,7 +174,7 @@ public static AsSingleton asSingleton() { * {@link PCollectionView} mapping each window to a {@link List} containing * all of the elements in the window. * - *

              The resulting list is required to fit in memory. + *

              Unlike with {@link #asIterable}, the resulting list is required to fit in memory. */ public static AsList asList() { return new AsList<>(); From 0cf454389129fbbe43d03ac3b26368e6d477d126 Mon Sep 17 00:00:00 2001 From: Thomas Groh Date: Fri, 25 Aug 2017 16:58:31 -0700 Subject: [PATCH 337/346] Initialize the Coder in DecodeAndEmitDoFn Ensure that the coder is available before it is used --- .../java/org/apache/beam/runners/dataflow/DataflowRunner.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 496681eb59810..afccfcaa10533 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 @@ -1102,7 +1102,7 @@ private DecodeAndEmitDoFn(Collection elements, Coder coder) throws IO @ProcessElement public void processElement(ProcessContext context) throws IOException { for (byte[] element : elements) { - context.output(CoderUtils.decodeFromByteArray(coder, element)); + context.output(CoderUtils.decodeFromByteArray(getCoder(), element)); } } } From 08a448743e3b53e055d0ccf1983b5d128c5c0692 Mon Sep 17 00:00:00 2001 From: Robert Bradshaw Date: Thu, 24 Aug 2017 11:01:20 -0700 Subject: [PATCH 338/346] Wrap unknown coders in LengthPrefixCoder. --- sdks/python/apache_beam/coders/coders.py | 10 ++ .../runners/portability/fn_api_runner.py | 99 +++++++++++++++++-- 2 files changed, 100 insertions(+), 9 deletions(-) diff --git a/sdks/python/apache_beam/coders/coders.py b/sdks/python/apache_beam/coders/coders.py index e204369b31037..10fb07b6a34f9 100644 --- a/sdks/python/apache_beam/coders/coders.py +++ b/sdks/python/apache_beam/coders/coders.py @@ -707,6 +707,16 @@ def __eq__(self, other): def __hash__(self): return hash(self._coders) + def to_runner_api_parameter(self, context): + if self.is_kv_coder(): + return urns.KV_CODER, None, self.coders() + else: + return super(TupleCoder, self).to_runner_api_parameter(context) + + @Coder.register_urn(urns.KV_CODER, None) + def from_runner_api_parameter(unused_payload, components, unused_context): + return TupleCoder(components) + class TupleSequenceCoder(FastCoder): """Coder of homogeneous tuple objects.""" diff --git a/sdks/python/apache_beam/runners/portability/fn_api_runner.py b/sdks/python/apache_beam/runners/portability/fn_api_runner.py index 7c0c06fe11109..c9b3d9a5e8310 100644 --- a/sdks/python/apache_beam/runners/portability/fn_api_runner.py +++ b/sdks/python/apache_beam/runners/portability/fn_api_runner.py @@ -122,7 +122,7 @@ def process(self, source): class _GroupingBuffer(object): """Used to accumulate groupded (shuffled) results.""" def __init__(self, pre_grouped_coder, post_grouped_coder): - self._key_coder = pre_grouped_coder.value_coder().key_coder() + self._key_coder = pre_grouped_coder.key_coder() self._pre_grouped_coder = pre_grouped_coder self._post_grouped_coder = post_grouped_coder self._table = collections.defaultdict(list) @@ -249,13 +249,80 @@ def deduplicate_read(self): # Now define the "optimization" phases. + safe_coders = {} + def expand_gbk(stages): """Transforms each GBK into a write followed by a read. """ + good_coder_urns = set(beam.coders.Coder._known_urns.keys()) - set([ + urns.PICKLED_CODER]) + coders = pipeline_components.coders + + for coder_id, coder_proto in coders.items(): + if coder_proto.spec.spec.urn == urns.BYTES_CODER: + bytes_coder_id = coder_id + break + else: + bytes_coder_id = unique_name(coders, 'bytes_coder') + pipeline_components.coders[bytes_coder_id].CopyFrom( + beam.coders.BytesCoder().to_runner_api(None)) + + coder_substitutions = {} + + def wrap_unknown_coders(coder_id, with_bytes): + if (coder_id, with_bytes) not in coder_substitutions: + wrapped_coder_id = None + coder_proto = coders[coder_id] + if coder_proto.spec.spec.urn == urns.LENGTH_PREFIX_CODER: + coder_substitutions[coder_id, with_bytes] = ( + bytes_coder_id if with_bytes else coder_id) + elif coder_proto.spec.spec.urn in good_coder_urns: + wrapped_components = [wrap_unknown_coders(c, with_bytes) + for c in coder_proto.component_coder_ids] + if wrapped_components == list(coder_proto.component_coder_ids): + # Use as is. + coder_substitutions[coder_id, with_bytes] = coder_id + else: + wrapped_coder_id = unique_name( + coders, + coder_id + ("_bytes" if with_bytes else "_len_prefix")) + coders[wrapped_coder_id].CopyFrom(coder_proto) + coders[wrapped_coder_id].component_coder_ids[:] = [ + wrap_unknown_coders(c, with_bytes) + for c in coder_proto.component_coder_ids] + coder_substitutions[coder_id, with_bytes] = wrapped_coder_id + else: + # Not a known coder. + if with_bytes: + coder_substitutions[coder_id, with_bytes] = bytes_coder_id + else: + wrapped_coder_id = unique_name(coders, coder_id + "_len_prefix") + len_prefix_coder_proto = beam_runner_api_pb2.Coder( + spec=beam_runner_api_pb2.SdkFunctionSpec( + spec=beam_runner_api_pb2.FunctionSpec( + urn=urns.LENGTH_PREFIX_CODER)), + component_coder_ids=[coder_id]) + coders[wrapped_coder_id].CopyFrom(len_prefix_coder_proto) + coder_substitutions[coder_id, with_bytes] = wrapped_coder_id + # This operation is idempotent. + if wrapped_coder_id: + coder_substitutions[wrapped_coder_id, with_bytes] = wrapped_coder_id + return coder_substitutions[coder_id, with_bytes] + + def fix_pcoll_coder(pcoll): + new_coder_id = wrap_unknown_coders(pcoll.coder_id, False) + safe_coders[new_coder_id] = wrap_unknown_coders(pcoll.coder_id, True) + pcoll.coder_id = new_coder_id + for stage in stages: assert len(stage.transforms) == 1 transform = stage.transforms[0] if transform.spec.urn == urns.GROUP_BY_KEY_ONLY_TRANSFORM: + for pcoll_id in transform.inputs.values(): + fix_pcoll_coder(pipeline_components.pcollections[pcoll_id]) + for pcoll_id in transform.outputs.values(): + fix_pcoll_coder(pipeline_components.pcollections[pcoll_id]) + # This is used later to correlate the read and write. param = str("group:%s" % stage.name) gbk_write = Stage( @@ -547,9 +614,9 @@ def process(stage): logging.debug('Stages: %s', [str(s) for s in stages]) # Return the (possibly mutated) context and ordered set of stages. - return pipeline_components, stages + return pipeline_components, stages, safe_coders - def run_stages(self, pipeline_components, stages, direct=True): + def run_stages(self, pipeline_components, stages, safe_coders, direct=True): if direct: controller = FnApiRunner.DirectController() @@ -559,13 +626,15 @@ def run_stages(self, pipeline_components, stages, direct=True): try: pcoll_buffers = collections.defaultdict(list) for stage in stages: - self.run_stage(controller, pipeline_components, stage, pcoll_buffers) + self.run_stage( + controller, pipeline_components, stage, pcoll_buffers, safe_coders) finally: controller.close() return maptask_executor_runner.WorkerRunnerResult(PipelineState.DONE) - def run_stage(self, controller, pipeline_components, stage, pcoll_buffers): + def run_stage( + self, controller, pipeline_components, stage, pcoll_buffers, safe_coders): coders = pipeline_context.PipelineContext(pipeline_components).coders data_operation_spec = controller.data_operation_spec() @@ -666,10 +735,10 @@ def extract_endpoints(stage): original_gbk_transform] input_pcoll = only_element(transform_proto.inputs.values()) output_pcoll = only_element(transform_proto.outputs.values()) - pre_gbk_coder = coders[ - pipeline_components.pcollections[input_pcoll].coder_id] - post_gbk_coder = coders[ - pipeline_components.pcollections[output_pcoll].coder_id] + pre_gbk_coder = coders[safe_coders[ + pipeline_components.pcollections[input_pcoll].coder_id]] + post_gbk_coder = coders[safe_coders[ + pipeline_components.pcollections[output_pcoll].coder_id]] pcoll_buffers[pcoll_id] = _GroupingBuffer( pre_gbk_coder, post_gbk_coder) pcoll_buffers[pcoll_id].append(output.data) @@ -1000,3 +1069,15 @@ def close(self): def only_element(iterable): element, = iterable return element + + +def unique_name(existing, prefix): + if prefix in existing: + counter = 0 + while True: + counter += 1 + prefix_counter = prefix + "_%s" % counter + if prefix_counter not in existing: + return prefix_counter + else: + return prefix From 4764883a19174f0887ec091aa109e36881e102bf Mon Sep 17 00:00:00 2001 From: Robert Bradshaw Date: Mon, 28 Aug 2017 10:10:35 -0700 Subject: [PATCH 339/346] Fix beam_job_api to conform to proto naming conventions. --- .../src/main/proto/beam_job_api.proto | 38 +++++++++---------- 1 file changed, 19 insertions(+), 19 deletions(-) diff --git a/sdks/common/runner-api/src/main/proto/beam_job_api.proto b/sdks/common/runner-api/src/main/proto/beam_job_api.proto index 8946d2a723346..5fa02ba6bb1ac 100644 --- a/sdks/common/runner-api/src/main/proto/beam_job_api.proto +++ b/sdks/common/runner-api/src/main/proto/beam_job_api.proto @@ -36,22 +36,22 @@ import "google/protobuf/struct.proto"; service JobService { // Prepare a job for execution. The job will not be executed until a call is made to run with the // returned preparationId. - rpc prepare (PrepareJobRequest) returns (PrepareJobResponse); + rpc Prepare (PrepareJobRequest) returns (PrepareJobResponse); // Submit the job for execution - rpc run (RunJobRequest) returns (RunJobResponse); + rpc Run (RunJobRequest) returns (RunJobResponse); // Get the current state of the job - rpc getState (GetJobStateRequest) returns (GetJobStateResponse); + rpc GetState (GetJobStateRequest) returns (GetJobStateResponse); // Cancel the job - rpc cancel (CancelJobRequest) returns (CancelJobResponse); + rpc Cancel (CancelJobRequest) returns (CancelJobResponse); // Subscribe to a stream of state changes of the job, will immediately return the current state of the job as the first response. - rpc getStateStream (GetJobStateRequest) returns (stream GetJobStateResponse); + rpc GetStateStream (GetJobStateRequest) returns (stream GetJobStateResponse); // Subscribe to a stream of state changes and messages from the job - rpc getMessageStream (JobMessagesRequest) returns (stream JobMessagesResponse); + rpc GetMessageStream (JobMessagesRequest) returns (stream JobMessagesResponse); } @@ -61,14 +61,14 @@ service JobService { // Throws error UNKNOWN for all other issues message PrepareJobRequest { org.apache.beam.runner_api.v1.Pipeline pipeline = 1; // (required) - google.protobuf.Struct pipelineOptions = 2; // (required) - string jobName = 3; // (required) + google.protobuf.Struct pipeline_options = 2; // (required) + string job_name = 3; // (required) } message PrepareJobResponse { // (required) The ID used to associate calls made while preparing the job. preparationId is used // to run the job, as well as in other pre-execution APIs such as Artifact staging. - string preparationId = 1; + string preparation_id = 1; } @@ -79,15 +79,15 @@ message PrepareJobResponse { message RunJobRequest { // (required) The ID provided by an earlier call to prepare. Runs the job. All prerequisite tasks // must have been completed. - string preparationId = 1; + string preparation_id = 1; // (optional) If any artifacts have been staged for this job, contains the staging_token returned // from the CommitManifestResponse. - string stagingToken = 2; + string staging_token = 2; } message RunJobResponse { - string jobId = 1; // (required) The ID for the executing job + string job_id = 1; // (required) The ID for the executing job } @@ -95,7 +95,7 @@ message RunJobResponse { // Throws error GRPC_STATUS_UNAVAILABLE if server is down // Throws error NOT_FOUND if the jobId is not found message CancelJobRequest { - string jobId = 1; // (required) + string job_id = 1; // (required) } @@ -109,7 +109,7 @@ message CancelJobResponse { // Throws error GRPC_STATUS_UNAVAILABLE if server is down // Throws error NOT_FOUND if the jobId is not found message GetJobStateRequest { - string jobId = 1; // (required) + string job_id = 1; // (required) } @@ -123,15 +123,15 @@ message GetJobStateResponse { // and job messages back; one is used for logging and the other for detecting // the job ended. message JobMessagesRequest { - string jobId = 1; // (required) + string job_id = 1; // (required) } message JobMessage { - string messageId = 1; + string message_id = 1; string time = 2; MessageImportance importance = 3; - string messageText = 4; + string message_text = 4; enum MessageImportance { JOB_MESSAGE_DEBUG = 0; @@ -144,8 +144,8 @@ message JobMessage { message JobMessagesResponse { oneof response { - JobMessage messageResponse = 1; - GetJobStateResponse stateResponse = 2; + JobMessage message_response = 1; + GetJobStateResponse state_response = 2; } } From 16f11326b875cc6598123f17135f3908e0acf0cb Mon Sep 17 00:00:00 2001 From: Ahmet Altay Date: Mon, 28 Aug 2017 13:32:32 -0700 Subject: [PATCH 340/346] Use the same termination logic in different places --- sdks/python/apache_beam/runners/dataflow/dataflow_runner.py | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/sdks/python/apache_beam/runners/dataflow/dataflow_runner.py b/sdks/python/apache_beam/runners/dataflow/dataflow_runner.py index 813759e125312..2b52f7881de97 100644 --- a/sdks/python/apache_beam/runners/dataflow/dataflow_runner.py +++ b/sdks/python/apache_beam/runners/dataflow/dataflow_runner.py @@ -949,7 +949,9 @@ def wait_until_finish(self, duration=None): while thread.isAlive(): time.sleep(5.0) - terminated = self._is_in_terminal_state() + # TODO: Merge the termination code in poll_for_job_completion and + # _is_in_terminal_state. + terminated = (str(self._job.currentState) != 'JOB_STATE_RUNNING') assert duration or terminated, ( 'Job did not reach to a terminal state after waiting indefinitely.') From 0eb8abc8d132628835e6575371d0c0f22900c6ad Mon Sep 17 00:00:00 2001 From: Eugene Kirpichov Date: Fri, 25 Aug 2017 18:40:49 -0700 Subject: [PATCH 341/346] Updates Dataflow worker to 20170825 --- runners/google-cloud-dataflow-java/pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/runners/google-cloud-dataflow-java/pom.xml b/runners/google-cloud-dataflow-java/pom.xml index 46352fbc52e99..4d55209d0b715 100644 --- a/runners/google-cloud-dataflow-java/pom.xml +++ b/runners/google-cloud-dataflow-java/pom.xml @@ -33,7 +33,7 @@ jar - beam-master-20170706 + beam-master-20170825 1 6 From bf1b0a5e9e95abf0521e081ba575c789f46ba499 Mon Sep 17 00:00:00 2001 From: Joey Baruch Date: Tue, 29 Aug 2017 16:38:24 +0300 Subject: [PATCH 342/346] Add README.md to KafkaIO --- sdks/java/io/kafka/README.md | 36 ++++++++++++++++++++++++++++++++++++ 1 file changed, 36 insertions(+) create mode 100644 sdks/java/io/kafka/README.md diff --git a/sdks/java/io/kafka/README.md b/sdks/java/io/kafka/README.md new file mode 100644 index 0000000000000..07d00a152e94a --- /dev/null +++ b/sdks/java/io/kafka/README.md @@ -0,0 +1,36 @@ + + +KafkaIO contains I/O transforms which allow you to read/write messages from/to [Apache Kafka](http://kafka.apache.org/). + +## Dependencies + +To use KafkaIO you must first add a dependency on `beam-sdks-java-io-kafka` + +```maven + + org.apache.beam + beam-sdks-java-io-kafka + ... + +``` + +## Documentation + +- [KafkaIO.java](https://github.com/apache/beam/blob/master/sdks/java/io/kafka/src/main/java/org/apache/beam/sdk/io/kafka/KafkaIO.java) From 12362b949e8f558548a886e832af31cac21db6b3 Mon Sep 17 00:00:00 2001 From: Ahmet Altay Date: Fri, 25 Aug 2017 18:21:11 -0700 Subject: [PATCH 343/346] Remove the circular dependency between pipeline & runner --- sdks/python/apache_beam/runners/dataflow/dataflow_runner.py | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/sdks/python/apache_beam/runners/dataflow/dataflow_runner.py b/sdks/python/apache_beam/runners/dataflow/dataflow_runner.py index 2b52f7881de97..56cc46c815a85 100644 --- a/sdks/python/apache_beam/runners/dataflow/dataflow_runner.py +++ b/sdks/python/apache_beam/runners/dataflow/dataflow_runner.py @@ -40,7 +40,6 @@ from apache_beam.runners.dataflow.internal.clients import dataflow as dataflow_api from apache_beam.runners.dataflow.internal.names import PropertyNames from apache_beam.runners.dataflow.internal.names import TransformNames -from apache_beam.runners.dataflow.ptransform_overrides import CreatePTransformOverride from apache_beam.runners.runner import PValueCache from apache_beam.runners.runner import PipelineResult from apache_beam.runners.runner import PipelineRunner @@ -72,6 +71,11 @@ class DataflowRunner(PipelineRunner): # not change. # For internal SDK use only. This should not be updated by Beam pipeline # authors. + + # Imported here to avoid circular dependencies. + # TODO: Remove the apache_beam.pipeline dependency in CreatePTransformOverride + from apache_beam.runners.dataflow.ptransform_overrides import CreatePTransformOverride + _PTRANSFORM_OVERRIDES = [ CreatePTransformOverride(), ] From b0e03a33cf0c2c573a2d34d88506e19ebb28c934 Mon Sep 17 00:00:00 2001 From: Reuven Lax Date: Sun, 30 Jul 2017 21:42:59 -0700 Subject: [PATCH 344/346] Add support for TimePartitioning in BigQueryIO.write(). --- pom.xml | 2 +- ...ltCoderCloudObjectTranslatorRegistrar.java | 2 + .../beam/sdk/io/gcp/bigquery/BatchLoads.java | 4 +- .../sdk/io/gcp/bigquery/BigQueryHelpers.java | 8 +++ .../beam/sdk/io/gcp/bigquery/BigQueryIO.java | 47 +++++++++++++++ .../sdk/io/gcp/bigquery/CreateTables.java | 15 +++-- .../bigquery/DynamicDestinationsHelpers.java | 27 ++++++++- .../sdk/io/gcp/bigquery/TableDestination.java | 39 +++++++++++- .../gcp/bigquery/TableDestinationCoder.java | 2 + .../gcp/bigquery/TableDestinationCoderV2.java | 59 +++++++++++++++++++ .../beam/sdk/io/gcp/bigquery/WriteTables.java | 7 ++- .../sdk/io/gcp/bigquery/BigQueryIOTest.java | 50 ++++++++++++++++ .../sdk/io/gcp/bigquery/FakeJobService.java | 32 ++++++++-- .../sdk/io/gcp/bigquery/TableContainer.java | 2 + 14 files changed, 278 insertions(+), 18 deletions(-) create mode 100644 sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TableDestinationCoderV2.java diff --git a/pom.xml b/pom.xml index 81c80034267a7..b563f8c8f59d2 100644 --- a/pom.xml +++ b/pom.xml @@ -107,7 +107,7 @@ 2.24.0 1.0.0-rc2 1.8.2 - v2-rev295-1.22.0 + v2-rev355-1.22.0 0.9.7.1 v1-rev6-1.22.0 0.1.0 diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/DefaultCoderCloudObjectTranslatorRegistrar.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/DefaultCoderCloudObjectTranslatorRegistrar.java index 5d42a5fedde01..ff89933dadef8 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/DefaultCoderCloudObjectTranslatorRegistrar.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/DefaultCoderCloudObjectTranslatorRegistrar.java @@ -48,6 +48,7 @@ import org.apache.beam.sdk.coders.VarIntCoder; import org.apache.beam.sdk.coders.VoidCoder; import org.apache.beam.sdk.io.gcp.bigquery.TableDestinationCoder; +import org.apache.beam.sdk.io.gcp.bigquery.TableDestinationCoderV2; import org.apache.beam.sdk.io.gcp.bigquery.TableRowJsonCoder; /** @@ -97,6 +98,7 @@ public class DefaultCoderCloudObjectTranslatorRegistrar RandomAccessDataCoder.class, StringUtf8Coder.class, TableDestinationCoder.class, + TableDestinationCoderV2.class, TableRowJsonCoder.class, TextualIntegerCoder.class, VarIntCoder.class, diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BatchLoads.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BatchLoads.java index 0a1306dfc1151..76cf7e84d5913 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BatchLoads.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BatchLoads.java @@ -266,7 +266,7 @@ private WriteResult expandTriggered(PCollection> inpu .apply(WithKeys.>of((Void) null)) .setCoder( KvCoder.of( - VoidCoder.of(), KvCoder.of(TableDestinationCoder.of(), StringUtf8Coder.of()))) + VoidCoder.of(), KvCoder.of(TableDestinationCoderV2.of(), StringUtf8Coder.of()))) .apply(GroupByKey.>create()) .apply(Values.>>create()) .apply( @@ -323,7 +323,7 @@ public WriteResult expandUntriggered(PCollection> inp tempTables .apply("ReifyRenameInput", new ReifyAsIterable>()) - .setCoder(IterableCoder.of(KvCoder.of(TableDestinationCoder.of(), StringUtf8Coder.of()))) + .setCoder(IterableCoder.of(KvCoder.of(TableDestinationCoderV2.of(), StringUtf8Coder.of()))) .apply( "WriteRenameUntriggered", ParDo.of( diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryHelpers.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryHelpers.java index 78dcdde097ad4..7f9e27ac4d951 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryHelpers.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryHelpers.java @@ -24,6 +24,7 @@ import com.google.api.services.bigquery.model.JobStatus; import com.google.api.services.bigquery.model.TableReference; import com.google.api.services.bigquery.model.TableSchema; +import com.google.api.services.bigquery.model.TimePartitioning; import com.google.cloud.hadoop.util.ApiErrorExtractor; import com.google.common.annotations.VisibleForTesting; import com.google.common.hash.Hashing; @@ -291,6 +292,13 @@ public TableReference apply(String from) { } } + static class TimePartitioningToJson implements SerializableFunction { + @Override + public String apply(TimePartitioning partitioning) { + return toJsonString(partitioning); + } + } + static String createJobIdToken(String jobName, String stepUuid) { return String.format("beam_job_%s_%s", stepUuid, jobName.replaceAll("-", "")); } 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 feb085db6b52f..29828e439b114 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 @@ -31,6 +31,7 @@ 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.api.services.bigquery.model.TimePartitioning; import com.google.auto.value.AutoValue; import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Predicates; @@ -60,9 +61,11 @@ import org.apache.beam.sdk.io.gcp.bigquery.BigQueryHelpers.TableRefToJson; import org.apache.beam.sdk.io.gcp.bigquery.BigQueryHelpers.TableSchemaToJsonSchema; import org.apache.beam.sdk.io.gcp.bigquery.BigQueryHelpers.TableSpecToTableRef; +import org.apache.beam.sdk.io.gcp.bigquery.BigQueryHelpers.TimePartitioningToJson; import org.apache.beam.sdk.io.gcp.bigquery.BigQueryServices.DatasetService; import org.apache.beam.sdk.io.gcp.bigquery.BigQueryServices.JobService; import org.apache.beam.sdk.io.gcp.bigquery.DynamicDestinationsHelpers.ConstantSchemaDestinations; +import org.apache.beam.sdk.io.gcp.bigquery.DynamicDestinationsHelpers.ConstantTimePartitioningDestinations; import org.apache.beam.sdk.io.gcp.bigquery.DynamicDestinationsHelpers.SchemaFromViewDestinations; import org.apache.beam.sdk.io.gcp.bigquery.DynamicDestinationsHelpers.TableFunctionDestinations; import org.apache.beam.sdk.options.PipelineOptions; @@ -824,6 +827,7 @@ public enum Method { @Nullable abstract DynamicDestinations getDynamicDestinations(); @Nullable abstract PCollectionView> getSchemaFromView(); @Nullable abstract ValueProvider getJsonSchema(); + @Nullable abstract ValueProvider getJsonTimePartitioning(); abstract CreateDisposition getCreateDisposition(); abstract WriteDisposition getWriteDisposition(); /** Table description. Default is empty. */ @@ -854,6 +858,7 @@ abstract Builder setTableFunction( abstract Builder setDynamicDestinations(DynamicDestinations dynamicDestinations); abstract Builder setSchemaFromView(PCollectionView> view); abstract Builder setJsonSchema(ValueProvider jsonSchema); + abstract Builder setJsonTimePartitioning(ValueProvider jsonTimePartitioning); abstract Builder setCreateDisposition(CreateDisposition createDisposition); abstract Builder setWriteDisposition(WriteDisposition writeDisposition); abstract Builder setTableDescription(String tableDescription); @@ -1022,6 +1027,33 @@ public Write withSchemaFromView(PCollectionView> view) { return toBuilder().setSchemaFromView(view).build(); } + /** + * Allows newly created tables to include a {@link TimePartitioning} class. Can only be used + * when writing to a single table. If {@link #to(SerializableFunction)} or + * {@link #to(DynamicDestinations)} is used to write dynamic tables, time partitioning can be + * directly in the returned {@link TableDestination}. + */ + public Write withTimePartitioning(TimePartitioning partitioning) { + return withJsonTimePartitioning( + StaticValueProvider.of(BigQueryHelpers.toJsonString(partitioning))); + } + + /** + * Like {@link #withTimePartitioning(TimePartitioning)} but using a deferred + * {@link ValueProvider}. + */ + public Write withTimePartitioning(ValueProvider partition) { + return withJsonTimePartitioning(NestedValueProvider.of( + partition, new TimePartitioningToJson())); + } + + /** + * The same as {@link #withTimePartitioning}, but takes a JSON-serialized object. + */ + public Write withJsonTimePartitioning(ValueProvider partition) { + return toBuilder().setJsonTimePartitioning(partition).build(); + } + /** Specifies whether the table should be created if it does not exist. */ public Write withCreateDisposition(CreateDisposition createDisposition) { return toBuilder().setCreateDisposition(createDisposition).build(); @@ -1183,6 +1215,15 @@ public WriteResult expand(PCollection input) { input.isBounded(), method); } + if (getJsonTimePartitioning() != null) { + checkArgument(getDynamicDestinations() == null, + "The supplied DynamicDestinations object can directly set TimePartitioning." + + " There is no need to call BigQueryIO.Write.withTimePartitioning."); + checkArgument(getTableFunction() == null, + "The supplied getTableFunction object can directly set TimePartitioning." + + " There is no need to call BigQueryIO.Write.withTimePartitioning."); + } + DynamicDestinations dynamicDestinations = getDynamicDestinations(); if (dynamicDestinations == null) { if (getJsonTableRef() != null) { @@ -1205,6 +1246,12 @@ public WriteResult expand(PCollection input) { (DynamicDestinations) dynamicDestinations, getSchemaFromView()); } + + // Wrap with a DynamicDestinations class that will provide the proper TimePartitioning. + if (getJsonTimePartitioning() != null) { + dynamicDestinations = new ConstantTimePartitioningDestinations( + dynamicDestinations, getJsonTimePartitioning()); + } } return expandTyped(input, dynamicDestinations); } diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/CreateTables.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/CreateTables.java index 3dc10b08a0242..7f83b83b4026c 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/CreateTables.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/CreateTables.java @@ -73,7 +73,7 @@ private CreateTables( } CreateTables withTestServices(BigQueryServices bqServices) { - return new CreateTables(createDisposition, bqServices, dynamicDestinations); + return new CreateTables<>(createDisposition, bqServices, dynamicDestinations); } @Override @@ -124,11 +124,14 @@ private void possibleCreateTable( DatasetService datasetService = bqServices.getDatasetService(options); if (!createdTables.contains(tableSpec)) { if (datasetService.getTable(tableReference) == null) { - datasetService.createTable( - new Table() - .setTableReference(tableReference) - .setSchema(tableSchema) - .setDescription(tableDescription)); + Table table = new Table() + .setTableReference(tableReference) + .setSchema(tableSchema) + .setDescription(tableDescription); + if (tableDestination.getTimePartitioning() != null) { + table.setTimePartitioning(tableDestination.getTimePartitioning()); + } + datasetService.createTable(table); } createdTables.add(tableSpec); } diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/DynamicDestinationsHelpers.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/DynamicDestinationsHelpers.java index 530e2b6feb6d7..818ea34253f02 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/DynamicDestinationsHelpers.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/DynamicDestinationsHelpers.java @@ -108,7 +108,7 @@ public TableDestination getTable(TableDestination destination) { @Override public Coder getDestinationCoder() { - return TableDestinationCoder.of(); + return TableDestinationCoderV2.of(); } } @@ -164,6 +164,31 @@ public TableSchema getSchema(TableDestination destination) { } } + static class ConstantTimePartitioningDestinations + extends DelegatingDynamicDestinations { + + @Nullable + private final ValueProvider jsonTimePartitioning; + + ConstantTimePartitioningDestinations(DynamicDestinations inner, + ValueProvider jsonTimePartitioning) { + super(inner); + this.jsonTimePartitioning = jsonTimePartitioning; + } + + @Override + public TableDestination getDestination(ValueInSingleWindow element) { + TableDestination destination = super.getDestination(element); + return new TableDestination(destination.getTableSpec(), destination.getTableDescription(), + jsonTimePartitioning.get()); + } + + @Override + public Coder getDestinationCoder() { + return TableDestinationCoderV2.of(); + } + } + /** * Takes in a side input mapping tablespec to json table schema, and always returns the * matching schema from the side input. diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TableDestination.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TableDestination.java index ecf35d8d80e72..79f1b22e83927 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TableDestination.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TableDestination.java @@ -19,6 +19,7 @@ package org.apache.beam.sdk.io.gcp.bigquery; import com.google.api.services.bigquery.model.TableReference; +import com.google.api.services.bigquery.model.TimePartitioning; import java.io.Serializable; import java.util.Objects; import javax.annotation.Nullable; @@ -31,18 +32,38 @@ public class TableDestination implements Serializable { private final String tableSpec; @Nullable private final String tableDescription; + @Nullable + private final String jsonTimePartitioning; public TableDestination(String tableSpec, @Nullable String tableDescription) { - this.tableSpec = tableSpec; - this.tableDescription = tableDescription; + this(tableSpec, tableDescription, (String) null); } public TableDestination(TableReference tableReference, @Nullable String tableDescription) { - this.tableSpec = BigQueryHelpers.toTableSpec(tableReference); + this(tableReference, tableDescription, null); + } + + public TableDestination(TableReference tableReference, @Nullable String tableDescription, + TimePartitioning timePartitioning) { + this(BigQueryHelpers.toTableSpec(tableReference), tableDescription, + timePartitioning != null ? BigQueryHelpers.toJsonString(timePartitioning) : null); + } + + public TableDestination(String tableSpec, @Nullable String tableDescription, + TimePartitioning timePartitioning) { + this(tableSpec, tableDescription, + timePartitioning != null ? BigQueryHelpers.toJsonString(timePartitioning) : null); + } + + public TableDestination(String tableSpec, @Nullable String tableDescription, + @Nullable String jsonTimePartitioning) { + this.tableSpec = tableSpec; this.tableDescription = tableDescription; + this.jsonTimePartitioning = jsonTimePartitioning; } + public String getTableSpec() { return tableSpec; } @@ -51,6 +72,18 @@ public TableReference getTableReference() { return BigQueryHelpers.parseTableSpec(tableSpec); } + public String getJsonTimePartitioning() { + return jsonTimePartitioning; + } + + public TimePartitioning getTimePartitioning() { + if (jsonTimePartitioning == null) { + return null; + } else { + return BigQueryHelpers.fromJsonString(jsonTimePartitioning, TimePartitioning.class); + } + } + @Nullable public String getTableDescription() { return tableDescription; diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TableDestinationCoder.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TableDestinationCoder.java index f034a030da605..2bfc2cac6aa23 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TableDestinationCoder.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TableDestinationCoder.java @@ -33,6 +33,8 @@ public class TableDestinationCoder extends AtomicCoder { private static final Coder tableSpecCoder = StringUtf8Coder.of(); private static final Coder tableDescriptionCoder = NullableCoder.of(StringUtf8Coder.of()); + private TableDestinationCoder() {} + public static TableDestinationCoder of() { return INSTANCE; } diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TableDestinationCoderV2.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TableDestinationCoderV2.java new file mode 100644 index 0000000000000..5bdab0d909b59 --- /dev/null +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/TableDestinationCoderV2.java @@ -0,0 +1,59 @@ +/* + * 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 java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import org.apache.beam.sdk.coders.AtomicCoder; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.coders.NullableCoder; +import org.apache.beam.sdk.coders.StringUtf8Coder; + +/** + * A {@link Coder} for {@link TableDestination} that includes time partitioning information. This + * is a new coder (instead of extending the old {@link TableDestinationCoder}) for compatibility + * reasons. The old coder is kept around for the same compatibility reasons. + */ +public class TableDestinationCoderV2 extends AtomicCoder { + private static final TableDestinationCoderV2 INSTANCE = new TableDestinationCoderV2(); + private static final Coder timePartitioningCoder = NullableCoder.of(StringUtf8Coder.of()); + + public static TableDestinationCoderV2 of() { + return INSTANCE; + } + + @Override + public void encode(TableDestination value, OutputStream outStream) throws IOException { + TableDestinationCoder.of().encode(value, outStream); + timePartitioningCoder.encode(value.getJsonTimePartitioning(), outStream); + } + + @Override + public TableDestination decode(InputStream inStream) throws IOException { + TableDestination destination = TableDestinationCoder.of().decode(inStream); + String jsonTimePartitioning = timePartitioningCoder.decode(inStream); + return new TableDestination( + destination.getTableSpec(), destination.getTableDescription(), jsonTimePartitioning); + } + + @Override + public void verifyDeterministic() throws NonDeterministicException {} +} diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/WriteTables.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/WriteTables.java index c8fab75f75739..a646f17513c2b 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/WriteTables.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/WriteTables.java @@ -23,6 +23,7 @@ import com.google.api.services.bigquery.model.JobReference; import com.google.api.services.bigquery.model.TableReference; import com.google.api.services.bigquery.model.TableSchema; +import com.google.api.services.bigquery.model.TimePartitioning; import com.google.common.base.Strings; import com.google.common.collect.ImmutableList; import com.google.common.collect.Lists; @@ -135,6 +136,7 @@ public void processElement(ProcessContext c) throws Exception { bqServices.getDatasetService(c.getPipelineOptions().as(BigQueryOptions.class)), jobIdPrefix, tableReference, + tableDestination.getTimePartitioning(), tableSchema, partitionFiles, writeDisposition, @@ -150,6 +152,7 @@ private void load( DatasetService datasetService, String jobIdPrefix, TableReference ref, + TimePartitioning timePartitioning, @Nullable TableSchema schema, List gcsUris, WriteDisposition writeDisposition, @@ -164,7 +167,9 @@ private void load( .setWriteDisposition(writeDisposition.name()) .setCreateDisposition(createDisposition.name()) .setSourceFormat("NEWLINE_DELIMITED_JSON"); - + if (timePartitioning != null) { + loadConfig.setTimePartitioning(timePartitioning); + } String projectId = ref.getProjectId(); Job lastFailedLoadJob = null; for (int i = 0; i < BatchLoads.MAX_RETRY_JOBS; ++i) { 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 0ece3ee0ba1df..18547cd248484 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 @@ -47,6 +47,7 @@ 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.api.services.bigquery.model.TimePartitioning; import com.google.common.collect.ArrayListMultimap; import com.google.common.collect.HashBasedTable; import com.google.common.collect.ImmutableList; @@ -637,6 +638,55 @@ private void verifySideInputs() { } } + @Test + public void testTimePartitioningStreamingInserts() throws Exception { + testTimePartitioning(Method.STREAMING_INSERTS); + } + + @Test + public void testTimePartitioningBatchLoads() throws Exception { + testTimePartitioning(Method.FILE_LOADS); + } + + public void testTimePartitioning(BigQueryIO.Write.Method insertMethod) throws Exception { + BigQueryOptions bqOptions = TestPipeline.testingPipelineOptions().as(BigQueryOptions.class); + bqOptions.setProject("project-id"); + bqOptions.setTempLocation(testFolder.newFolder("BigQueryIOTest").getAbsolutePath()); + + FakeDatasetService datasetService = new FakeDatasetService(); + FakeBigQueryServices fakeBqServices = + new FakeBigQueryServices() + .withJobService(new FakeJobService()) + .withDatasetService(datasetService); + datasetService.createDataset("project-id", "dataset-id", "", ""); + + Pipeline p = TestPipeline.create(bqOptions); + TableRow row1 = new TableRow().set("name", "a").set("number", "1"); + TableRow row2 = new TableRow().set("name", "b").set("number", "2"); + + TimePartitioning timePartitioning = new TimePartitioning() + .setType("DAY") + .setExpirationMs(1000L); + TableSchema schema = new TableSchema() + .setFields( + ImmutableList.of( + new TableFieldSchema().setName("number").setType("INTEGER"))); + p.apply(Create.of(row1, row1)) + .apply( + BigQueryIO.writeTableRows() + .to("project-id:dataset-id.table-id") + .withTestServices(fakeBqServices) + .withMethod(insertMethod) + .withSchema(schema) + .withTimePartitioning(timePartitioning) + .withoutValidation()); + p.run(); + Table table = datasetService.getTable( + BigQueryHelpers.parseTableSpec("project-id:dataset-id.table-id")); + assertEquals(schema, table.getSchema()); + assertEquals(timePartitioning, table.getTimePartitioning()); + } + @Test @Category({ValidatesRunner.class, UsesTestStream.class}) public void testTriggeredFileLoads() throws Exception { diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/FakeJobService.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/FakeJobService.java index 7d5101d782f9d..cc600d1a51340 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/FakeJobService.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/FakeJobService.java @@ -19,6 +19,7 @@ package org.apache.beam.sdk.io.gcp.bigquery; import static com.google.common.base.Preconditions.checkArgument; +import static com.google.common.base.Preconditions.checkNotNull; import com.google.api.client.json.JsonFactory; import com.google.api.client.util.BackOff; @@ -39,6 +40,7 @@ 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.api.services.bigquery.model.TimePartitioning; import com.google.common.collect.HashBasedTable; import com.google.common.collect.ImmutableList; import com.google.common.collect.Lists; @@ -310,8 +312,13 @@ private JobStatus runLoadJob(JobReference jobRef, JobConfigurationLoad load) if (!validateDispositions(existingTable, createDisposition, writeDisposition)) { return new JobStatus().setState("FAILED").setErrorResult(new ErrorProto()); } - - datasetService.createTable(new Table().setTableReference(destination).setSchema(schema)); + if (existingTable == null) { + existingTable = new Table().setTableReference(destination).setSchema(schema); + if (load.getTimePartitioning() != null) { + existingTable = existingTable.setTimePartitioning(load.getTimePartitioning()); + } + datasetService.createTable(existingTable); + } List rows = Lists.newArrayList(); for (ResourceId filename : sourceFiles) { @@ -331,13 +338,30 @@ private JobStatus runCopyJob(JobConfigurationTableCopy copy) if (!validateDispositions(existingTable, createDisposition, writeDisposition)) { return new JobStatus().setState("FAILED").setErrorResult(new ErrorProto()); } - + TimePartitioning partitioning = null; + TableSchema schema = null; + boolean first = true; List allRows = Lists.newArrayList(); for (TableReference source : sources) { + Table table = checkNotNull(datasetService.getTable(source)); + if (!first) { + if (partitioning != table.getTimePartitioning()) { + return new JobStatus().setState("FAILED").setErrorResult(new ErrorProto()); + } + if (schema != table.getSchema()) { + return new JobStatus().setState("FAILED").setErrorResult(new ErrorProto()); + } + } + partitioning = table.getTimePartitioning(); + schema = table.getSchema(); + first = false; allRows.addAll(datasetService.getAllRows( source.getProjectId(), source.getDatasetId(), source.getTableId())); } - datasetService.createTable(new Table().setTableReference(destination)); + datasetService.createTable(new Table() + .setTableReference(destination) + .setSchema(schema) + .setTimePartitioning(partitioning)); datasetService.insertAll(destination, allRows, null); return new JobStatus().setState("DONE"); } diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/TableContainer.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/TableContainer.java index 89150698114c2..e016c98c82229 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/TableContainer.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/TableContainer.java @@ -32,6 +32,7 @@ class TableContainer { Long sizeBytes; TableContainer(Table table) { this.table = table; + this.rows = new ArrayList<>(); this.ids = new ArrayList<>(); this.sizeBytes = 0L; @@ -54,6 +55,7 @@ Table getTable() { return table; } + List getRows() { return rows; } From 97810b4b23037fe333af103661bbb15acec96a57 Mon Sep 17 00:00:00 2001 From: Eugene Kirpichov Date: Thu, 17 Aug 2017 19:44:17 -0700 Subject: [PATCH 345/346] Removes unnecessary calls to ValueProvider.isAccessible --- .../java/org/apache/beam/sdk/io/AvroIO.java | 10 +--- .../beam/sdk/io/DefaultFilenamePolicy.java | 25 ++++------ .../org/apache/beam/sdk/io/FileBasedSink.java | 4 +- .../apache/beam/sdk/io/FileBasedSource.java | 25 ++-------- .../org/apache/beam/sdk/io/TFRecordIO.java | 15 +----- .../java/org/apache/beam/sdk/io/TextIO.java | 14 +----- .../org/apache/beam/sdk/io/WriteFiles.java | 6 +-- .../beam/sdk/options/ValueProvider.java | 18 ++++--- .../sdk/transforms/display/DisplayData.java | 8 ++-- .../apache/beam/sdk/io/WriteFilesTest.java | 2 +- .../beam/sdk/options/ValueProviderTest.java | 15 +++--- .../beam/sdk/io/gcp/bigquery/BigQueryIO.java | 47 ++++++++++--------- .../io/gcp/bigquery/BigQueryTableSource.java | 2 - .../beam/sdk/io/gcp/pubsub/PubsubIO.java | 35 +++++++------- .../io/gcp/pubsub/PubsubUnboundedSink.java | 6 +-- .../io/gcp/pubsub/PubsubUnboundedSource.java | 23 +++------ 16 files changed, 95 insertions(+), 160 deletions(-) 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 910d8e2bfc494..9e0422e48e048 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 @@ -989,19 +989,11 @@ public PDone expandTyped( public void populateDisplayData(DisplayData.Builder builder) { super.populateDisplayData(builder); resolveDynamicDestinations().populateDisplayData(builder); - - String tempDirectory = null; - if (getTempDirectory() != null) { - tempDirectory = - getTempDirectory().isAccessible() - ? getTempDirectory().get().toString() - : getTempDirectory().toString(); - } builder .addIfNotDefault( DisplayData.item("numShards", getNumShards()).withLabel("Maximum Output Shards"), 0) .addIfNotNull( - DisplayData.item("tempDirectory", tempDirectory) + DisplayData.item("tempDirectory", getTempDirectory()) .withLabel("Directory for temporary files")); } } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/DefaultFilenamePolicy.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/DefaultFilenamePolicy.java index 1f438d57de003..2f22e82200b42 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/DefaultFilenamePolicy.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/DefaultFilenamePolicy.java @@ -368,26 +368,21 @@ private String paneInfoToString(PaneInfo paneInfo) { @Override public void populateDisplayData(DisplayData.Builder builder) { - String filenamePattern; - if (params.baseFilename.isAccessible()) { - filenamePattern = - String.format("%s%s%s", params.baseFilename.get(), params.shardTemplate, params.suffix); - } else { - filenamePattern = - String.format("%s%s%s", params.baseFilename, params.shardTemplate, params.suffix); - } - - String outputPrefixString = null; - outputPrefixString = + String displayBaseFilename = params.baseFilename.isAccessible() ? params.baseFilename.get().toString() - : params.baseFilename.toString(); - builder.add(DisplayData.item("filenamePattern", filenamePattern).withLabel("Filename Pattern")); - builder.add(DisplayData.item("filePrefix", outputPrefixString).withLabel("Output File Prefix")); - builder.add(DisplayData.item("fileSuffix", params.suffix).withLabel("Output file Suffix")); + : ("(" + params.baseFilename + ")"); + builder.add( + DisplayData.item( + "filenamePattern", + String.format("%s%s%s", displayBaseFilename, params.shardTemplate, params.suffix)) + .withLabel("Filename pattern")); + builder.add( + DisplayData.item("filePrefix", params.baseFilename).withLabel("Output File Prefix")); builder.add( DisplayData.item("shardNameTemplate", params.shardTemplate) .withLabel("Output Shard Name Template")); + builder.add(DisplayData.item("fileSuffix", params.suffix).withLabel("Output file Suffix")); } private static String extractFilename(ResourceId input) { 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 4e2b61c34c09c..d6186478f209c 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 @@ -747,12 +747,10 @@ public FileBasedSink getSink() { @Override public String toString() { - String tempDirectoryStr = - tempDirectory.isAccessible() ? tempDirectory.get().toString() : tempDirectory.toString(); return getClass().getSimpleName() + "{" + "tempDirectory=" - + tempDirectoryStr + + tempDirectory + ", windowedWrites=" + windowedWrites + '}'; 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 7f865deb4639d..f835fa4311f79 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 @@ -211,10 +211,6 @@ public final long getEstimatedSizeBytes(PipelineOptions options) throws IOExcept // This implementation of method getEstimatedSizeBytes is provided to simplify subclasses. Here // we perform the size estimation of files and file patterns using the interface provided by // FileSystem. - checkState( - fileOrPatternSpec.isAccessible(), - "Cannot estimate size of a FileBasedSource with inaccessible file pattern: {}.", - fileOrPatternSpec); String fileOrPattern = fileOrPatternSpec.get(); if (mode == Mode.FILEPATTERN) { @@ -240,10 +236,9 @@ public final long getEstimatedSizeBytes(PipelineOptions options) throws IOExcept public void populateDisplayData(DisplayData.Builder builder) { super.populateDisplayData(builder); if (mode == Mode.FILEPATTERN) { - String patternDisplay = getFileOrPatternSpecProvider().isAccessible() - ? getFileOrPatternSpecProvider().get() - : getFileOrPatternSpecProvider().toString(); - builder.add(DisplayData.item("filePattern", patternDisplay).withLabel("File Pattern")); + builder.add( + DisplayData.item("filePattern", getFileOrPatternSpecProvider()) + .withLabel("File Pattern")); } } @@ -254,10 +249,6 @@ public final List> split( // split a FileBasedSource based on a file pattern to FileBasedSources based on full single // files. For files that can be efficiently seeked, we further split FileBasedSources based on // those files to FileBasedSources based on sub ranges of single files. - checkState( - fileOrPatternSpec.isAccessible(), - "Cannot split a FileBasedSource without access to the file or pattern specification: {}.", - fileOrPatternSpec); String fileOrPattern = fileOrPatternSpec.get(); if (mode == Mode.FILEPATTERN) { @@ -326,10 +317,6 @@ protected boolean isSplittable() throws Exception { public final BoundedReader createReader(PipelineOptions options) throws IOException { // Validate the current source prior to creating a reader for it. this.validate(); - checkState( - fileOrPatternSpec.isAccessible(), - "Cannot create a file reader without access to the file or pattern specification: {}.", - fileOrPatternSpec); String fileOrPattern = fileOrPatternSpec.get(); if (mode == Mode.FILEPATTERN) { @@ -358,13 +345,11 @@ 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 fileString; + return fileOrPatternSpec.toString(); case SINGLE_FILE_OR_SUBRANGE: - return fileString + " range " + super.toString(); + return fileOrPatternSpec + " range " + super.toString(); default: throw new IllegalStateException("Unexpected mode: " + mode); } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/TFRecordIO.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/TFRecordIO.java index c75051fe7bc64..526c50ea06e8d 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/TFRecordIO.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/TFRecordIO.java @@ -195,15 +195,12 @@ protected FileBasedSource getSource() { @Override public void populateDisplayData(DisplayData.Builder builder) { super.populateDisplayData(builder); - - String filepatternDisplay = getFilepattern().isAccessible() - ? getFilepattern().get() : getFilepattern().toString(); builder .add(DisplayData.item("compressionType", getCompressionType().toString()) .withLabel("Compression Type")) .addIfNotDefault(DisplayData.item("validation", getValidate()) .withLabel("Validation Enabled"), true) - .addIfNotNull(DisplayData.item("filePattern", filepatternDisplay) + .addIfNotNull(DisplayData.item("filePattern", getFilepattern()) .withLabel("File Pattern")); } } @@ -360,16 +357,8 @@ public PDone expand(PCollection input) { @Override public void populateDisplayData(DisplayData.Builder builder) { super.populateDisplayData(builder); - - String outputPrefixString = null; - if (getOutputPrefix().isAccessible()) { - ResourceId dir = getOutputPrefix().get(); - outputPrefixString = dir.toString(); - } else { - outputPrefixString = getOutputPrefix().toString(); - } builder - .add(DisplayData.item("filePrefix", outputPrefixString) + .add(DisplayData.item("filePrefix", getOutputPrefix()) .withLabel("Output File Prefix")) .addIfNotNull(DisplayData.item("fileSuffix", getFilenameSuffix()) .withLabel("Output File Suffix")) 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 612f5c5917083..cbc17ff8547ee 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 @@ -399,15 +399,12 @@ private static FileBasedSource wrapWithCompression( @Override public void populateDisplayData(DisplayData.Builder builder) { super.populateDisplayData(builder); - - String filepatternDisplay = getFilepattern().isAccessible() - ? getFilepattern().get() : getFilepattern().toString(); builder .add( DisplayData.item("compressionType", getCompressionType().toString()) .withLabel("Compression Type")) .addIfNotNull( - DisplayData.item("filePattern", filepatternDisplay).withLabel("File Pattern")) + DisplayData.item("filePattern", getFilepattern()).withLabel("File Pattern")) .add( DisplayData.item("emptyMatchTreatment", getEmptyMatchTreatment().toString()) .withLabel("Treatment of filepatterns that match no files")) @@ -904,18 +901,11 @@ public void populateDisplayData(DisplayData.Builder builder) { super.populateDisplayData(builder); resolveDynamicDestinations().populateDisplayData(builder); - String tempDirectory = null; - if (getTempDirectory() != null) { - tempDirectory = - getTempDirectory().isAccessible() - ? getTempDirectory().get().toString() - : getTempDirectory().toString(); - } builder .addIfNotDefault( DisplayData.item("numShards", getNumShards()).withLabel("Maximum Output Shards"), 0) .addIfNotNull( - DisplayData.item("tempDirectory", tempDirectory) + DisplayData.item("tempDirectory", getTempDirectory()) .withLabel("Directory for temporary files")) .addIfNotNull(DisplayData.item("fileHeader", getHeader()).withLabel("File Header")) .addIfNotNull(DisplayData.item("fileFooter", getFooter()).withLabel("File Footer")) diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/WriteFiles.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/WriteFiles.java index 85c5652db4950..7878c7313efb6 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/io/WriteFiles.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/io/WriteFiles.java @@ -205,10 +205,8 @@ public void populateDisplayData(DisplayData.Builder builder) { .include("sink", sink); if (getSharding() != null) { builder.include("sharding", getSharding()); - } else if (getNumShards() != null) { - String numShards = getNumShards().isAccessible() - ? getNumShards().get().toString() : getNumShards().toString(); - builder.add(DisplayData.item("numShards", numShards) + } else { + builder.addIfNotNull(DisplayData.item("numShards", getNumShards()) .withLabel("Fixed Number of Shards")); } } 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 94187a9349a52..15413e850cdf2 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 @@ -101,9 +101,7 @@ public boolean isAccessible() { @Override public String toString() { - return MoreObjects.toStringHelper(this) - .add("value", value) - .toString(); + return String.valueOf(value); } } @@ -160,8 +158,12 @@ public String propertyName() { @Override public String toString() { + if (isAccessible()) { + return String.valueOf(get()); + } return MoreObjects.toStringHelper(this) .add("value", value) + .add("translator", translator.getClass().getSimpleName()) .toString(); } } @@ -226,7 +228,8 @@ static void setRuntimeOptions(PipelineOptions runtimeOptions) { public T get() { PipelineOptions options = optionsMap.get(optionsId); if (options == null) { - throw new RuntimeException("Not called from a runtime context."); + throw new IllegalStateException( + "Value only available at runtime, but accessed from a non-runtime context: " + this); } try { Method method = klass.getMethod(methodName); @@ -249,8 +252,7 @@ public T get() { @Override public boolean isAccessible() { - PipelineOptions options = optionsMap.get(optionsId); - return options != null; + return optionsMap.get(optionsId) != null; } /** @@ -262,10 +264,12 @@ public String propertyName() { @Override public String toString() { + if (isAccessible()) { + return String.valueOf(get()); + } return MoreObjects.toStringHelper(this) .add("propertyName", propertyName) .add("default", defaultValue) - .add("value", isAccessible() ? get() : null) .toString(); } } 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 3c4337bba3dc3..10ef428af1ec2 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 @@ -882,12 +882,12 @@ public static ItemSpec item(String key, @Nullable ValueProvider value) { return item(key, Type.STRING, null); } Type type = inferType(got); - if (type == null) { - throw new RuntimeException(String.format("Unknown value type: %s", got)); + if (type != null) { + return item(key, type, got); } - return item(key, type, got); } - return item(key, Type.STRING, value.toString()); + // General case: not null and type not inferable. Fall back to toString of the VP itself. + return item(key, Type.STRING, String.valueOf(value)); } /** diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/WriteFilesTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/WriteFilesTest.java index 1d4ce08f70cd6..5e0d6856a3c2e 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/io/WriteFilesTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/io/WriteFilesTest.java @@ -518,7 +518,7 @@ public void populateDisplayData(DisplayData.Builder builder) { DisplayData displayData = DisplayData.from(write); assertThat(displayData, hasDisplayItem("sink", sink.getClass())); assertThat(displayData, includesDisplayDataFor("sink", sink)); - assertThat(displayData, hasDisplayItem("numShards", "1")); + assertThat(displayData, hasDisplayItem("numShards", 1)); } @Test 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 e596cc106d40e..7bbbf7e49b90a 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 @@ -88,7 +88,7 @@ public void testStaticValueProvider() { ValueProvider provider = StaticValueProvider.of("foo"); assertEquals("foo", provider.get()); assertTrue(provider.isAccessible()); - assertEquals("StaticValueProvider{value=foo}", provider.toString()); + assertEquals("foo", provider.toString()); } @Test @@ -97,8 +97,9 @@ public void testNoDefaultRuntimeProvider() { ValueProvider provider = options.getFoo(); assertFalse(provider.isAccessible()); - expectedException.expect(RuntimeException.class); - expectedException.expectMessage("Not called from a runtime context"); + expectedException.expect(IllegalStateException.class); + expectedException.expectMessage("Value only available at runtime"); + expectedException.expectMessage("foo"); provider.get(); } @@ -108,7 +109,7 @@ public void testRuntimePropertyName() { ValueProvider provider = options.getFoo(); assertEquals("foo", ((RuntimeValueProvider) provider).propertyName()); assertEquals( - "RuntimeValueProvider{propertyName=foo, default=null, value=null}", + "RuntimeValueProvider{propertyName=foo, default=null}", provider.toString()); } @@ -239,9 +240,7 @@ public String apply(String from) { }); assertTrue(nvp.isAccessible()); assertEquals("foobar", nvp.get()); - assertEquals( - "NestedValueProvider{value=StaticValueProvider{value=foo}}", - nvp.toString()); + assertEquals("foobar", nvp.toString()); } @Test @@ -266,7 +265,7 @@ public String apply(String from) { assertEquals("bar", ((NestedValueProvider) doubleNvp).propertyName()); assertFalse(nvp.isAccessible()); expectedException.expect(RuntimeException.class); - expectedException.expectMessage("Not called from a runtime context"); + expectedException.expectMessage("Value only available at runtime"); nvp.get(); } 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 29828e439b114..1e0ab30525ac7 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 @@ -451,8 +451,7 @@ Read withTestServices(BigQueryServices testServices) { private BigQuerySourceBase createSource(String jobUuid) { BigQuerySourceBase source; - if (getQuery() == null - || (getQuery().isAccessible() && Strings.isNullOrEmpty(getQuery().get()))) { + if (getQuery() == null) { source = BigQueryTableSource.create(jobUuid, getTableProvider(), getBigQueryServices()); } else { source = @@ -517,26 +516,30 @@ public void validate(PipelineOptions options) { // Note that a table or query check can fail if the table or dataset are created by // 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 (getValidate() && table != null && table.isAccessible() - && table.get().getProjectId() != 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); - BigQueryHelpers.verifyDatasetPresence(datasetService, table.get()); - BigQueryHelpers.verifyTablePresence(datasetService, table.get()); - } else if (getValidate() && getQuery() != null) { - checkState(getQuery().isAccessible(), "Cannot call validate if query is dynamically set."); - JobService jobService = getBigQueryServices().getJobService(bqOptions); - try { - jobService.dryRunQuery( - bqOptions.getProject(), - new JobConfigurationQuery() - .setQuery(getQuery().get()) - .setFlattenResults(getFlattenResults()) - .setUseLegacySql(getUseLegacySql())); - } catch (Exception e) { - throw new IllegalArgumentException( - String.format(QUERY_VALIDATION_FAILURE_ERROR, getQuery().get()), e); + if (getValidate()) { + if (table != null) { + checkState(table.isAccessible(), "Cannot call validate if table is dynamically set."); + } + if (table != null && table.get().getProjectId() != null) { + // Check for source table presence for early failure notification. + DatasetService datasetService = getBigQueryServices().getDatasetService(bqOptions); + BigQueryHelpers.verifyDatasetPresence(datasetService, table.get()); + BigQueryHelpers.verifyTablePresence(datasetService, table.get()); + } else if (getQuery() != null) { + checkState( + getQuery().isAccessible(), "Cannot call validate if query is dynamically set."); + JobService jobService = getBigQueryServices().getJobService(bqOptions); + try { + jobService.dryRunQuery( + bqOptions.getProject(), + new JobConfigurationQuery() + .setQuery(getQuery().get()) + .setFlattenResults(getFlattenResults()) + .setUseLegacySql(getUseLegacySql())); + } catch (Exception e) { + throw new IllegalArgumentException( + String.format(QUERY_VALIDATION_FAILURE_ERROR, getQuery().get()), e); + } } } } diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryTableSource.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryTableSource.java index 1d45641f978ac..52b8259915292 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryTableSource.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryTableSource.java @@ -63,7 +63,6 @@ private BigQueryTableSource( @Override protected TableReference getTableToExtract(BigQueryOptions bqOptions) throws IOException { - checkState(jsonTable.isAccessible()); TableReference tableReference = BigQueryIO.JSON_FACTORY.fromString(jsonTable.get(), TableReference.class); return setDefaultProjectIfAbsent(bqOptions, tableReference); @@ -94,7 +93,6 @@ private TableReference setDefaultProjectIfAbsent( @Override public BoundedReader createReader(PipelineOptions options) throws IOException { BigQueryOptions bqOptions = options.as(BigQueryOptions.class); - checkState(jsonTable.isAccessible()); TableReference tableRef = BigQueryIO.JSON_FACTORY.fromString(jsonTable.get(), TableReference.class); return new BigQueryReader(this, bqServices.getReaderFromTable(bqOptions, tableRef)); diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubIO.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubIO.java index 46c2df4b5f18f..e3780b4c089b8 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubIO.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubIO.java @@ -146,17 +146,11 @@ private static void validatePubsubName(String name) { private static void populateCommonDisplayData(DisplayData.Builder builder, String timestampAttribute, String idAttribute, ValueProvider topic) { builder - .addIfNotNull(DisplayData.item("timestampAttribute", timestampAttribute) - .withLabel("Timestamp Attribute")) - .addIfNotNull(DisplayData.item("idAttribute", idAttribute) - .withLabel("ID Attribute")); - - if (topic != null) { - String topicString = topic.isAccessible() ? topic.get().asPath() - : topic.toString(); - builder.add(DisplayData.item("topic", topicString) - .withLabel("Pubsub Topic")); - } + .addIfNotNull( + DisplayData.item("timestampAttribute", timestampAttribute) + .withLabel("Timestamp Attribute")) + .addIfNotNull(DisplayData.item("idAttribute", idAttribute).withLabel("ID Attribute")) + .addIfNotNull(DisplayData.item("topic", topic).withLabel("Pubsub Topic")); } /** @@ -263,6 +257,11 @@ public String asPath() { return subscription; } } + + @Override + public String toString() { + return asPath(); + } } /** @@ -428,6 +427,11 @@ public String asPath() { return topic; } } + + @Override + public String toString() { + return asPath(); + } } /** Returns A {@link PTransform} that continuously reads from a Google Cloud Pub/Sub stream. */ @@ -734,13 +738,8 @@ public void populateDisplayData(DisplayData.Builder builder) { super.populateDisplayData(builder); populateCommonDisplayData( builder, getTimestampAttribute(), getIdAttribute(), getTopicProvider()); - - if (getSubscriptionProvider() != null) { - String subscriptionString = getSubscriptionProvider().isAccessible() - ? getSubscriptionProvider().get().asPath() : getSubscriptionProvider().toString(); - builder.add(DisplayData.item("subscription", subscriptionString) - .withLabel("Pubsub Subscription")); - } + builder.addIfNotNull(DisplayData.item("subscription", getSubscriptionProvider()) + .withLabel("Pubsub Subscription")); } } diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubUnboundedSink.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubUnboundedSink.java index ad38e2800f9e4..a8f6fa270b9a8 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubUnboundedSink.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubUnboundedSink.java @@ -295,11 +295,7 @@ public void finishBundle() throws Exception { @Override public void populateDisplayData(Builder builder) { super.populateDisplayData(builder); - String topicString = - topic == null ? null - : topic.isAccessible() ? topic.get().getPath() - : topic.toString(); - builder.add(DisplayData.item("topic", topicString)); + builder.add(DisplayData.item("topic", topic)); builder.add(DisplayData.item("transport", pubsubFactory.getKind())); builder.addIfNotNull(DisplayData.item("timestampAttribute", timestampAttribute)); builder.addIfNotNull(DisplayData.item("idAttribute", idAttribute)); diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubUnboundedSource.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubUnboundedSource.java index 8da6ff4f80c5f..bf3a1217253f5 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubUnboundedSource.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsub/PubsubUnboundedSource.java @@ -1222,21 +1222,12 @@ public void processElement(ProcessContext c) throws Exception { @Override public void populateDisplayData(Builder builder) { super.populateDisplayData(builder); - if (subscription != null) { - String subscriptionString = subscription.isAccessible() - ? subscription.get().getPath() - : subscription.toString(); - builder.add(DisplayData.item("subscription", subscriptionString)); - } - if (topic != null) { - String topicString = topic.isAccessible() - ? topic.get().getPath() - : topic.toString(); - builder.add(DisplayData.item("topic", topicString)); - } - builder.add(DisplayData.item("transport", pubsubFactory.getKind())); - builder.addIfNotNull(DisplayData.item("timestampAttribute", timestampAttribute)); - builder.addIfNotNull(DisplayData.item("idAttribute", idAttribute)); + builder + .addIfNotNull(DisplayData.item("subscription", subscription)) + .addIfNotNull(DisplayData.item("topic", topic)) + .add(DisplayData.item("transport", pubsubFactory.getKind())) + .addIfNotNull(DisplayData.item("timestampAttribute", timestampAttribute)) + .addIfNotNull(DisplayData.item("idAttribute", idAttribute)); } } @@ -1416,8 +1407,6 @@ private SubscriptionPath createRandomSubscription(PipelineOptions options) { try (PubsubClient pubsubClient = pubsubFactory.newClient( timestampAttribute, idAttribute, 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.get(), topic.get(), DEAULT_ACK_TIMEOUT_SEC); From 43fafcd18c755ec38cb0c8e469927fe6323c97f6 Mon Sep 17 00:00:00 2001 From: Nawaid Shamim Date: Tue, 11 Jul 2017 15:19:43 +0100 Subject: [PATCH 346/346] BEAM-2581: KinesisClientProvider interface needs to be public --- .../org/apache/beam/sdk/io/kinesis/KinesisClientProvider.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/KinesisClientProvider.java b/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/KinesisClientProvider.java index b5b721e23c544..c48f9cc0d3db0 100644 --- a/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/KinesisClientProvider.java +++ b/sdks/java/io/kinesis/src/main/java/org/apache/beam/sdk/io/kinesis/KinesisClientProvider.java @@ -27,7 +27,7 @@ *

              Please note, that any instance of {@link KinesisClientProvider} must be * {@link Serializable} to ensure it can be sent to worker machines. */ -interface KinesisClientProvider extends Serializable { +public interface KinesisClientProvider extends Serializable { AmazonKinesis get(); }